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/04/07 14:56:34 UTC

[1/6] incubator-metron git commit: METRON-93: Generalize the HBase threat intel infrastructure to support enrichments closes apache/incubator-metron#64

Repository: incubator-metron
Updated Branches:
  refs/heads/master e59b1a31d -> 9621c55ec


http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/9621c55e/metron-streaming/Metron-Testing/src/main/java/org/apache/metron/integration/EnrichmentIntegrationTest.java
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-Testing/src/main/java/org/apache/metron/integration/EnrichmentIntegrationTest.java b/metron-streaming/Metron-Testing/src/main/java/org/apache/metron/integration/EnrichmentIntegrationTest.java
index ee4077b..fa140ed 100644
--- a/metron-streaming/Metron-Testing/src/main/java/org/apache/metron/integration/EnrichmentIntegrationTest.java
+++ b/metron-streaming/Metron-Testing/src/main/java/org/apache/metron/integration/EnrichmentIntegrationTest.java
@@ -18,19 +18,19 @@
 package org.apache.metron.integration;
 
 import com.fasterxml.jackson.core.type.TypeReference;
-import com.google.common.base.Function;
-import com.google.common.base.Joiner;
-import com.google.common.base.Predicate;
-import com.google.common.base.Predicates;
-import com.google.common.base.Splitter;
+import com.google.common.base.*;
+import com.google.common.collect.ImmutableList;
+
 import com.google.common.collect.Iterables;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.client.HTableInterface;
 import org.apache.metron.Constants;
 import org.apache.metron.domain.Configurations;
+import org.apache.metron.domain.SensorEnrichmentConfig;
 import org.apache.metron.hbase.TableProvider;
-import org.apache.metron.hbase.converters.threatintel.ThreatIntelKey;
-import org.apache.metron.hbase.converters.threatintel.ThreatIntelValue;
+import org.apache.metron.hbase.converters.enrichment.EnrichmentKey;
+import org.apache.metron.hbase.converters.enrichment.EnrichmentValue;
+import org.apache.metron.integration.util.EnrichmentHelper;
 import org.apache.metron.integration.util.TestUtils;
 import org.apache.metron.integration.util.UnitTestHelper;
 import org.apache.metron.integration.util.integration.ComponentRunner;
@@ -40,8 +40,8 @@ import org.apache.metron.integration.util.integration.components.FluxTopologyCom
 import org.apache.metron.integration.util.integration.components.KafkaWithZKComponent;
 import org.apache.metron.integration.util.mock.MockGeoAdapter;
 import org.apache.metron.integration.util.mock.MockHTable;
-import org.apache.metron.integration.util.threatintel.ThreatIntelHelper;
 import org.apache.metron.reference.lookup.LookupKV;
+
 import org.apache.metron.util.SampleUtil;
 import org.apache.metron.utils.JSONUtils;
 import org.junit.Assert;
@@ -63,11 +63,16 @@ import java.util.Stack;
 public abstract class EnrichmentIntegrationTest extends BaseIntegrationTest {
   private static final String SRC_IP = "ip_src_addr";
   private static final String DST_IP = "ip_dst_addr";
+  private static final String MALICIOUS_IP_TYPE = "malicious_ip";
+  private static final String PLAYFUL_CLASSIFICATION_TYPE = "playful_classification";
+  private static final Map<String, String> PLAYFUL_ENRICHMENT = new HashMap<String, String>() {{
+    put("orientation", "north");
+  }};
   private String fluxPath = "../Metron-Topologies/src/main/resources/Metron_Configs/topologies/enrichment/test.yaml";
   protected String hdfsDir = "target/enrichmentIntegrationTest/hdfs";
   private String sampleParsedPath = "../Metron-Testing/src/main/resources/sample/data/SampleParsed/YafExampleParsed";
   private String sampleIndexedPath = "../Metron-Testing/src/main/resources/sample/data/SampleIndexed/YafIndexed";
-  private Map<String, String> sensorConfigs = new HashMap<>();
+
 
   public static class Provider implements TableProvider, Serializable {
     MockHTable.Provider  provider = new MockHTable.Provider();
@@ -140,20 +145,28 @@ public abstract class EnrichmentIntegrationTest extends BaseIntegrationTest {
   public void test() throws Exception {
     cleanHdfsDir(hdfsDir);
     final Configurations configurations = SampleUtil.getSampleConfigs();
+    final String dateFormat = "yyyy.MM.dd.hh";
     final List<byte[]> inputMessages = TestUtils.readSampleData(sampleParsedPath);
     final String cf = "cf";
-    final String trackerHBaseTable = "tracker";
-    final String ipThreatIntelTable = "ip_threat_intel";
+    final String trackerHBaseTableName = "tracker";
+    final String threatIntelTableName = "threat_intel";
+    final String enrichmentsTableName = "enrichments";
     final Properties topologyProperties = new Properties() {{
       setProperty("org.apache.metron.enrichment.host.known_hosts", "[{\"ip\":\"10.1.128.236\", \"local\":\"YES\", \"type\":\"webserver\", \"asset_value\" : \"important\"},\n" +
               "{\"ip\":\"10.1.128.237\", \"local\":\"UNKNOWN\", \"type\":\"unknown\", \"asset_value\" : \"important\"},\n" +
               "{\"ip\":\"10.60.10.254\", \"local\":\"YES\", \"type\":\"printer\", \"asset_value\" : \"important\"},\n" +
               "{\"ip\":\"10.0.2.15\", \"local\":\"YES\", \"type\":\"printer\", \"asset_value\" : \"important\"}]");
       setProperty("hbase.provider.impl","" + Provider.class.getName());
-      setProperty("threat.intel.tracker.table", trackerHBaseTable);
+      setProperty("threat.intel.tracker.table", trackerHBaseTableName);
       setProperty("threat.intel.tracker.cf", cf);
-      setProperty("threat.intel.ip.table", ipThreatIntelTable);
-      setProperty("threat.intel.ip.cf", cf);
+      setProperty("threat.intel.simple.hbase.table", threatIntelTableName);
+      setProperty("threat.intel.simple.hbase.cf", cf);
+      setProperty("enrichment.simple.hbase.table", enrichmentsTableName);
+      setProperty("enrichment.simple.hbase.cf", cf);
+      setProperty("es.clustername", "metron");
+      setProperty("es.port", "9300");
+      setProperty("es.ip", "localhost");
+      setProperty("index.date.format", dateFormat);
       setProperty("index.hdfs.output", hdfsDir);
     }};
     setAdditionalProperties(topologyProperties);
@@ -162,12 +175,18 @@ public abstract class EnrichmentIntegrationTest extends BaseIntegrationTest {
     }});
 
     //create MockHBaseTables
-    final MockHTable trackerTable = (MockHTable)MockHTable.Provider.addToCache(trackerHBaseTable, cf);
-    final MockHTable ipTable = (MockHTable)MockHTable.Provider.addToCache(ipThreatIntelTable, cf);
-    ThreatIntelHelper.INSTANCE.load(ipTable, cf, new ArrayList<LookupKV<ThreatIntelKey, ThreatIntelValue>>(){{
-      add(new LookupKV<>(new ThreatIntelKey("10.0.2.3"), new ThreatIntelValue(new HashMap<String, String>())));
+    final MockHTable trackerTable = (MockHTable)MockHTable.Provider.addToCache(trackerHBaseTableName, cf);
+    final MockHTable threatIntelTable = (MockHTable)MockHTable.Provider.addToCache(threatIntelTableName, cf);
+    EnrichmentHelper.INSTANCE.load(threatIntelTable, cf, new ArrayList<LookupKV<EnrichmentKey, EnrichmentValue>>(){{
+      add(new LookupKV<>(new EnrichmentKey(MALICIOUS_IP_TYPE, "10.0.2.3"), new EnrichmentValue(new HashMap<String, String>())));
+    }});
+    final MockHTable enrichmentTable = (MockHTable)MockHTable.Provider.addToCache(enrichmentsTableName, cf);
+    EnrichmentHelper.INSTANCE.load(enrichmentTable, cf, new ArrayList<LookupKV<EnrichmentKey, EnrichmentValue>>(){{
+      add(new LookupKV<>(new EnrichmentKey(PLAYFUL_CLASSIFICATION_TYPE, "10.0.2.3")
+                        , new EnrichmentValue(PLAYFUL_ENRICHMENT )
+                        )
+         );
     }});
-
     FluxTopologyComponent fluxComponent = new FluxTopologyComponent.Builder()
             .withTopologyLocation(new File(fluxPath))
             .withTopologyName("test")
@@ -195,6 +214,7 @@ public abstract class EnrichmentIntegrationTest extends BaseIntegrationTest {
       List<Map<String, Object>> cleanedDocs = cleanDocs(docs);
       validateAll(cleanedDocs);
 
+
       List<Map<String, Object>> docsFromDisk = readDocsFromDisk(hdfsDir);
       Assert.assertEquals(docsFromDisk.size(), docs.size()) ;
       Assert.assertEquals(new File(hdfsDir).list().length, 1);
@@ -225,12 +245,13 @@ public abstract class EnrichmentIntegrationTest extends BaseIntegrationTest {
       hostEnrichmentValidation(doc);
       geoEnrichmentValidation(doc);
       threatIntelValidation(doc);
+      simpleEnrichmentValidation(doc);
     }
   }
 
   public static void baseValidation(Map<String, Object> jsonDoc) {
-    assertEnrichmentsExists("threatintels.", setOf("ip"), jsonDoc.keySet());
-    assertEnrichmentsExists("enrichments.", setOf("geo", "host"), jsonDoc.keySet());
+    assertEnrichmentsExists("threatintels.", setOf("hbaseThreatIntel"), jsonDoc.keySet());
+    assertEnrichmentsExists("enrichments.", setOf("geo", "host", "hbaseEnrichment" ), jsonDoc.keySet());
     for(Map.Entry<String, Object> kv : jsonDoc.entrySet()) {
       //ensure no values are empty.
       Assert.assertTrue(kv.getValue().toString().length() > 0);
@@ -313,25 +334,47 @@ public abstract class EnrichmentIntegrationTest extends BaseIntegrationTest {
       }
     }
   }
+  private static void simpleEnrichmentValidation(Map<String, Object> indexedDoc) {
+    if(indexedDoc.get(SRC_IP).equals("10.0.2.3")
+            || indexedDoc.get(DST_IP).equals("10.0.2.3")
+            ) {
+      Assert.assertTrue(keyPatternExists("enrichments.hbaseEnrichment", indexedDoc));
+      if(indexedDoc.get(SRC_IP).equals("10.0.2.3")) {
+        Assert.assertEquals(indexedDoc.get("enrichments.hbaseEnrichment." + SRC_IP + "." + PLAYFUL_CLASSIFICATION_TYPE+ ".orientation")
+                , PLAYFUL_ENRICHMENT.get("orientation")
+        );
+      }
+      else if(indexedDoc.get(DST_IP).equals("10.0.2.3")) {
+        Assert.assertEquals( indexedDoc.get("enrichments.hbaseEnrichment." + DST_IP + "." + PLAYFUL_CLASSIFICATION_TYPE + ".orientation")
+                , PLAYFUL_ENRICHMENT.get("orientation")
+        );
+      }
+    }
+
+  }
   private static void threatIntelValidation(Map<String, Object> indexedDoc) {
-    if(keyPatternExists("threatintels.", indexedDoc)) {
+    if(indexedDoc.get(SRC_IP).equals("10.0.2.3")
+    || indexedDoc.get(DST_IP).equals("10.0.2.3")
+            ) {
       //if we have any threat intel messages, we want to tag is_alert to true
+      Assert.assertTrue(keyPatternExists("threatintels.", indexedDoc));
       Assert.assertEquals(indexedDoc.get("is_alert"), "true");
     }
     else {
       //For YAF this is the case, but if we do snort later on, this will be invalid.
       Assert.assertNull(indexedDoc.get("is_alert"));
+      Assert.assertFalse(keyPatternExists("threatintels.", indexedDoc));
     }
     //ip threat intels
-    if(keyPatternExists("threatintels.ip.", indexedDoc)) {
+    if(keyPatternExists("threatintels.hbaseThreatIntel.", indexedDoc)) {
       if(indexedDoc.get(SRC_IP).equals("10.0.2.3")) {
-        Assert.assertEquals(indexedDoc.get("threatintels.ip." + SRC_IP + ".ip_threat_intel"), "alert");
+        Assert.assertEquals(indexedDoc.get("threatintels.hbaseThreatIntel." + SRC_IP + "." + MALICIOUS_IP_TYPE), "alert");
       }
       else if(indexedDoc.get(DST_IP).equals("10.0.2.3")) {
-        Assert.assertEquals(indexedDoc.get("threatintels.ip." + DST_IP + ".ip_threat_intel"), "alert");
+        Assert.assertEquals(indexedDoc.get("threatintels.hbaseThreatIntel." + DST_IP + "." + MALICIOUS_IP_TYPE), "alert");
       }
       else {
-        Assert.fail("There was a threat intels that I did not expect.");
+        Assert.fail("There was a threat intels that I did not expect: " + indexedDoc);
       }
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/9621c55e/metron-streaming/Metron-Testing/src/main/java/org/apache/metron/integration/util/EnrichmentHelper.java
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-Testing/src/main/java/org/apache/metron/integration/util/EnrichmentHelper.java b/metron-streaming/Metron-Testing/src/main/java/org/apache/metron/integration/util/EnrichmentHelper.java
new file mode 100644
index 0000000..e06a2a5
--- /dev/null
+++ b/metron-streaming/Metron-Testing/src/main/java/org/apache/metron/integration/util/EnrichmentHelper.java
@@ -0,0 +1,39 @@
+/**
+ * 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.integration.util;
+
+import org.apache.hadoop.hbase.client.HTableInterface;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.metron.hbase.converters.enrichment.EnrichmentConverter;
+import org.apache.metron.hbase.converters.enrichment.EnrichmentKey;
+import org.apache.metron.hbase.converters.enrichment.EnrichmentValue;
+import org.apache.metron.reference.lookup.LookupKV;
+
+import java.io.IOException;
+
+public enum EnrichmentHelper {
+    INSTANCE;
+    EnrichmentConverter converter = new EnrichmentConverter();
+
+    public void load(HTableInterface table, String cf, Iterable<LookupKV<EnrichmentKey, EnrichmentValue>> results) throws IOException {
+        for(LookupKV<EnrichmentKey, EnrichmentValue> result : results) {
+            Put put = converter.toPut(cf, result.getKey(), result.getValue());
+            table.put(put);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/9621c55e/metron-streaming/Metron-Testing/src/main/java/org/apache/metron/integration/util/mock/MockGeoAdapter.java
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-Testing/src/main/java/org/apache/metron/integration/util/mock/MockGeoAdapter.java b/metron-streaming/Metron-Testing/src/main/java/org/apache/metron/integration/util/mock/MockGeoAdapter.java
index ee71cda..9b6a7f4 100644
--- a/metron-streaming/Metron-Testing/src/main/java/org/apache/metron/integration/util/mock/MockGeoAdapter.java
+++ b/metron-streaming/Metron-Testing/src/main/java/org/apache/metron/integration/util/mock/MockGeoAdapter.java
@@ -18,12 +18,13 @@
 package org.apache.metron.integration.util.mock;
 
 import com.google.common.base.Joiner;
+import org.apache.metron.enrichment.bolt.CacheKey;
 import org.apache.metron.enrichment.interfaces.EnrichmentAdapter;
 import org.json.simple.JSONObject;
 
 import java.io.Serializable;
 
-public class MockGeoAdapter implements EnrichmentAdapter<String>,
+public class MockGeoAdapter implements EnrichmentAdapter<CacheKey>,
         Serializable {
 
   public static final String DEFAULT_LOC_ID = "1";
@@ -36,11 +37,11 @@ public class MockGeoAdapter implements EnrichmentAdapter<String>,
   public static final String DEFAULT_LOCATION_POINT= Joiner.on(',').join(DEFAULT_LONGITUDE, DEFAULT_LATITUDE);
 
   @Override
-  public void logAccess(String value) {
+  public void logAccess(CacheKey value) {
 
   }
 
-  public JSONObject enrich(String metadata) {
+  public JSONObject enrich(CacheKey cache ) {
     JSONObject enriched = new JSONObject();
     enriched.put("locID", DEFAULT_LOC_ID);
     enriched.put("country", DEFAULT_COUNTRY);

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/9621c55e/metron-streaming/Metron-Testing/src/main/java/org/apache/metron/integration/util/threatintel/ThreatIntelHelper.java
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-Testing/src/main/java/org/apache/metron/integration/util/threatintel/ThreatIntelHelper.java b/metron-streaming/Metron-Testing/src/main/java/org/apache/metron/integration/util/threatintel/ThreatIntelHelper.java
deleted file mode 100644
index b44a48a..0000000
--- a/metron-streaming/Metron-Testing/src/main/java/org/apache/metron/integration/util/threatintel/ThreatIntelHelper.java
+++ /dev/null
@@ -1,39 +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.integration.util.threatintel;
-
-import org.apache.hadoop.hbase.client.HTableInterface;
-import org.apache.hadoop.hbase.client.Put;
-import org.apache.metron.hbase.converters.threatintel.ThreatIntelConverter;
-import org.apache.metron.hbase.converters.threatintel.ThreatIntelKey;
-import org.apache.metron.hbase.converters.threatintel.ThreatIntelValue;
-import org.apache.metron.reference.lookup.LookupKV;
-
-import java.io.IOException;
-
-public enum ThreatIntelHelper {
-    INSTANCE;
-    ThreatIntelConverter converter = new ThreatIntelConverter();
-
-    public void load(HTableInterface table, String cf, Iterable<LookupKV<ThreatIntelKey, ThreatIntelValue>> results) throws IOException {
-        for(LookupKV<ThreatIntelKey, ThreatIntelValue> result : results) {
-            Put put = converter.toPut(cf, result.getKey(), result.getValue());
-            table.put(put);
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/9621c55e/metron-streaming/Metron-Testing/src/main/resources/sample/config/sensors/bro.json
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-Testing/src/main/resources/sample/config/sensors/bro.json b/metron-streaming/Metron-Testing/src/main/resources/sample/config/sensors/bro.json
index 34109b8..8886495 100644
--- a/metron-streaming/Metron-Testing/src/main/resources/sample/config/sensors/bro.json
+++ b/metron-streaming/Metron-Testing/src/main/resources/sample/config/sensors/bro.json
@@ -8,7 +8,12 @@
   },
   "threatIntelFieldMap":
   {
-    "ip": ["ip_dst_addr", "ip_src_addr"]
+    "hbaseThreatIntel": ["ip_dst_addr", "ip_src_addr"]
+  },
+  "fieldToThreatIntelTypeMap":
+  {
+    "ip_dst_addr" : [ "malicious_ip" ]
+    ,"ip_src_addr" : [ "malicious_ip" ]
   }
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/9621c55e/metron-streaming/Metron-Testing/src/main/resources/sample/config/sensors/snort.json
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-Testing/src/main/resources/sample/config/sensors/snort.json b/metron-streaming/Metron-Testing/src/main/resources/sample/config/sensors/snort.json
index 1208637..b7cc22b 100644
--- a/metron-streaming/Metron-Testing/src/main/resources/sample/config/sensors/snort.json
+++ b/metron-streaming/Metron-Testing/src/main/resources/sample/config/sensors/snort.json
@@ -8,7 +8,12 @@
   },
   "threatIntelFieldMap":
   {
-    "ip": ["ip_dst_addr", "ip_src_addr"]
+    "hbaseThreatIntel": ["ip_dst_addr", "ip_src_addr"]
+  },
+  "fieldToThreatIntelTypeMap":
+  {
+    "ip_dst_addr" : [ "malicious_ip" ]
+    ,"ip_src_addr" : [ "malicious_ip" ]
   }
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/9621c55e/metron-streaming/Metron-Testing/src/main/resources/sample/config/sensors/yaf.json
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-Testing/src/main/resources/sample/config/sensors/yaf.json b/metron-streaming/Metron-Testing/src/main/resources/sample/config/sensors/yaf.json
index cfdcbc2..98da265 100644
--- a/metron-streaming/Metron-Testing/src/main/resources/sample/config/sensors/yaf.json
+++ b/metron-streaming/Metron-Testing/src/main/resources/sample/config/sensors/yaf.json
@@ -4,11 +4,22 @@
   "enrichmentFieldMap":
   {
     "geo": ["ip_src_addr", "ip_dst_addr"],
-    "host": ["ip_src_addr", "ip_dst_addr"]
+    "host": ["ip_src_addr", "ip_dst_addr"],
+    "hbaseEnrichment" : ["ip_src_addr", "ip_dst_addr"]
   },
   "threatIntelFieldMap":
   {
-    "ip": ["ip_src_addr", "ip_dst_addr"]
+    "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"]
   }
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/9621c55e/metron-streaming/Metron-Topologies/pom.xml
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-Topologies/pom.xml b/metron-streaming/Metron-Topologies/pom.xml
index ad60980..f4c2e8a 100644
--- a/metron-streaming/Metron-Topologies/pom.xml
+++ b/metron-streaming/Metron-Topologies/pom.xml
@@ -86,6 +86,21 @@
                     <artifactId>log4j</artifactId>
                     <groupId>log4j</groupId>
                 </exclusion>
+                <exclusion>
+                    <artifactId>org.apache.hadoop</artifactId>
+                    <groupId>hadoop-common</groupId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.hadoop</groupId>
+            <artifactId>hadoop-common</artifactId>
+            <version>${global_hadoop_version}</version>
+            <exclusions>
+                <exclusion>
+                    <artifactId>servlet-api</artifactId>
+                    <groupId>javax.servlet</groupId>
+                </exclusion>
             </exclusions>
         </dependency>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/9621c55e/metron-streaming/Metron-Topologies/src/main/bash/zk_load_configs.sh
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-Topologies/src/main/bash/zk_load_configs.sh b/metron-streaming/Metron-Topologies/src/main/bash/zk_load_configs.sh
index 7a6d471..0339e78 100755
--- a/metron-streaming/Metron-Topologies/src/main/bash/zk_load_configs.sh
+++ b/metron-streaming/Metron-Topologies/src/main/bash/zk_load_configs.sh
@@ -30,4 +30,4 @@ export METRON_VERSION=0.1BETA
 export METRON_HOME=/usr/metron/$METRON_VERSION
 export TOPOLOGIES_JAR=Metron-Topologies-$METRON_VERSION.jar
 export ZK_HOME=${ZK_HOME:-/usr/hdp/current/hbase-client}
-java -cp $METRON_HOME/lib/$TOPOLOGIES_JAR org.apache.metron.utils.ConfigurationsUtils "$@"
\ No newline at end of file
+java -cp $METRON_HOME/lib/$TOPOLOGIES_JAR org.apache.metron.utils.ConfigurationsUtils "$@"

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/9621c55e/metron-streaming/Metron-Topologies/src/main/resources/Metron_Configs/topologies/enrichment/remote.yaml
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-Topologies/src/main/resources/Metron_Configs/topologies/enrichment/remote.yaml b/metron-streaming/Metron-Topologies/src/main/resources/Metron_Configs/topologies/enrichment/remote.yaml
index 5423bf1..ddf0d9e 100644
--- a/metron-streaming/Metron-Topologies/src/main/resources/Metron_Configs/topologies/enrichment/remote.yaml
+++ b/metron-streaming/Metron-Topologies/src/main/resources/Metron_Configs/topologies/enrichment/remote.yaml
@@ -54,6 +54,30 @@ components:
         constructorArgs:
             -   "host"
             -   ref: "hostEnrichmentAdapter"
+
+    -   id: "simpleHBaseEnrichmentConfig"
+        className: "org.apache.metron.enrichment.adapters.simplehbase.SimpleHBaseConfig"
+        configMethods:
+            -   name: "withProviderImpl"
+                args:
+                    - "${hbase.provider.impl}"
+            -   name: "withHBaseTable"
+                args:
+                    - "${enrichment.simple.hbase.table}"
+            -   name: "withHBaseCF"
+                args:
+                    - "${enrichment.simple.hbase.cf}"
+    -   id: "simpleHBaseEnrichmentAdapter"
+        className: "org.apache.metron.enrichment.adapters.simplehbase.SimpleHBaseAdapter"
+        configMethods:
+           -    name: "withConfig"
+                args:
+                    - ref: "simpleHBaseEnrichmentConfig"
+    -   id: "simpleHBaseEnrichment"
+        className: "org.apache.metron.domain.Enrichment"
+        constructorArgs:
+          -   "hbaseEnrichment"
+          -   ref: "simpleHBaseEnrichmentAdapter"
     -   id: "enrichments"
         className: "java.util.ArrayList"
         configMethods:
@@ -63,10 +87,13 @@ components:
             -   name: "add"
                 args:
                     - ref: "hostEnrichment"
-
+            -   name: "add"
+                args:
+                    - ref: "simpleHBaseEnrichment"
 # Threat Intel
-    -   id: "ipThreatIntelConfig"
-        className: "org.apache.metron.threatintel.ThreatIntelConfig"
+
+    -   id: "simpleHBaseThreatIntelConfig"
+        className: "org.apache.metron.enrichment.adapters.threatintel.ThreatIntelConfig"
         configMethods:
             -   name: "withProviderImpl"
                 args:
@@ -79,27 +106,28 @@ components:
                     - "${threat.intel.tracker.cf}"
             -   name: "withHBaseTable"
                 args:
-                    - "${threat.intel.ip.table}"
+                    - "${threat.intel.simple.hbase.table}"
             -   name: "withHBaseCF"
                 args:
-                    - "${threat.intel.ip.cf}"
-    -   id: "ipThreatIntelAdapter"
-        className: "org.apache.metron.threatintel.ThreatIntelAdapter"
+                    - "${threat.intel.simple.hbase.cf}"
+    -   id: "simpleHBaseThreatIntelAdapter"
+        className: "org.apache.metron.enrichment.adapters.threatintel.ThreatIntelAdapter"
         configMethods:
            -    name: "withConfig"
                 args:
-                    - ref: "ipThreatIntelConfig"
-    -   id: "ipThreatIntelEnrichment"
+                    - ref: "simpleHBaseThreatIntelConfig"
+    -   id: "simpleHBaseThreatIntelEnrichment"
         className: "org.apache.metron.domain.Enrichment"
         constructorArgs:
-          -   "ip"
-          -   ref: "ipThreatIntelAdapter"
+          -   "hbaseThreatIntel"
+          -   ref: "simpleHBaseThreatIntelAdapter"
+
     -   id: "threatIntels"
         className: "java.util.ArrayList"
         configMethods:
             -   name: "add"
                 args:
-                    - ref: "ipThreatIntelEnrichment"
+                    - ref: "simpleHBaseThreatIntelEnrichment"
 
     -   id: "fileNameFormat"
         className: "org.apache.storm.hdfs.bolt.format.DefaultFileNameFormat"
@@ -194,6 +222,18 @@ bolts:
                 args: [10000]
             -   name: "withMaxTimeRetain"
                 args: [10]
+    -   id: "simpleHBaseEnrichmentBolt"
+        className: "org.apache.metron.enrichment.bolt.GenericEnrichmentBolt"
+        constructorArgs:
+            - "${kafka.zk}"
+        configMethods:
+            -   name: "withEnrichment"
+                args:
+                    - ref: "simpleHBaseEnrichment"
+            -   name: "withMaxCacheSize"
+                args: [10000]
+            -   name: "withMaxTimeRetain"
+                args: [10]
     -   id: "enrichmentJoinBolt"
         className: "org.apache.metron.enrichment.bolt.EnrichmentJoinBolt"
         constructorArgs:
@@ -218,14 +258,14 @@ bolts:
                     - ref: "threatIntels"
             -   name: "withMessageFieldName"
                 args: ["message"]
-    -   id: "ipThreatIntelBolt"
+    -   id: "simpleHBaseThreatIntelBolt"
         className: "org.apache.metron.enrichment.bolt.GenericEnrichmentBolt"
         constructorArgs:
             - "${kafka.zk}"
         configMethods:
             -   name: "withEnrichment"
                 args:
-                    - ref: "ipThreatIntelEnrichment"
+                    - ref: "simpleHBaseThreatIntelEnrichment"
             -   name: "withMaxCacheSize"
                 args: [10000]
             -   name: "withMaxTimeRetain"
@@ -284,6 +324,15 @@ streams:
             streamId: "geo"
             type: FIELDS
             args: ["key"]
+
+    -   name: "enrichmentSplit -> simpleHBaseEnrichmentBolt"
+        from: "enrichmentSplitBolt"
+        to: "simpleHBaseEnrichmentBolt"
+        grouping:
+            streamId: "hbaseEnrichment"
+            type: FIELDS
+            args: ["key"]
+
     -   name: "splitter -> join"
         from: "enrichmentSplitBolt"
         to: "enrichmentJoinBolt"
@@ -298,6 +347,15 @@ streams:
             streamId: "geo"
             type: FIELDS
             args: ["key"]
+
+
+    -   name: "simpleHBaseEnrichmentBolt -> join"
+        from: "simpleHBaseEnrichmentBolt"
+        to: "enrichmentJoinBolt"
+        grouping:
+            streamId: "hbaseEnrichment"
+            type: FIELDS
+            args: ["key"]
     -   name: "host -> join"
         from: "hostEnrichmentBolt"
         to: "enrichmentJoinBolt"
@@ -315,19 +373,19 @@ streams:
             type: FIELDS
             args: ["key"]
 
-    -   name: "threatSplit -> ip"
+    -   name: "threatSplit -> simpleHBaseThreatIntel"
         from: "threatIntelSplitBolt"
-        to: "ipThreatIntelBolt"
+        to: "simpleHBaseThreatIntelBolt"
         grouping:
-            streamId: "ip"
+            streamId: "hbaseThreatIntel"
             type: FIELDS
             args: ["key"]
 
-    -   name: "ip -> join"
-        from: "ipThreatIntelBolt"
+    -   name: "simpleHBaseThreatIntel -> join"
+        from: "simpleHBaseThreatIntelBolt"
         to: "threatIntelJoinBolt"
         grouping:
-            streamId: "ip"
+            streamId: "hbaseThreatIntel"
             type: FIELDS
             args: ["key"]
     -   name: "threatIntelSplit -> threatIntelJoin"
@@ -345,6 +403,7 @@ streams:
             streamId: "message"
             type: FIELDS
             args: ["key"]
+
     -   name: "threatIntelJoin -> hdfs"
         from: "threatIntelJoinBolt"
         to: "hdfsIndexingBolt"

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/9621c55e/metron-streaming/Metron-Topologies/src/main/resources/Metron_Configs/topologies/enrichment/test.yaml
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-Topologies/src/main/resources/Metron_Configs/topologies/enrichment/test.yaml b/metron-streaming/Metron-Topologies/src/main/resources/Metron_Configs/topologies/enrichment/test.yaml
index 1296cb3..60cce7d 100644
--- a/metron-streaming/Metron-Topologies/src/main/resources/Metron_Configs/topologies/enrichment/test.yaml
+++ b/metron-streaming/Metron-Topologies/src/main/resources/Metron_Configs/topologies/enrichment/test.yaml
@@ -36,6 +36,30 @@ components:
         constructorArgs:
             -   "host"
             -   ref: "hostEnrichmentAdapter"
+
+    -   id: "simpleHBaseEnrichmentConfig"
+        className: "org.apache.metron.enrichment.adapters.simplehbase.SimpleHBaseConfig"
+        configMethods:
+            -   name: "withProviderImpl"
+                args:
+                    - "${hbase.provider.impl}"
+            -   name: "withHBaseTable"
+                args:
+                    - "${enrichment.simple.hbase.table}"
+            -   name: "withHBaseCF"
+                args:
+                    - "${enrichment.simple.hbase.cf}"
+    -   id: "simpleHBaseEnrichmentAdapter"
+        className: "org.apache.metron.enrichment.adapters.simplehbase.SimpleHBaseAdapter"
+        configMethods:
+           -    name: "withConfig"
+                args:
+                    - ref: "simpleHBaseEnrichmentConfig"
+    -   id: "simpleHBaseEnrichment"
+        className: "org.apache.metron.domain.Enrichment"
+        constructorArgs:
+          -   "hbaseEnrichment"
+          -   ref: "simpleHBaseEnrichmentAdapter"
     -   id: "enrichments"
         className: "java.util.ArrayList"
         configMethods:
@@ -45,10 +69,13 @@ components:
             -   name: "add"
                 args:
                     - ref: "hostEnrichment"
-
+            -   name: "add"
+                args:
+                    - ref: "simpleHBaseEnrichment"
 # Threat Intel
-    -   id: "ipThreatIntelConfig"
-        className: "org.apache.metron.threatintel.ThreatIntelConfig"
+
+    -   id: "simpleHBaseThreatIntelConfig"
+        className: "org.apache.metron.enrichment.adapters.threatintel.ThreatIntelConfig"
         configMethods:
             -   name: "withProviderImpl"
                 args:
@@ -61,27 +88,28 @@ components:
                     - "${threat.intel.tracker.cf}"
             -   name: "withHBaseTable"
                 args:
-                    - "${threat.intel.ip.table}"
+                    - "${threat.intel.simple.hbase.table}"
             -   name: "withHBaseCF"
                 args:
-                    - "${threat.intel.ip.cf}"
-    -   id: "ipThreatIntelAdapter"
-        className: "org.apache.metron.threatintel.ThreatIntelAdapter"
+                    - "${threat.intel.simple.hbase.cf}"
+    -   id: "simpleHBaseThreatIntelAdapter"
+        className: "org.apache.metron.enrichment.adapters.threatintel.ThreatIntelAdapter"
         configMethods:
            -    name: "withConfig"
                 args:
-                    - ref: "ipThreatIntelConfig"
-    -   id: "ipThreatIntelEnrichment"
+                    - ref: "simpleHBaseThreatIntelConfig"
+    -   id: "simpleHBaseThreatIntelEnrichment"
         className: "org.apache.metron.domain.Enrichment"
         constructorArgs:
-          -   "ip"
-          -   ref: "ipThreatIntelAdapter"
+          -   "hbaseThreatIntel"
+          -   ref: "simpleHBaseThreatIntelAdapter"
+
     -   id: "threatIntels"
         className: "java.util.ArrayList"
         configMethods:
             -   name: "add"
                 args:
-                    - ref: "ipThreatIntelEnrichment"
+                    - ref: "simpleHBaseThreatIntelEnrichment"
 
     -   id: "fileNameFormat"
         className: "org.apache.storm.hdfs.bolt.format.DefaultFileNameFormat"
@@ -176,6 +204,18 @@ bolts:
                 args: [10000]
             -   name: "withMaxTimeRetain"
                 args: [10]
+    -   id: "simpleHBaseEnrichmentBolt"
+        className: "org.apache.metron.enrichment.bolt.GenericEnrichmentBolt"
+        constructorArgs:
+            - "${kafka.zk}"
+        configMethods:
+            -   name: "withEnrichment"
+                args:
+                    - ref: "simpleHBaseEnrichment"
+            -   name: "withMaxCacheSize"
+                args: [10000]
+            -   name: "withMaxTimeRetain"
+                args: [10]
     -   id: "enrichmentJoinBolt"
         className: "org.apache.metron.enrichment.bolt.EnrichmentJoinBolt"
         constructorArgs:
@@ -200,14 +240,14 @@ bolts:
                     - ref: "threatIntels"
             -   name: "withMessageFieldName"
                 args: ["message"]
-    -   id: "ipThreatIntelBolt"
+    -   id: "simpleHBaseThreatIntelBolt"
         className: "org.apache.metron.enrichment.bolt.GenericEnrichmentBolt"
         constructorArgs:
             - "${kafka.zk}"
         configMethods:
             -   name: "withEnrichment"
                 args:
-                    - ref: "ipThreatIntelEnrichment"
+                    - ref: "simpleHBaseThreatIntelEnrichment"
             -   name: "withMaxCacheSize"
                 args: [10000]
             -   name: "withMaxTimeRetain"
@@ -266,6 +306,15 @@ streams:
             streamId: "geo"
             type: FIELDS
             args: ["key"]
+
+    -   name: "enrichmentSplit -> simpleHBaseEnrichmentBolt"
+        from: "enrichmentSplitBolt"
+        to: "simpleHBaseEnrichmentBolt"
+        grouping:
+            streamId: "hbaseEnrichment"
+            type: FIELDS
+            args: ["key"]
+
     -   name: "splitter -> join"
         from: "enrichmentSplitBolt"
         to: "enrichmentJoinBolt"
@@ -280,6 +329,15 @@ streams:
             streamId: "geo"
             type: FIELDS
             args: ["key"]
+
+
+    -   name: "simpleHBaseEnrichmentBolt -> join"
+        from: "simpleHBaseEnrichmentBolt"
+        to: "enrichmentJoinBolt"
+        grouping:
+            streamId: "hbaseEnrichment"
+            type: FIELDS
+            args: ["key"]
     -   name: "host -> join"
         from: "hostEnrichmentBolt"
         to: "enrichmentJoinBolt"
@@ -297,19 +355,19 @@ streams:
             type: FIELDS
             args: ["key"]
 
-    -   name: "threatSplit -> ip"
+    -   name: "threatSplit -> simpleHBaseThreatIntel"
         from: "threatIntelSplitBolt"
-        to: "ipThreatIntelBolt"
+        to: "simpleHBaseThreatIntelBolt"
         grouping:
-            streamId: "ip"
+            streamId: "hbaseThreatIntel"
             type: FIELDS
             args: ["key"]
 
-    -   name: "ip -> join"
-        from: "ipThreatIntelBolt"
+    -   name: "simpleHBaseThreatIntel -> join"
+        from: "simpleHBaseThreatIntelBolt"
         to: "threatIntelJoinBolt"
         grouping:
-            streamId: "ip"
+            streamId: "hbaseThreatIntel"
             type: FIELDS
             args: ["key"]
     -   name: "threatIntelSplit -> threatIntelJoin"

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/9621c55e/metron-streaming/pom.xml
----------------------------------------------------------------------
diff --git a/metron-streaming/pom.xml b/metron-streaming/pom.xml
index c4222e8..d4241df 100644
--- a/metron-streaming/pom.xml
+++ b/metron-streaming/pom.xml
@@ -88,6 +88,13 @@
 			<version>4.12</version>
 			<scope>test</scope>
 		</dependency>
+		<dependency>
+			<groupId>org.adrianwalker</groupId>
+			<artifactId>multiline-string</artifactId>
+			<version>0.1.2</version>
+			<scope>test</scope>
+
+		</dependency>
 	</dependencies>
 	<build>
     <plugins>
@@ -191,10 +198,18 @@
 			</plugin>
 		</plugins>
 	</reporting>
-	<repositories>
-		<repository>
-			<id>clojars.org</id>
-			<url>http://clojars.org/repo</url>
-		</repository>
-	</repositories>
+    <repositories>
+        <repository>
+            <id>clojars.org</id>
+            <url>http://clojars.org/repo</url>
+        </repository>
+        <repository>
+            <id>multiline-release-repo</id>
+            <url>https://raw.github.com/benelog/multiline/master/maven-repository</url>
+            <snapshots>
+                <enabled>false</enabled>
+            </snapshots>
+        </repository>
+
+    </repositories>
 </project>



[6/6] incubator-metron git commit: METRON-93: Generalize the HBase threat intel infrastructure to support enrichments closes apache/incubator-metron#64

Posted by ce...@apache.org.
METRON-93: Generalize the HBase threat intel infrastructure to support enrichments closes apache/incubator-metron#64


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

Branch: refs/heads/master
Commit: 9621c55ec61dc3f013ddbdff7135442c25033321
Parents: e59b1a3
Author: cestella <ce...@gmail.com>
Authored: Thu Apr 7 08:56:23 2016 -0400
Committer: cstella <ce...@gmail.com>
Committed: Thu Apr 7 08:56:23 2016 -0400

----------------------------------------------------------------------
 LICENSE                                         |   5 +
 deployment/amazon-ec2/conf/defaults.yml         |   7 +-
 .../inventory/metron_example/group_vars/all     |   3 +-
 .../inventory/multinode-vagrant/group_vars/all  |   3 +-
 .../inventory/singlenode-vagrant/group_vars/all |   3 +-
 deployment/roles/hadoop_setup/tasks/main.yml    |   3 +-
 .../files/config/sensors/bro.json               |   7 +-
 .../files/config/sensors/pcap.json              |   7 +-
 .../files/config/sensors/snort.json             |  10 +-
 .../files/config/sensors/yaf.json               |   7 +-
 .../roles/metron_streaming/files/extractor.json |   1 +
 .../roles/metron_streaming/tasks/main.yml       |  12 +-
 .../metron_streaming/tasks/threat_intel.yml     |   2 +-
 .../main/java/org/apache/metron/Constants.java  |   3 +-
 .../metron/bolt/BulkMessageWriterBolt.java      |   3 +-
 .../org/apache/metron/bolt/ConfiguredBolt.java  |  12 +
 .../java/org/apache/metron/bolt/JoinBolt.java   |   6 +
 .../metron/domain/SensorEnrichmentConfig.java   |  60 +++
 .../metron/enrichment/EnrichmentConfig.java     | 203 ++++++++++
 .../hbase/converters/AbstractConverter.java     |  95 ++---
 .../enrichment/EnrichmentConverter.java         |  38 ++
 .../converters/enrichment/EnrichmentKey.java    | 119 ++++++
 .../converters/enrichment/EnrichmentValue.java  | 107 +++++
 .../threatintel/ThreatIntelConverter.java       |  39 --
 .../converters/threatintel/ThreatIntelKey.java  |  89 ----
 .../threatintel/ThreatIntelValue.java           | 110 -----
 .../metron/hbase/lookup/EnrichmentLookup.java   | 108 +++++
 .../apache/metron/reference/lookup/Lookup.java  |  93 +++--
 .../accesstracker/BloomAccessTracker.java       |   1 +
 .../lookup/accesstracker/NoopAccessTracker.java |  65 +++
 .../reference/lookup/handler/Handler.java       |   6 +-
 .../metron/threatintel/ThreatIntelResults.java  |  71 ----
 .../threatintel/hbase/ThreatIntelLookup.java    |  74 ----
 .../metron/utils/ConfigurationsUtils.java       |  41 +-
 .../java/org/apache/metron/utils/JSONUtils.java |  10 +-
 .../metron/enrichment/EnrichmentConfigTest.java | 197 +++++++++
 .../enrichment/EnrichmentConverterTest.java     |  34 ++
 metron-streaming/Metron-DataLoads/README.md     | 254 ++++++++++--
 .../src/main/bash/flatfile_loader.sh            |  39 ++
 .../src/main/bash/threatintel_taxii_load.sh     |   2 +-
 .../metron/dataloads/ThreatIntelLoader.java     | 195 ---------
 .../dataloads/bulk/LeastRecentlyUsedPruner.java |   2 +-
 .../dataloads/bulk/ThreatIntelBulkLoader.java   | 371 +++++++++--------
 .../metron/dataloads/extractor/Extractor.java   |   4 +-
 .../dataloads/extractor/ExtractorHandler.java   |  12 +-
 .../dataloads/extractor/csv/CSVExtractor.java   | 174 ++++----
 .../extractor/csv/LookupConverter.java          |   2 +-
 .../extractor/csv/LookupConverters.java         |  13 +-
 .../dataloads/extractor/stix/StixExtractor.java |   2 -
 .../stix/types/AbstractObjectTypeHandler.java   |   4 +-
 .../extractor/stix/types/AddressHandler.java    | 101 +++--
 .../extractor/stix/types/DomainHandler.java     |  72 ++--
 .../extractor/stix/types/HostnameHandler.java   |  57 ++-
 .../extractor/stix/types/ObjectTypeHandler.java |   8 +-
 .../dataloads/hbase/mr/BulkLoadMapper.java      |   4 -
 .../metron/dataloads/hbase/mr/PrunerMapper.java |   2 +-
 .../SimpleEnrichmentFlatFileLoader.java         | 263 ++++++++++++
 .../dataloads/nonbulk/taxii/ConnectionType.java |  23 ++
 .../dataloads/nonbulk/taxii/TableInfo.java      |  71 ++++
 .../nonbulk/taxii/TaxiiConnectionConfig.java    | 222 ++++++++++
 .../dataloads/nonbulk/taxii/TaxiiHandler.java   | 406 +++++++++++++++++++
 .../dataloads/nonbulk/taxii/TaxiiLoader.java    | 208 ++++++++++
 .../metron/dataloads/taxii/ConnectionType.java  |  23 --
 .../metron/dataloads/taxii/TableInfo.java       |  72 ----
 .../dataloads/taxii/TaxiiConnectionConfig.java  | 196 ---------
 .../metron/dataloads/taxii/TaxiiHandler.java    | 403 ------------------
 .../metron/dataloads/taxii/TaxiiLoader.java     | 180 --------
 .../dataloads/extractor/ExtractorTest.java      |  20 +-
 .../extractor/csv/CSVExtractorTest.java         | 130 ++----
 .../extractor/stix/StixExtractorTest.java       | 152 +++----
 .../hbase/HBaseEnrichmentConverterTest.java     |  74 ++++
 .../hbase/HBaseThreatIntelConverterTest.java    |  76 ----
 .../hbase/mr/BulkLoadMapperIntegrationTest.java |  32 +-
 .../dataloads/hbase/mr/BulkLoadMapperTest.java  |  34 +-
 .../LeastRecentlyUsedPrunerIntegrationTest.java |  34 +-
 .../nonbulk/taxii/MockTaxiiService.java         |  94 +++++
 .../nonbulk/taxii/TaxiiIntegrationTest.java     | 119 ++++++
 .../dataloads/taxii/MockTaxiiService.java       |  99 -----
 .../dataloads/taxii/TaxiiIntegrationTest.java   | 121 ------
 metron-streaming/Metron-Elasticsearch/pom.xml   |   5 +
 .../metron/writer/ElasticsearchWriter.java      |   2 +-
 .../ElasticsearchEnrichmentIntegrationTest.java |   2 +-
 .../Metron-EnrichmentAdapters/pom.xml           |   3 +-
 .../adapters/cif/AbstractCIFAdapter.java        |   3 +-
 .../adapters/cif/CIFHbaseAdapter.java           |   9 +-
 .../enrichment/adapters/geo/GeoAdapter.java     |  12 +-
 .../adapters/host/AbstractHostAdapter.java      |   5 +-
 .../adapters/host/HostFromJSONListAdapter.java  |   7 +-
 .../host/HostFromPropertiesFileAdapter.java     |   9 +-
 .../enrichment/adapters/jdbc/JdbcAdapter.java   |   3 +-
 .../simplehbase/SimpleHBaseAdapter.java         | 119 ++++++
 .../adapters/simplehbase/SimpleHBaseConfig.java |  55 +++
 .../adapters/threat/AbstractThreatAdapter.java  |  41 --
 .../adapters/threat/ThreatHbaseAdapter.java     | 140 -------
 .../threatintel/ThreatIntelAdapter.java         | 135 ++++++
 .../adapters/threatintel/ThreatIntelConfig.java | 108 +++++
 .../adapters/whois/WhoisHBaseAdapter.java       |  11 +-
 .../apache/metron/enrichment/bolt/CacheKey.java |  73 ++++
 .../enrichment/bolt/EnrichmentJoinBolt.java     |  24 +-
 .../enrichment/bolt/GenericEnrichmentBolt.java  |  44 +-
 .../enrichment/bolt/ThreatIntelJoinBolt.java    |  12 +-
 .../enrichment/utils/EnrichmentUtils.java       |  49 +++
 .../metron/threatintel/ThreatIntelAdapter.java  | 108 -----
 .../metron/threatintel/ThreatIntelConfig.java   | 125 ------
 .../adapters/cif/CIFHbaseAdapterTest.java       |   9 +-
 .../adapters/geo/GeoMysqlAdapterTest.java       |   3 +-
 .../adapters/whois/WhoisHBaseAdapterTest.java   |   5 +-
 metron-streaming/Metron-MessageParsers/pom.xml  |  14 +-
 metron-streaming/Metron-Solr/pom.xml            |   5 +
 .../integration/EnrichmentIntegrationTest.java  |  95 +++--
 .../integration/util/EnrichmentHelper.java      |  39 ++
 .../integration/util/mock/MockGeoAdapter.java   |   7 +-
 .../util/threatintel/ThreatIntelHelper.java     |  39 --
 .../resources/sample/config/sensors/bro.json    |   7 +-
 .../resources/sample/config/sensors/snort.json  |   7 +-
 .../resources/sample/config/sensors/yaf.json    |  15 +-
 metron-streaming/Metron-Topologies/pom.xml      |  15 +
 .../src/main/bash/zk_load_configs.sh            |   2 +-
 .../topologies/enrichment/remote.yaml           |  99 ++++-
 .../topologies/enrichment/test.yaml             |  98 ++++-
 metron-streaming/pom.xml                        |  27 +-
 121 files changed, 4512 insertions(+), 3102 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/9621c55e/LICENSE
----------------------------------------------------------------------
diff --git a/LICENSE b/LICENSE
index 4628dd1..da4e5a0 100644
--- a/LICENSE
+++ b/LICENSE
@@ -768,6 +768,11 @@ The above copyright notice and this permission notice shall be included in all c
 THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE.
 
 -----------------------------------------------------------------------
+This bundles the following released to the Public Domain:
+
+- Java Multiline String (https://github.com/benelog/multiline)
+
+-----------------------------------------------------------------------
 
 
 This product bundles normalize.css which is available under a "MIT" license. For details, see metron-ui/lib/public/vendor/LICENSE.normalize

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/9621c55e/deployment/amazon-ec2/conf/defaults.yml
----------------------------------------------------------------------
diff --git a/deployment/amazon-ec2/conf/defaults.yml b/deployment/amazon-ec2/conf/defaults.yml
index 77e841d..5f581f6 100644
--- a/deployment/amazon-ec2/conf/defaults.yml
+++ b/deployment/amazon-ec2/conf/defaults.yml
@@ -36,7 +36,8 @@ cluster_type: small_cluster
 # hbase
 pcap_hbase_table: pcap
 tracker_hbase_table: access_tracker
-threatintel_ip_hbase_table: malicious_ip
+threatintel_hbase_table: threatintel
+enrichment_hbase_table: enrichment
 
 # kafka
 num_partitions: 3
@@ -74,5 +75,5 @@ elasticsearch_data_dir: "/data1/elasticsearch,/data2/elasticsearch"
 install_elasticsearch: True
 install_solr: False
 elasticsearch_transport_port: 9300
-elasticsearch_network_interface: eth1
-elasticsearch_web_port: 9200
\ No newline at end of file
+elasticsearch_network_interface: eth0
+elasticsearch_web_port: 9200

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/9621c55e/deployment/inventory/metron_example/group_vars/all
----------------------------------------------------------------------
diff --git a/deployment/inventory/metron_example/group_vars/all b/deployment/inventory/metron_example/group_vars/all
index c5d81b0..4eba586 100644
--- a/deployment/inventory/metron_example/group_vars/all
+++ b/deployment/inventory/metron_example/group_vars/all
@@ -29,7 +29,8 @@ cluster_type: small_cluster
 # hbase
 pcap_hbase_table: pcap
 tracker_hbase_table: access_tracker
-threatintel_ip_hbase_table: malicious_ip
+threatintel_hbase_table: threatintel
+enrichment_hbase_table: enrichment
 
 # metron variables
 metron_version: 0.1BETA

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/9621c55e/deployment/inventory/multinode-vagrant/group_vars/all
----------------------------------------------------------------------
diff --git a/deployment/inventory/multinode-vagrant/group_vars/all b/deployment/inventory/multinode-vagrant/group_vars/all
index e486296..8f4e5ff 100644
--- a/deployment/inventory/multinode-vagrant/group_vars/all
+++ b/deployment/inventory/multinode-vagrant/group_vars/all
@@ -26,7 +26,8 @@ cluster_type: multi_vagrant_cluster
 # hbase
 pcap_hbase_table: pcap
 tracker_hbase_table: access_tracker
-threatintel_ip_hbase_table: malicious_ip
+threatintel_hbase_table: threatintel
+enrichment_hbase_table: enrichment
 
 #elasticsearch
 elasticsearch_transport_port: 9300

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/9621c55e/deployment/inventory/singlenode-vagrant/group_vars/all
----------------------------------------------------------------------
diff --git a/deployment/inventory/singlenode-vagrant/group_vars/all b/deployment/inventory/singlenode-vagrant/group_vars/all
index 364598f..6d96d5d 100644
--- a/deployment/inventory/singlenode-vagrant/group_vars/all
+++ b/deployment/inventory/singlenode-vagrant/group_vars/all
@@ -26,7 +26,8 @@ cluster_type: single_node_vm
 # hbase
 pcap_hbase_table: pcap
 tracker_hbase_table: access_tracker
-threatintel_ip_hbase_table: malicious_ip
+threatintel_hbase_table: threatintel
+enrichment_hbase_table: enrichment
 
 # metron variables
 metron_version: 0.1BETA

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/9621c55e/deployment/roles/hadoop_setup/tasks/main.yml
----------------------------------------------------------------------
diff --git a/deployment/roles/hadoop_setup/tasks/main.yml b/deployment/roles/hadoop_setup/tasks/main.yml
index 5b6c47c..de01abf 100644
--- a/deployment/roles/hadoop_setup/tasks/main.yml
+++ b/deployment/roles/hadoop_setup/tasks/main.yml
@@ -22,7 +22,8 @@
   with_items:
     - "{{ pcap_hbase_table }}"
     - "{{ tracker_hbase_table }}"
-    - "{{ threatintel_ip_hbase_table }}"
+    - "{{ threatintel_hbase_table }}"
+    - "{{ enrichment_hbase_table }}"
 
 #if kafka topic
 - name: Create Kafka topics

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/9621c55e/deployment/roles/metron_streaming/files/config/sensors/bro.json
----------------------------------------------------------------------
diff --git a/deployment/roles/metron_streaming/files/config/sensors/bro.json b/deployment/roles/metron_streaming/files/config/sensors/bro.json
index 34109b8..2b534b4 100644
--- a/deployment/roles/metron_streaming/files/config/sensors/bro.json
+++ b/deployment/roles/metron_streaming/files/config/sensors/bro.json
@@ -8,7 +8,12 @@
   },
   "threatIntelFieldMap":
   {
-    "ip": ["ip_dst_addr", "ip_src_addr"]
+    "hbaseThreatIntel": ["ip_src_addr", "ip_dst_addr"]
+  },
+  "fieldToThreatIntelTypeMap":
+  {
+    "ip_src_addr" : ["malicious_ip"],
+    "ip_dst_addr" : ["malicious_ip"]
   }
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/9621c55e/deployment/roles/metron_streaming/files/config/sensors/pcap.json
----------------------------------------------------------------------
diff --git a/deployment/roles/metron_streaming/files/config/sensors/pcap.json b/deployment/roles/metron_streaming/files/config/sensors/pcap.json
index 4b9c639..7792165 100644
--- a/deployment/roles/metron_streaming/files/config/sensors/pcap.json
+++ b/deployment/roles/metron_streaming/files/config/sensors/pcap.json
@@ -8,7 +8,12 @@
   },
   "threatIntelFieldMap":
   {
-    "ip": ["ip_src_addr", "ip_dst_addr"]
+    "hbaseThreatIntel": ["ip_dst_addr", "ip_src_addr"]
+  },
+  "fieldToThreatIntelTypeMap":
+  {
+    "ip_dst_addr" : [ "malicious_ip" ]
+    ,"ip_src_addr" : [ "malicious_ip" ]
   }
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/9621c55e/deployment/roles/metron_streaming/files/config/sensors/snort.json
----------------------------------------------------------------------
diff --git a/deployment/roles/metron_streaming/files/config/sensors/snort.json b/deployment/roles/metron_streaming/files/config/sensors/snort.json
index 1208637..c5b6dcc 100644
--- a/deployment/roles/metron_streaming/files/config/sensors/snort.json
+++ b/deployment/roles/metron_streaming/files/config/sensors/snort.json
@@ -6,9 +6,13 @@
     "geo": ["ip_dst_addr", "ip_src_addr"],
     "host": ["host"]
   },
-  "threatIntelFieldMap":
+ "threatIntelFieldMap":
   {
-    "ip": ["ip_dst_addr", "ip_src_addr"]
+    "hbaseThreatIntel": ["ip_src_addr", "ip_dst_addr"]
+  },
+  "fieldToThreatIntelTypeMap":
+  {
+    "ip_src_addr" : ["malicious_ip"],
+    "ip_dst_addr" : ["malicious_ip"]
   }
 }
-

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/9621c55e/deployment/roles/metron_streaming/files/config/sensors/yaf.json
----------------------------------------------------------------------
diff --git a/deployment/roles/metron_streaming/files/config/sensors/yaf.json b/deployment/roles/metron_streaming/files/config/sensors/yaf.json
index 65de961..2b46c9a 100644
--- a/deployment/roles/metron_streaming/files/config/sensors/yaf.json
+++ b/deployment/roles/metron_streaming/files/config/sensors/yaf.json
@@ -8,7 +8,12 @@
   },
   "threatIntelFieldMap":
   {
-    "ip": ["ip_dst_addr", "ip_src_addr"]
+    "hbaseThreatIntel": ["ip_src_addr", "ip_dst_addr"]
+  },
+  "fieldToThreatIntelTypeMap":
+  {
+    "ip_src_addr" : ["malicious_ip"],
+    "ip_dst_addr" : ["malicious_ip"]
   }
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/9621c55e/deployment/roles/metron_streaming/files/extractor.json
----------------------------------------------------------------------
diff --git a/deployment/roles/metron_streaming/files/extractor.json b/deployment/roles/metron_streaming/files/extractor.json
index 81429e8..545202a 100644
--- a/deployment/roles/metron_streaming/files/extractor.json
+++ b/deployment/roles/metron_streaming/files/extractor.json
@@ -4,6 +4,7 @@
       "ip": 0
     },
     "indicator_column": "ip",
+    "type" : "malicious_ip",
     "separator": ","
   },
   "extractor": "CSV"

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/9621c55e/deployment/roles/metron_streaming/tasks/main.yml
----------------------------------------------------------------------
diff --git a/deployment/roles/metron_streaming/tasks/main.yml b/deployment/roles/metron_streaming/tasks/main.yml
index 863ecd8..b913011 100644
--- a/deployment/roles/metron_streaming/tasks/main.yml
+++ b/deployment/roles/metron_streaming/tasks/main.yml
@@ -83,8 +83,10 @@
     - { regexp: "bolt.hbase.table.name=", line: "bolt.hbase.table.name={{ pcap_hbase_table }}" }
     - { regexp: "threat.intel.tracker.table=", line: "threat.intel.tracker.table={{ tracker_hbase_table }}" }
     - { regexp: "threat.intel.tracker.cf=", line: "threat.intel.tracker.cf=t" }
-    - { regexp: "threat.intel.ip.table=", line: "threat.intel.ip.table={{ threatintel_ip_hbase_table }}" }
-    - { regexp: "threat.intel.ip.cf=", line: "threat.intel.ip.cf=t" }
+    - { regexp: "threat.intel.simple.hbase.table=", line: "threat.intel.simple.hbase.table={{ threatintel_hbase_table }}" }
+    - { regexp: "threat.intel.simple.hbase.cf=", line: "threat.intel.simple.hbase.cf=t" }
+    - { regexp: "enrichment.simple.hbase.table=", line: "enrichment.simple.hbase.table={{ enrichment_hbase_table }}" }
+    - { regexp: "enrichment.simple.hbase.cf=", line: "enrichment.simple.hbase.cf=t" }
     - { regexp: "mysql.ip=", line: "mysql.ip={{ groups.mysql[0] }}" }
     - { regexp: "mysql.password=", line: "mysql.password={{ mysql_root_password }}" }
     - { regexp: "index.hdfs.output=", line: "index.hdfs.output={{ metron_hdfs_output_dir }}/enrichment/indexed" }
@@ -109,8 +111,10 @@
     - { regexp: "bolt.hbase.table.name=", line: "bolt.hbase.table.name={{ pcap_hbase_table }}" }
     - { regexp: "threat.intel.tracker.table=", line: "threat.intel.tracker.table={{ tracker_hbase_table }}" }
     - { regexp: "threat.intel.tracker.cf=", line: "threat.intel.tracker.cf=t" }
-    - { regexp: "threat.intel.ip.table=", line: "threat.intel.ip.table={{ threatintel_ip_hbase_table }}" }
-    - { regexp: "threat.intel.ip.cf=", line: "threat.intel.ip.cf=t" }
+    - { regexp: "threat.intel.simple.hbase.table=", line: "threat.intel.simple.hbase.table={{ threatintel_hbase_table }}" }
+    - { regexp: "threat.intel.simple.hbase.cf=", line: "threat.intel.simple.hbase.cf=t" }
+    - { regexp: "enrichment.simple.hbase.table=", line: "enrichment.simple.hbase.table={{ enrichment_hbase_table }}" }
+    - { regexp: "enrichment.simple.hbase.cf=", line: "enrichment.simple.hbase.cf=t" }
     - { regexp: "mysql.ip=", line: "mysql.ip={{ groups.mysql[0] }}" }
     - { regexp: "mysql.password=", line: "mysql.password={{ mysql_root_password }}" }
     - { regexp: "index.hdfs.output=", line: "index.hdfs.output={{ metron_hdfs_output_dir }}/enrichment/indexed" }

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/9621c55e/deployment/roles/metron_streaming/tasks/threat_intel.yml
----------------------------------------------------------------------
diff --git a/deployment/roles/metron_streaming/tasks/threat_intel.yml b/deployment/roles/metron_streaming/tasks/threat_intel.yml
index 6798319..6dd8144 100644
--- a/deployment/roles/metron_streaming/tasks/threat_intel.yml
+++ b/deployment/roles/metron_streaming/tasks/threat_intel.yml
@@ -37,7 +37,7 @@
   command: "hdfs dfs -put {{ threat_intel_work_dir }}/{{ threat_intel_csv_filename }} ."
 
 - name: Run Threat Intel Bulk Load
-  shell: "{{ threat_intel_bin }} -f t --table malicious_ip -e {{ threat_intel_work_dir }}/extractor.json  -i /user/root && touch {{ threat_intel_work_dir }}/loaded"
+  shell: "{{ threat_intel_bin }} -f t --table {{threatintel_hbase_table}} -e {{ threat_intel_work_dir }}/extractor.json  -i /user/root && touch {{ threat_intel_work_dir }}/loaded"
   args:
     creates: "{{ threat_intel_work_dir }}/loaded"
 

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/9621c55e/metron-streaming/Metron-Common/src/main/java/org/apache/metron/Constants.java
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-Common/src/main/java/org/apache/metron/Constants.java b/metron-streaming/Metron-Common/src/main/java/org/apache/metron/Constants.java
index 78efa5c..32f1441 100644
--- a/metron-streaming/Metron-Common/src/main/java/org/apache/metron/Constants.java
+++ b/metron-streaming/Metron-Common/src/main/java/org/apache/metron/Constants.java
@@ -29,6 +29,7 @@ public class Constants {
   public static final String SENSOR_TYPE = "source.type";
   public static final String ENRICHMENT_TOPIC = "enrichments";
   public static final String ERROR_STREAM = "error";
-
+  public static final String SIMPLE_HBASE_ENRICHMENT = "hbaseEnrichment";
+  public static final String SIMPLE_HBASE_THREAT_INTEL = "hbaseThreatIntel";
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/9621c55e/metron-streaming/Metron-Common/src/main/java/org/apache/metron/bolt/BulkMessageWriterBolt.java
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-Common/src/main/java/org/apache/metron/bolt/BulkMessageWriterBolt.java b/metron-streaming/Metron-Common/src/main/java/org/apache/metron/bolt/BulkMessageWriterBolt.java
index 1f12f7a..f1f79f0 100644
--- a/metron-streaming/Metron-Common/src/main/java/org/apache/metron/bolt/BulkMessageWriterBolt.java
+++ b/metron-streaming/Metron-Common/src/main/java/org/apache/metron/bolt/BulkMessageWriterBolt.java
@@ -81,8 +81,7 @@ public class BulkMessageWriterBolt extends ConfiguredBolt {
       sensorMessageMap.put(sensorType, messageList);
     } else {
       try {
-        String esType = sensorType + "_doc";
-        bulkMessageWriter.write(esType, configurations, tupleList, messageList);
+        bulkMessageWriter.write(sensorType, configurations, tupleList, messageList);
         for(Tuple t: tupleList) {
           collector.ack(t);
         }

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/9621c55e/metron-streaming/Metron-Common/src/main/java/org/apache/metron/bolt/ConfiguredBolt.java
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-Common/src/main/java/org/apache/metron/bolt/ConfiguredBolt.java b/metron-streaming/Metron-Common/src/main/java/org/apache/metron/bolt/ConfiguredBolt.java
index a832ebb..3d22fe9 100644
--- a/metron-streaming/Metron-Common/src/main/java/org/apache/metron/bolt/ConfiguredBolt.java
+++ b/metron-streaming/Metron-Common/src/main/java/org/apache/metron/bolt/ConfiguredBolt.java
@@ -50,6 +50,9 @@ public abstract class ConfiguredBolt extends BaseRichBolt {
     this.zookeeperUrl = zookeeperUrl;
   }
 
+  protected void reloadCallback() {
+  }
+
   public ConfiguredBolt withTimeout(long timeout) {
     this.timeout = timeout;
     return this;
@@ -70,12 +73,21 @@ public abstract class ConfiguredBolt extends BaseRichBolt {
             String path = event.getData().getPath();
             byte[] data = event.getData().getData();
             updateConfig(path, data);
+            reloadCallback();
           }
         }
       };
       cache.getListenable().addListener(listener);
+      try {
+        ConfigurationsUtils.updateConfigsFromZookeeper(configurations, client);
+      }
+      catch(Exception e) {
+        LOG.warn("Unable to load configs from zookeeper, but the cache should load lazily...");
+      }
+
       cache.start();
     } catch (Exception e) {
+      LOG.error(e.getMessage(), e);
       throw new RuntimeException(e);
     }
   }

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/9621c55e/metron-streaming/Metron-Common/src/main/java/org/apache/metron/bolt/JoinBolt.java
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-Common/src/main/java/org/apache/metron/bolt/JoinBolt.java b/metron-streaming/Metron-Common/src/main/java/org/apache/metron/bolt/JoinBolt.java
index 653eade..9c2c67e 100644
--- a/metron-streaming/Metron-Common/src/main/java/org/apache/metron/bolt/JoinBolt.java
+++ b/metron-streaming/Metron-Common/src/main/java/org/apache/metron/bolt/JoinBolt.java
@@ -23,6 +23,7 @@ import backtype.storm.topology.OutputFieldsDeclarer;
 import backtype.storm.tuple.Fields;
 import backtype.storm.tuple.Tuple;
 import backtype.storm.tuple.Values;
+import com.google.common.base.Joiner;
 import com.google.common.cache.CacheBuilder;
 import com.google.common.cache.CacheLoader;
 import com.google.common.cache.LoadingCache;
@@ -104,6 +105,11 @@ public abstract class JoinBolt<V> extends ConfiguredBolt {
         cache.invalidate(key);
       } else {
         cache.put(key, streamMessageMap);
+        if(LOG.isDebugEnabled()) {
+          LOG.debug(getClass().getSimpleName() + ": Missed joining portions for "+ key + ". Expected " + Joiner.on(",").join(streamIds)
+                  + " != " + Joiner.on(",").join(streamMessageKeys)
+                   );
+        }
       }
     } catch (ExecutionException e) {
       collector.reportError(e);

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/9621c55e/metron-streaming/Metron-Common/src/main/java/org/apache/metron/domain/SensorEnrichmentConfig.java
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-Common/src/main/java/org/apache/metron/domain/SensorEnrichmentConfig.java b/metron-streaming/Metron-Common/src/main/java/org/apache/metron/domain/SensorEnrichmentConfig.java
index b24e8a8..e37c6d0 100644
--- a/metron-streaming/Metron-Common/src/main/java/org/apache/metron/domain/SensorEnrichmentConfig.java
+++ b/metron-streaming/Metron-Common/src/main/java/org/apache/metron/domain/SensorEnrichmentConfig.java
@@ -17,6 +17,11 @@
  */
 package org.apache.metron.domain;
 
+import com.fasterxml.jackson.core.JsonProcessingException;
+import org.apache.metron.utils.JSONUtils;
+
+import java.io.IOException;
+import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 
@@ -25,6 +30,8 @@ public class SensorEnrichmentConfig {
   private String index;
   private Map<String, List<String>> enrichmentFieldMap;
   private Map<String, List<String>> threatIntelFieldMap;
+  private Map<String, List<String>> fieldToEnrichmentTypeMap = new HashMap<>();
+  private Map<String, List<String>> fieldToThreatIntelTypeMap = new HashMap<>();
   private int batchSize;
 
   public String getIndex() {
@@ -51,6 +58,20 @@ public class SensorEnrichmentConfig {
     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;
   }
@@ -59,4 +80,43 @@ public class SensorEnrichmentConfig {
     this.batchSize = batchSize;
   }
 
+  public static SensorEnrichmentConfig fromBytes(byte[] config) throws IOException {
+    return JSONUtils.INSTANCE.load(new String(config), SensorEnrichmentConfig.class);
+  }
+  public String toJSON(boolean pretty) throws JsonProcessingException {
+    return JSONUtils.INSTANCE.toJSON(this, pretty);
+  }
+  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 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/9621c55e/metron-streaming/Metron-Common/src/main/java/org/apache/metron/enrichment/EnrichmentConfig.java
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-Common/src/main/java/org/apache/metron/enrichment/EnrichmentConfig.java b/metron-streaming/Metron-Common/src/main/java/org/apache/metron/enrichment/EnrichmentConfig.java
new file mode 100644
index 0000000..92913d9
--- /dev/null
+++ b/metron-streaming/Metron-Common/src/main/java/org/apache/metron/enrichment/EnrichmentConfig.java
@@ -0,0 +1,203 @@
+/**
+ * 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.enrichment;
+
+import com.google.common.base.Joiner;
+import org.apache.curator.framework.CuratorFramework;
+import org.apache.metron.Constants;
+import org.apache.metron.domain.SensorEnrichmentConfig;
+import org.apache.metron.utils.ConfigurationsUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+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 {
+      return SensorEnrichmentConfig.fromBytes(ConfigurationsUtils.readSensorEnrichmentConfigBytesFromZookeeper(sensor, client));
+    }
+
+    @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);
+        }
+        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);
+        }
+        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/9621c55e/metron-streaming/Metron-Common/src/main/java/org/apache/metron/hbase/converters/AbstractConverter.java
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-Common/src/main/java/org/apache/metron/hbase/converters/AbstractConverter.java b/metron-streaming/Metron-Common/src/main/java/org/apache/metron/hbase/converters/AbstractConverter.java
index 9072c22..c58dc22 100644
--- a/metron-streaming/Metron-Common/src/main/java/org/apache/metron/hbase/converters/AbstractConverter.java
+++ b/metron-streaming/Metron-Common/src/main/java/org/apache/metron/hbase/converters/AbstractConverter.java
@@ -35,59 +35,62 @@ import java.util.*;
 
 
 public abstract class AbstractConverter<KEY_T extends LookupKey, VALUE_T extends LookupValue> implements HbaseConverter<KEY_T,VALUE_T> {
-    public static Function<Cell, Map.Entry<byte[], byte[]>> CELL_TO_ENTRY  = new Function<Cell, Map.Entry<byte[], byte[]>>() {
+  public static Function<Cell, Map.Entry<byte[], byte[]>> CELL_TO_ENTRY  = new Function<Cell, Map.Entry<byte[], byte[]>>() {
 
-        @Nullable
-        @Override
-        public Map.Entry<byte[], byte[]> apply(@Nullable Cell cell) {
-            return new AbstractMap.SimpleEntry<byte[], byte[]>(cell.getQualifier(), cell.getValue());
-        }
-    };
+    @Nullable
     @Override
-    public Put toPut(String columnFamily, KEY_T key, VALUE_T values) throws IOException {
-        Put put = new Put(key.toBytes());
-        byte[] cf = Bytes.toBytes(columnFamily);
-        for(Map.Entry<byte[], byte[]> kv : values.toColumns()) {
-            put.add(cf, kv.getKey(), kv.getValue());
-        }
-        return put;
+    public Map.Entry<byte[], byte[]> apply(@Nullable Cell cell) {
+      return new AbstractMap.SimpleEntry<>(cell.getQualifier(), cell.getValue());
     }
-
-    public LookupKV<KEY_T, VALUE_T> fromPut(Put put, String columnFamily, KEY_T key, VALUE_T value) throws IOException {
-        key.fromBytes(put.getRow());
-        byte[] cf = Bytes.toBytes(columnFamily);
-        value.fromColumns(Iterables.transform(put.getFamilyCellMap().get(cf), CELL_TO_ENTRY));
-        return new LookupKV<>(key, value);
+  };
+  @Override
+  public Put toPut(String columnFamily, KEY_T key, VALUE_T values) throws IOException {
+    Put put = new Put(key.toBytes());
+    byte[] cf = Bytes.toBytes(columnFamily);
+    for(Map.Entry<byte[], byte[]> kv : values.toColumns()) {
+      put.add(cf, kv.getKey(), kv.getValue());
     }
+    return put;
+  }
 
-    @Override
-    public Result toResult(String columnFamily, KEY_T key, VALUE_T values) throws IOException {
-        Put put = toPut(columnFamily, key, values);
-        return Result.create(put.getFamilyCellMap().get(Bytes.toBytes(columnFamily)));
-    }
+  public LookupKV<KEY_T, VALUE_T> fromPut(Put put, String columnFamily, KEY_T key, VALUE_T value) throws IOException {
+    key.fromBytes(put.getRow());
+    byte[] cf = Bytes.toBytes(columnFamily);
+    value.fromColumns(Iterables.transform(put.getFamilyCellMap().get(cf), CELL_TO_ENTRY));
+    return new LookupKV<>(key, value);
+  }
 
-    public LookupKV<KEY_T, VALUE_T> fromResult(Result result, String columnFamily, KEY_T key, VALUE_T value) throws IOException {
-        key.fromBytes(result.getRow());
-        byte[] cf = Bytes.toBytes(columnFamily);
-        NavigableMap<byte[], byte[]> cols = result.getFamilyMap(cf);
-        value.fromColumns(cols.entrySet());
-        return new LookupKV<>(key, value);
-    }
-    @Override
-    public Get toGet(String columnFamily, KEY_T key) {
-        Get ret = new Get(key.toBytes());
-        ret.addFamily(Bytes.toBytes(columnFamily));
-        return ret;
+  @Override
+  public Result toResult(String columnFamily, KEY_T key, VALUE_T values) throws IOException {
+    Put put = toPut(columnFamily, key, values);
+    return Result.create(put.getFamilyCellMap().get(Bytes.toBytes(columnFamily)));
+  }
+
+  public LookupKV<KEY_T, VALUE_T> fromResult(Result result, String columnFamily, KEY_T key, VALUE_T value) throws IOException {
+    if(result == null || result.getRow() == null) {
+      return null;
     }
+    key.fromBytes(result.getRow());
+    byte[] cf = Bytes.toBytes(columnFamily);
+    NavigableMap<byte[], byte[]> cols = result.getFamilyMap(cf);
+    value.fromColumns(cols.entrySet());
+    return new LookupKV<>(key, value);
+  }
+  @Override
+  public Get toGet(String columnFamily, KEY_T key) {
+    Get ret = new Get(key.toBytes());
+    ret.addFamily(Bytes.toBytes(columnFamily));
+    return ret;
+  }
 
-    public static Iterable<Map.Entry<byte[], byte[]>> toEntries(byte[]... kvs) {
-        if(kvs.length % 2 != 0)  {
-            throw new IllegalStateException("Must be an even size");
-        }
-        List<Map.Entry<byte[], byte[]>> ret = new ArrayList<>(kvs.length/2);
-        for(int i = 0;i < kvs.length;i += 2) {
-            ret.add(new AbstractMap.SimpleImmutableEntry<>(kvs[i], kvs[i+1])) ;
-        }
-        return ret;
+  public static Iterable<Map.Entry<byte[], byte[]>> toEntries(byte[]... kvs) {
+    if(kvs.length % 2 != 0)  {
+      throw new IllegalStateException("Must be an even size");
+    }
+    List<Map.Entry<byte[], byte[]>> ret = new ArrayList<>(kvs.length/2);
+    for(int i = 0;i < kvs.length;i += 2) {
+      ret.add(new AbstractMap.SimpleImmutableEntry<>(kvs[i], kvs[i+1])) ;
     }
+    return ret;
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/9621c55e/metron-streaming/Metron-Common/src/main/java/org/apache/metron/hbase/converters/enrichment/EnrichmentConverter.java
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-Common/src/main/java/org/apache/metron/hbase/converters/enrichment/EnrichmentConverter.java b/metron-streaming/Metron-Common/src/main/java/org/apache/metron/hbase/converters/enrichment/EnrichmentConverter.java
new file mode 100644
index 0000000..a044498
--- /dev/null
+++ b/metron-streaming/Metron-Common/src/main/java/org/apache/metron/hbase/converters/enrichment/EnrichmentConverter.java
@@ -0,0 +1,38 @@
+/**
+ * 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.hbase.converters.enrichment;
+
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.metron.hbase.converters.AbstractConverter;
+import org.apache.metron.reference.lookup.LookupKV;
+
+import java.io.IOException;
+
+public class EnrichmentConverter extends AbstractConverter<EnrichmentKey, EnrichmentValue> {
+
+  @Override
+  public LookupKV<EnrichmentKey, EnrichmentValue> fromPut(Put put, String columnFamily) throws IOException {
+    return fromPut(put, columnFamily, new EnrichmentKey(), new EnrichmentValue());
+  }
+
+  @Override
+  public LookupKV<EnrichmentKey, EnrichmentValue> fromResult(Result result, String columnFamily) throws IOException {
+    return fromResult(result, columnFamily, new EnrichmentKey(), new EnrichmentValue());
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/9621c55e/metron-streaming/Metron-Common/src/main/java/org/apache/metron/hbase/converters/enrichment/EnrichmentKey.java
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-Common/src/main/java/org/apache/metron/hbase/converters/enrichment/EnrichmentKey.java b/metron-streaming/Metron-Common/src/main/java/org/apache/metron/hbase/converters/enrichment/EnrichmentKey.java
new file mode 100644
index 0000000..b688ad3
--- /dev/null
+++ b/metron-streaming/Metron-Common/src/main/java/org/apache/metron/hbase/converters/enrichment/EnrichmentKey.java
@@ -0,0 +1,119 @@
+/**
+ * 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.hbase.converters.enrichment;
+
+import com.google.common.base.Joiner;
+import com.google.common.base.Splitter;
+import com.google.common.collect.Iterables;
+import com.google.common.hash.HashFunction;
+import com.google.common.hash.Hasher;
+import com.google.common.hash.Hashing;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.metron.reference.lookup.LookupKey;
+
+import java.io.*;
+
+public class EnrichmentKey implements LookupKey {
+  private static final int SEED = 0xDEADBEEF;
+  private static final int HASH_PREFIX_SIZE=16;
+  ThreadLocal<HashFunction> hFunction= new ThreadLocal<HashFunction>() {
+    @Override
+    protected HashFunction initialValue() {
+      return Hashing.murmur3_128(SEED);
+    }
+  };
+
+  public String indicator;
+  public String type;
+
+  public EnrichmentKey() {
+
+  }
+  public EnrichmentKey(String type, String indicator) {
+    this.indicator = indicator;
+    this.type = type;
+  }
+
+  private byte[] typedIndicatorToBytes() throws IOException {
+    ByteArrayOutputStream baos = new ByteArrayOutputStream();
+    DataOutputStream w = new DataOutputStream(baos);
+    w.writeUTF(type);
+    w.writeUTF(indicator);
+    w.flush();
+    return baos.toByteArray();
+  }
+
+  @Override
+  public byte[] toBytes() {
+    byte[] indicatorBytes = new byte[0];
+    try {
+      indicatorBytes = typedIndicatorToBytes();
+    } catch (IOException e) {
+      throw new RuntimeException("Unable to convert type and indicator to bytes", e);
+    }
+    Hasher hasher = hFunction.get().newHasher();
+    hasher.putBytes(Bytes.toBytes(indicator));
+    byte[] prefix = hasher.hash().asBytes();
+    byte[] val = new byte[indicatorBytes.length + prefix.length];
+    int offset = 0;
+    System.arraycopy(prefix, 0, val, offset, prefix.length);
+    offset += prefix.length;
+    System.arraycopy(indicatorBytes, 0, val, offset, indicatorBytes.length);
+    return val;
+  }
+
+  @Override
+  public void fromBytes(byte[] row) {
+    ByteArrayInputStream baos = new ByteArrayInputStream(row);
+    baos.skip(HASH_PREFIX_SIZE);
+    DataInputStream w = new DataInputStream(baos);
+    try {
+      type = w.readUTF();
+      indicator = w.readUTF();
+    } catch (IOException e) {
+      throw new RuntimeException("Unable to convert type and indicator from bytes", e);
+    }
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) return true;
+    if (o == null || getClass() != o.getClass()) return false;
+
+    EnrichmentKey that = (EnrichmentKey) o;
+
+    if (indicator != null ? !indicator.equals(that.indicator) : that.indicator != null) return false;
+    return type != null ? type.equals(that.type) : that.type == null;
+
+  }
+
+  @Override
+  public int hashCode() {
+    int result = indicator != null ? indicator.hashCode() : 0;
+    result = 31 * result + (type != null ? type.hashCode() : 0);
+    return result;
+  }
+
+  @Override
+  public String toString() {
+    return "EnrichmentKey{" +
+            "indicator='" + indicator + '\'' +
+            ", type='" + type + '\'' +
+            '}';
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/9621c55e/metron-streaming/Metron-Common/src/main/java/org/apache/metron/hbase/converters/enrichment/EnrichmentValue.java
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-Common/src/main/java/org/apache/metron/hbase/converters/enrichment/EnrichmentValue.java b/metron-streaming/Metron-Common/src/main/java/org/apache/metron/hbase/converters/enrichment/EnrichmentValue.java
new file mode 100644
index 0000000..f733f8e
--- /dev/null
+++ b/metron-streaming/Metron-Common/src/main/java/org/apache/metron/hbase/converters/enrichment/EnrichmentValue.java
@@ -0,0 +1,107 @@
+/**
+ * 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.hbase.converters.enrichment;
+
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.metron.hbase.converters.AbstractConverter;
+import org.apache.metron.reference.lookup.LookupValue;
+import org.codehaus.jackson.map.ObjectMapper;
+import org.codehaus.jackson.type.TypeReference;
+
+import java.io.IOException;
+import java.util.Map;
+
+public class EnrichmentValue implements LookupValue {
+   private static final ThreadLocal<ObjectMapper> _mapper = new ThreadLocal<ObjectMapper>() {
+             @Override
+             protected ObjectMapper initialValue() {
+                return new ObjectMapper();
+             }
+    };
+    public static final String VALUE_COLUMN_NAME = "v";
+    public static final byte[] VALUE_COLUMN_NAME_B = Bytes.toBytes(VALUE_COLUMN_NAME);
+
+    private Map<String, String> metadata = null;
+
+    public EnrichmentValue()
+    {
+
+    }
+
+    public EnrichmentValue(Map<String, String> metadata) {
+        this.metadata = metadata;
+    }
+
+
+
+    public Map<String, String> getMetadata() {
+        return metadata;
+    }
+
+    @Override
+    public Iterable<Map.Entry<byte[], byte[]>> toColumns() {
+        return AbstractConverter.toEntries( VALUE_COLUMN_NAME_B, Bytes.toBytes(valueToString(metadata))
+                                  );
+    }
+
+    @Override
+    public void fromColumns(Iterable<Map.Entry<byte[], byte[]>> values) {
+        for(Map.Entry<byte[], byte[]> cell : values) {
+            if(Bytes.equals(cell.getKey(), VALUE_COLUMN_NAME_B)) {
+                metadata = stringToValue(Bytes.toString(cell.getValue()));
+            }
+        }
+    }
+    public Map<String, String> stringToValue(String s){
+        try {
+            return _mapper.get().readValue(s, new TypeReference<Map<String, String>>(){});
+        } catch (IOException e) {
+            throw new RuntimeException("Unable to convert string to metadata: " + s);
+        }
+    }
+    public String valueToString(Map<String, String> value) {
+        try {
+            return _mapper.get().writeValueAsString(value);
+        } catch (IOException e) {
+            throw new RuntimeException("Unable to convert metadata to string: " + value);
+        }
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (this == o) return true;
+        if (o == null || getClass() != o.getClass()) return false;
+
+        EnrichmentValue that = (EnrichmentValue) o;
+
+        return getMetadata() != null ? getMetadata().equals(that.getMetadata()) : that.getMetadata() == null;
+
+    }
+
+    @Override
+    public int hashCode() {
+        return getMetadata() != null ? getMetadata().hashCode() : 0;
+    }
+
+    @Override
+    public String toString() {
+        return "EnrichmentValue{" +
+                "metadata=" + metadata +
+                '}';
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/9621c55e/metron-streaming/Metron-Common/src/main/java/org/apache/metron/hbase/converters/threatintel/ThreatIntelConverter.java
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-Common/src/main/java/org/apache/metron/hbase/converters/threatintel/ThreatIntelConverter.java b/metron-streaming/Metron-Common/src/main/java/org/apache/metron/hbase/converters/threatintel/ThreatIntelConverter.java
deleted file mode 100644
index d534a52..0000000
--- a/metron-streaming/Metron-Common/src/main/java/org/apache/metron/hbase/converters/threatintel/ThreatIntelConverter.java
+++ /dev/null
@@ -1,39 +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.hbase.converters.threatintel;
-
-import org.apache.hadoop.hbase.client.Put;
-import org.apache.hadoop.hbase.client.Result;
-import org.apache.metron.hbase.converters.AbstractConverter;
-import org.apache.metron.reference.lookup.LookupKV;
-
-import java.io.IOException;
-
-public class ThreatIntelConverter extends AbstractConverter<ThreatIntelKey, ThreatIntelValue> {
-
-    @Override
-    public LookupKV<ThreatIntelKey, ThreatIntelValue> fromPut(Put put, String columnFamily) throws IOException {
-        return fromPut(put, columnFamily, new ThreatIntelKey(), new ThreatIntelValue());
-    }
-
-    @Override
-    public LookupKV<ThreatIntelKey, ThreatIntelValue> fromResult(Result result, String columnFamily) throws IOException {
-        return fromResult(result, columnFamily, new ThreatIntelKey(), new ThreatIntelValue());
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/9621c55e/metron-streaming/Metron-Common/src/main/java/org/apache/metron/hbase/converters/threatintel/ThreatIntelKey.java
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-Common/src/main/java/org/apache/metron/hbase/converters/threatintel/ThreatIntelKey.java b/metron-streaming/Metron-Common/src/main/java/org/apache/metron/hbase/converters/threatintel/ThreatIntelKey.java
deleted file mode 100644
index 3d898d9..0000000
--- a/metron-streaming/Metron-Common/src/main/java/org/apache/metron/hbase/converters/threatintel/ThreatIntelKey.java
+++ /dev/null
@@ -1,89 +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.hbase.converters.threatintel;
-
-import com.google.common.hash.HashFunction;
-import com.google.common.hash.Hasher;
-import com.google.common.hash.Hashing;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.metron.reference.lookup.LookupKey;
-
-public class ThreatIntelKey implements LookupKey{
-    private static final int SEED = 0xDEADBEEF;
-    private static final int HASH_PREFIX_SIZE=16;
-    ThreadLocal<HashFunction> hFunction= new ThreadLocal<HashFunction>() {
-        @Override
-        protected HashFunction initialValue() {
-            return Hashing.murmur3_128(SEED);
-        }
-    };
-    public ThreatIntelKey() {
-
-    }
-    public ThreatIntelKey(String indicator) {
-        this.indicator = indicator;
-    }
-
-    public String indicator;
-
-    @Override
-    public byte[] toBytes() {
-        byte[] indicatorBytes = Bytes.toBytes(indicator);
-        Hasher hasher = hFunction.get().newHasher();
-        hasher.putBytes(Bytes.toBytes(indicator));
-        byte[] prefix = hasher.hash().asBytes();
-        byte[] val = new byte[indicatorBytes.length + prefix.length];
-        int pos = 0;
-        for(int i = 0;pos < prefix.length;++pos,++i) {
-            val[pos] = prefix[i];
-        }
-        for(int i = 0;i < indicatorBytes.length;++pos,++i) {
-            val[pos] = indicatorBytes[i];
-        }
-        return val;
-    }
-
-    @Override
-    public void fromBytes(byte[] row) {
-        ThreatIntelKey key = this;
-        key.indicator = Bytes.toString(row, HASH_PREFIX_SIZE, row.length - HASH_PREFIX_SIZE);
-    }
-
-    @Override
-    public boolean equals(Object o) {
-        if (this == o) return true;
-        if (o == null || getClass() != o.getClass()) return false;
-
-        ThreatIntelKey that = (ThreatIntelKey) o;
-
-        return indicator != null ? indicator.equals(that.indicator) : that.indicator == null;
-
-    }
-
-    @Override
-    public int hashCode() {
-        return indicator != null ? indicator.hashCode() : 0;
-    }
-
-    @Override
-    public String toString() {
-        return "ThreatIntelKey{" +
-                "indicator='" + indicator + '\'' +
-                '}';
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/9621c55e/metron-streaming/Metron-Common/src/main/java/org/apache/metron/hbase/converters/threatintel/ThreatIntelValue.java
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-Common/src/main/java/org/apache/metron/hbase/converters/threatintel/ThreatIntelValue.java b/metron-streaming/Metron-Common/src/main/java/org/apache/metron/hbase/converters/threatintel/ThreatIntelValue.java
deleted file mode 100644
index 97f0762..0000000
--- a/metron-streaming/Metron-Common/src/main/java/org/apache/metron/hbase/converters/threatintel/ThreatIntelValue.java
+++ /dev/null
@@ -1,110 +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.hbase.converters.threatintel;
-
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.metron.hbase.converters.AbstractConverter;
-import org.apache.metron.reference.lookup.LookupValue;
-import org.codehaus.jackson.map.ObjectMapper;
-import org.codehaus.jackson.type.TypeReference;
-
-import java.io.IOException;
-import java.util.Map;
-
-public class ThreatIntelValue implements LookupValue {
-   private static final ThreadLocal<ObjectMapper> _mapper = new ThreadLocal<ObjectMapper>() {
-             @Override
-             protected ObjectMapper initialValue() {
-                return new ObjectMapper();
-             }
-    };
-    public static final String VALUE_COLUMN_NAME = "v";
-    public static final byte[] VALUE_COLUMN_NAME_B = Bytes.toBytes(VALUE_COLUMN_NAME);
-    public static final String LAST_SEEN_COLUMN_NAME = "t";
-    public static final byte[] LAST_SEEN_COLUMN_NAME_B = Bytes.toBytes(LAST_SEEN_COLUMN_NAME);
-
-    private Map<String, String> metadata = null;
-
-    public ThreatIntelValue()
-    {
-
-    }
-
-    public ThreatIntelValue(Map<String, String> metadata) {
-        this.metadata = metadata;
-    }
-
-
-
-    public Map<String, String> getMetadata() {
-        return metadata;
-    }
-
-    @Override
-    public Iterable<Map.Entry<byte[], byte[]>> toColumns() {
-        return AbstractConverter.toEntries( VALUE_COLUMN_NAME_B, Bytes.toBytes(valueToString(metadata))
-                                  );
-    }
-
-    @Override
-    public void fromColumns(Iterable<Map.Entry<byte[], byte[]>> values) {
-        for(Map.Entry<byte[], byte[]> cell : values) {
-            if(Bytes.equals(cell.getKey(), VALUE_COLUMN_NAME_B)) {
-                metadata = stringToValue(Bytes.toString(cell.getValue()));
-            }
-        }
-    }
-    public Map<String, String> stringToValue(String s){
-        try {
-            return _mapper.get().readValue(s, new TypeReference<Map<String, String>>(){});
-        } catch (IOException e) {
-            throw new RuntimeException("Unable to convert string to metadata: " + s);
-        }
-    }
-    public String valueToString(Map<String, String> value) {
-        try {
-            return _mapper.get().writeValueAsString(value);
-        } catch (IOException e) {
-            throw new RuntimeException("Unable to convert metadata to string: " + value);
-        }
-    }
-
-    @Override
-    public boolean equals(Object o) {
-        if (this == o) return true;
-        if (o == null || getClass() != o.getClass()) return false;
-
-        ThreatIntelValue that = (ThreatIntelValue) o;
-
-        return getMetadata() != null ? getMetadata().equals(that.getMetadata()) : that.getMetadata() == null;
-
-    }
-
-    @Override
-    public int hashCode() {
-        return getMetadata() != null ? getMetadata().hashCode() : 0;
-    }
-
-    @Override
-    public String toString() {
-        return "ThreatIntelValue{" +
-                "metadata=" + metadata +
-                '}';
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/9621c55e/metron-streaming/Metron-Common/src/main/java/org/apache/metron/hbase/lookup/EnrichmentLookup.java
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-Common/src/main/java/org/apache/metron/hbase/lookup/EnrichmentLookup.java b/metron-streaming/Metron-Common/src/main/java/org/apache/metron/hbase/lookup/EnrichmentLookup.java
new file mode 100644
index 0000000..059d6a6
--- /dev/null
+++ b/metron-streaming/Metron-Common/src/main/java/org/apache/metron/hbase/lookup/EnrichmentLookup.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.hbase.lookup;
+
+import com.google.common.collect.ImmutableList;
+import org.apache.hadoop.hbase.client.Get;
+import org.apache.hadoop.hbase.client.HTableInterface;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.metron.hbase.converters.HbaseConverter;
+import org.apache.metron.hbase.converters.enrichment.EnrichmentConverter;
+import org.apache.metron.hbase.converters.enrichment.EnrichmentKey;
+import org.apache.metron.hbase.converters.enrichment.EnrichmentValue;
+import org.apache.metron.reference.lookup.Lookup;
+import org.apache.metron.reference.lookup.LookupKV;
+import org.apache.metron.reference.lookup.accesstracker.AccessTracker;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+
+public class EnrichmentLookup extends Lookup<HTableInterface, EnrichmentKey, LookupKV<EnrichmentKey,EnrichmentValue>> implements AutoCloseable {
+
+  public static class Handler implements org.apache.metron.reference.lookup.handler.Handler<HTableInterface,EnrichmentKey,LookupKV<EnrichmentKey,EnrichmentValue>> {
+    String columnFamily;
+    HbaseConverter<EnrichmentKey, EnrichmentValue> converter = new EnrichmentConverter();
+    public Handler(String columnFamily) {
+      this.columnFamily = columnFamily;
+    }
+    @Override
+    public boolean exists(EnrichmentKey key, HTableInterface table, boolean logAccess) throws IOException {
+      return table.exists(converter.toGet(columnFamily, key));
+    }
+
+    @Override
+    public LookupKV<EnrichmentKey, EnrichmentValue> get(EnrichmentKey key, HTableInterface table, boolean logAccess) throws IOException {
+      return converter.fromResult(table.get(converter.toGet(columnFamily, key)), columnFamily);
+    }
+
+    private List<Get> keysToGets(Iterable<EnrichmentKey> keys) {
+      List<Get> ret = new ArrayList<>();
+      for(EnrichmentKey key : keys) {
+        ret.add(converter.toGet(columnFamily, key));
+      }
+      return ret;
+    }
+
+    @Override
+    public Iterable<Boolean> exists(Iterable<EnrichmentKey> key, HTableInterface table, boolean logAccess) throws IOException {
+      List<Boolean> ret = new ArrayList<>();
+      for(boolean b : table.existsAll(keysToGets(key))) {
+        ret.add(b);
+      }
+      return ret;
+    }
+
+    @Override
+    public Iterable<LookupKV<EnrichmentKey, EnrichmentValue>> get( Iterable<EnrichmentKey> keys
+                                                                 , HTableInterface table
+                                                                 , boolean logAccess
+                                                                 ) throws IOException
+    {
+      List<LookupKV<EnrichmentKey, EnrichmentValue>> ret = new ArrayList<>();
+      for(Result result : table.get(keysToGets(keys))) {
+        ret.add(converter.fromResult(result, columnFamily));
+      }
+      return ret;
+    }
+
+
+    @Override
+    public void close() throws Exception {
+
+    }
+  }
+  private HTableInterface table;
+  public EnrichmentLookup(HTableInterface table, String columnFamily, AccessTracker tracker) {
+    this.table = table;
+    this.setLookupHandler(new Handler(columnFamily));
+    this.setAccessTracker(tracker);
+  }
+
+  public HTableInterface getTable() {
+    return table;
+  }
+
+  @Override
+  public void close() throws Exception {
+    super.close();
+    table.close();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/9621c55e/metron-streaming/Metron-Common/src/main/java/org/apache/metron/reference/lookup/Lookup.java
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-Common/src/main/java/org/apache/metron/reference/lookup/Lookup.java b/metron-streaming/Metron-Common/src/main/java/org/apache/metron/reference/lookup/Lookup.java
index e93134e..bf871d1 100644
--- a/metron-streaming/Metron-Common/src/main/java/org/apache/metron/reference/lookup/Lookup.java
+++ b/metron-streaming/Metron-Common/src/main/java/org/apache/metron/reference/lookup/Lookup.java
@@ -23,53 +23,74 @@ import org.apache.metron.reference.lookup.handler.Handler;
 import java.io.IOException;
 
 public class Lookup<CONTEXT_T, KEY_T extends LookupKey, RESULT_T> implements Handler<CONTEXT_T, KEY_T, RESULT_T> {
-    private String name;
-    private AccessTracker accessTracker;
-    private Handler<CONTEXT_T, KEY_T, RESULT_T> lookupHandler;
+  private String name;
+  private AccessTracker accessTracker;
+  private Handler<CONTEXT_T, KEY_T, RESULT_T> lookupHandler;
 
-    public String getName() {
-        return name;
-    }
+  public String getName() {
+    return name;
+  }
 
-    public void setName(String name) {
-        this.name = name;
-    }
+  public void setName(String name) {
+    this.name = name;
+  }
 
-    public AccessTracker getAccessTracker() {
-        return accessTracker;
-    }
+  public AccessTracker getAccessTracker() {
+    return accessTracker;
+  }
 
-    public void setAccessTracker(AccessTracker accessTracker) {
-        this.accessTracker = accessTracker;
-    }
+  public void setAccessTracker(AccessTracker accessTracker) {
+    this.accessTracker = accessTracker;
+  }
 
-    public Handler< CONTEXT_T, KEY_T, RESULT_T > getLookupHandler() {
-        return lookupHandler;
-    }
+  public Handler< CONTEXT_T, KEY_T, RESULT_T > getLookupHandler() {
+    return lookupHandler;
+  }
+
+  public void setLookupHandler(Handler< CONTEXT_T, KEY_T, RESULT_T > lookupHandler) {
+    this.lookupHandler = lookupHandler;
+  }
 
-    public void setLookupHandler(Handler< CONTEXT_T, KEY_T, RESULT_T > lookupHandler) {
-        this.lookupHandler = lookupHandler;
+  @Override
+  public boolean exists(KEY_T key, CONTEXT_T context, boolean logAccess) throws IOException {
+    if(logAccess) {
+      accessTracker.logAccess(key);
     }
+    return lookupHandler.exists(key, context, logAccess);
+  }
 
-    @Override
-    public boolean exists(KEY_T key, CONTEXT_T context, boolean logAccess) throws IOException {
-        if(logAccess) {
-            accessTracker.logAccess(key);
-        }
-        return lookupHandler.exists(key, context, logAccess);
+  @Override
+  public RESULT_T get(KEY_T key, CONTEXT_T context, boolean logAccess) throws IOException {
+    if(logAccess) {
+      accessTracker.logAccess(key);
     }
+    return lookupHandler.get(key, context, logAccess);
+  }
 
-    @Override
-    public RESULT_T get(KEY_T key, CONTEXT_T context, boolean logAccess) throws IOException {
-        if(logAccess) {
-            accessTracker.logAccess(key);
-        }
-        return lookupHandler.get(key, context, logAccess);
+  @Override
+  public Iterable<Boolean> exists(Iterable<KEY_T> key, CONTEXT_T context, boolean logAccess) throws IOException {
+    if(logAccess) {
+      for (KEY_T k : key) {
+        accessTracker.logAccess(k);
+      }
     }
+    return lookupHandler.exists(key, context, logAccess);
+  }
 
-    @Override
-    public void close() throws Exception {
-        accessTracker.cleanup();
-        lookupHandler.close();
+
+  @Override
+  public Iterable<RESULT_T> get(Iterable<KEY_T> key, CONTEXT_T context, boolean logAccess) throws IOException {
+    if(logAccess) {
+      for (KEY_T k : key) {
+        accessTracker.logAccess(k);
+      }
     }
+    return lookupHandler.get(key, context, logAccess);
+  }
+
+  @Override
+  public void close() throws Exception {
+    accessTracker.cleanup();
+    lookupHandler.close();
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/9621c55e/metron-streaming/Metron-Common/src/main/java/org/apache/metron/reference/lookup/accesstracker/BloomAccessTracker.java
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-Common/src/main/java/org/apache/metron/reference/lookup/accesstracker/BloomAccessTracker.java b/metron-streaming/Metron-Common/src/main/java/org/apache/metron/reference/lookup/accesstracker/BloomAccessTracker.java
index d2700d6..2bb8414 100644
--- a/metron-streaming/Metron-Common/src/main/java/org/apache/metron/reference/lookup/accesstracker/BloomAccessTracker.java
+++ b/metron-streaming/Metron-Common/src/main/java/org/apache/metron/reference/lookup/accesstracker/BloomAccessTracker.java
@@ -43,6 +43,7 @@ public class BloomAccessTracker implements AccessTracker {
         public boolean equals(Object obj) {
             return this.getClass().equals(obj.getClass());
         }
+
     }
 
     private static Funnel<LookupKey> LOOKUPKEY_FUNNEL = new LookupKeyFunnel();

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/9621c55e/metron-streaming/Metron-Common/src/main/java/org/apache/metron/reference/lookup/accesstracker/NoopAccessTracker.java
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-Common/src/main/java/org/apache/metron/reference/lookup/accesstracker/NoopAccessTracker.java b/metron-streaming/Metron-Common/src/main/java/org/apache/metron/reference/lookup/accesstracker/NoopAccessTracker.java
new file mode 100644
index 0000000..fb6d3ba
--- /dev/null
+++ b/metron-streaming/Metron-Common/src/main/java/org/apache/metron/reference/lookup/accesstracker/NoopAccessTracker.java
@@ -0,0 +1,65 @@
+/**
+ * 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.reference.lookup.accesstracker;
+
+import org.apache.metron.reference.lookup.LookupKey;
+
+import java.io.IOException;
+import java.util.Map;
+
+public class NoopAccessTracker implements AccessTracker {
+  @Override
+  public void logAccess(LookupKey key) {
+
+  }
+
+  @Override
+  public void configure(Map<String, Object> config) {
+
+  }
+
+  @Override
+  public boolean hasSeen(LookupKey key) {
+    return false;
+  }
+
+  @Override
+  public String getName() {
+    return "noop";
+  }
+
+  @Override
+  public AccessTracker union(AccessTracker tracker) {
+    return null;
+  }
+
+  @Override
+  public void reset() {
+
+  }
+
+  @Override
+  public boolean isFull() {
+    return false;
+  }
+
+  @Override
+  public void cleanup() throws IOException {
+
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/9621c55e/metron-streaming/Metron-Common/src/main/java/org/apache/metron/reference/lookup/handler/Handler.java
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-Common/src/main/java/org/apache/metron/reference/lookup/handler/Handler.java b/metron-streaming/Metron-Common/src/main/java/org/apache/metron/reference/lookup/handler/Handler.java
index 9af8fbd..198f90e 100644
--- a/metron-streaming/Metron-Common/src/main/java/org/apache/metron/reference/lookup/handler/Handler.java
+++ b/metron-streaming/Metron-Common/src/main/java/org/apache/metron/reference/lookup/handler/Handler.java
@@ -22,6 +22,8 @@ import org.apache.metron.reference.lookup.LookupKey;
 import java.io.IOException;
 
 public interface Handler<CONTEXT_T, KEY_T extends LookupKey, RESULT_T> extends AutoCloseable{
-    boolean exists(KEY_T key, CONTEXT_T context, boolean logAccess) throws IOException;
-    RESULT_T get(KEY_T key, CONTEXT_T context, boolean logAccess) throws IOException;
+  boolean exists(KEY_T key, CONTEXT_T context, boolean logAccess) throws IOException;
+  RESULT_T get(KEY_T key, CONTEXT_T context, boolean logAccess) throws IOException;
+  Iterable<Boolean> exists(Iterable<KEY_T> key, CONTEXT_T context, boolean logAccess) throws IOException;
+  Iterable<RESULT_T> get(Iterable<KEY_T> key, CONTEXT_T context, boolean logAccess) throws IOException;
 }

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/9621c55e/metron-streaming/Metron-Common/src/main/java/org/apache/metron/threatintel/ThreatIntelResults.java
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-Common/src/main/java/org/apache/metron/threatintel/ThreatIntelResults.java b/metron-streaming/Metron-Common/src/main/java/org/apache/metron/threatintel/ThreatIntelResults.java
deleted file mode 100644
index 8186b38..0000000
--- a/metron-streaming/Metron-Common/src/main/java/org/apache/metron/threatintel/ThreatIntelResults.java
+++ /dev/null
@@ -1,71 +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.threatintel;
-
-import org.apache.metron.hbase.converters.threatintel.ThreatIntelKey;
-
-import java.util.HashMap;
-import java.util.Map;
-
-public class ThreatIntelResults {
-    private ThreatIntelKey key;
-    private Map<String, String> value;
-    public ThreatIntelResults() {
-        key = new ThreatIntelKey();
-        value = new HashMap<>();
-    }
-    public ThreatIntelResults(ThreatIntelKey key, Map<String, String> value) {
-        this.key = key;
-        this.value = value;
-    }
-
-    public ThreatIntelKey getKey() {
-        return key;
-    }
-
-    public Map<String, String> getValue() {
-        return value;
-    }
-
-    @Override
-    public boolean equals(Object o) {
-        if (this == o) return true;
-        if (o == null || getClass() != o.getClass()) return false;
-
-        ThreatIntelResults that = (ThreatIntelResults) o;
-
-        if (getKey() != null ? !getKey().equals(that.getKey()) : that.getKey() != null) return false;
-        return getValue() != null ? getValue().equals(that.getValue()) : that.getValue() == null;
-
-    }
-
-    @Override
-    public int hashCode() {
-        int result = getKey() != null ? getKey().hashCode() : 0;
-        result = 31 * result + (getValue() != null ? getValue().hashCode() : 0);
-        return result;
-    }
-
-    @Override
-    public String toString() {
-        return "ThreatIntelResults{" +
-                "key=" + key +
-                ", value=" + value +
-                '}';
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/9621c55e/metron-streaming/Metron-Common/src/main/java/org/apache/metron/threatintel/hbase/ThreatIntelLookup.java
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-Common/src/main/java/org/apache/metron/threatintel/hbase/ThreatIntelLookup.java b/metron-streaming/Metron-Common/src/main/java/org/apache/metron/threatintel/hbase/ThreatIntelLookup.java
deleted file mode 100644
index 13efc42..0000000
--- a/metron-streaming/Metron-Common/src/main/java/org/apache/metron/threatintel/hbase/ThreatIntelLookup.java
+++ /dev/null
@@ -1,74 +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.threatintel.hbase;
-
-import org.apache.hadoop.hbase.client.HTableInterface;
-import org.apache.metron.hbase.converters.HbaseConverter;
-import org.apache.metron.hbase.converters.threatintel.ThreatIntelConverter;
-import org.apache.metron.hbase.converters.threatintel.ThreatIntelKey;
-import org.apache.metron.hbase.converters.threatintel.ThreatIntelValue;
-import org.apache.metron.reference.lookup.Lookup;
-import org.apache.metron.reference.lookup.LookupKV;
-import org.apache.metron.reference.lookup.accesstracker.AccessTracker;
-
-import java.io.IOException;
-
-
-public class ThreatIntelLookup extends Lookup<HTableInterface, ThreatIntelKey, LookupKV<ThreatIntelKey,ThreatIntelValue>> implements AutoCloseable {
-
-
-
-    public static class Handler implements org.apache.metron.reference.lookup.handler.Handler<HTableInterface,ThreatIntelKey,LookupKV<ThreatIntelKey,ThreatIntelValue>> {
-        String columnFamily;
-        HbaseConverter<ThreatIntelKey, ThreatIntelValue> converter = new ThreatIntelConverter();
-        public Handler(String columnFamily) {
-            this.columnFamily = columnFamily;
-        }
-        @Override
-        public boolean exists(ThreatIntelKey key, HTableInterface table, boolean logAccess) throws IOException {
-            return table.exists(converter.toGet(columnFamily, key));
-        }
-
-        @Override
-        public LookupKV<ThreatIntelKey, ThreatIntelValue> get(ThreatIntelKey key, HTableInterface table, boolean logAccess) throws IOException {
-            return converter.fromResult(table.get(converter.toGet(columnFamily, key)), columnFamily);
-        }
-
-
-        @Override
-        public void close() throws Exception {
-
-        }
-    }
-    private HTableInterface table;
-    public ThreatIntelLookup(HTableInterface table, String columnFamily, AccessTracker tracker) {
-        this.table = table;
-        this.setLookupHandler(new Handler(columnFamily));
-        this.setAccessTracker(tracker);
-    }
-
-    public HTableInterface getTable() {
-        return table;
-    }
-
-    @Override
-    public void close() throws Exception {
-        super.close();
-        table.close();
-    }
-}


[3/6] incubator-metron git commit: METRON-93: Generalize the HBase threat intel infrastructure to support enrichments closes apache/incubator-metron#64

Posted by ce...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/9621c55e/metron-streaming/Metron-DataLoads/src/main/java/org/apache/metron/dataloads/taxii/TaxiiHandler.java
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-DataLoads/src/main/java/org/apache/metron/dataloads/taxii/TaxiiHandler.java b/metron-streaming/Metron-DataLoads/src/main/java/org/apache/metron/dataloads/taxii/TaxiiHandler.java
deleted file mode 100644
index 614adec..0000000
--- a/metron-streaming/Metron-DataLoads/src/main/java/org/apache/metron/dataloads/taxii/TaxiiHandler.java
+++ /dev/null
@@ -1,403 +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.dataloads.taxii;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.client.HTable;
-import org.apache.hadoop.hbase.client.HTableInterface;
-import org.apache.hadoop.hbase.client.Put;
-import org.apache.http.HttpHost;
-import org.apache.http.auth.AuthScope;
-import org.apache.http.auth.UsernamePasswordCredentials;
-import org.apache.http.client.AuthCache;
-import org.apache.http.client.CredentialsProvider;
-import org.apache.http.client.protocol.HttpClientContext;
-import org.apache.http.config.Registry;
-import org.apache.http.config.RegistryBuilder;
-import org.apache.http.conn.socket.ConnectionSocketFactory;
-import org.apache.http.conn.socket.PlainConnectionSocketFactory;
-import org.apache.http.conn.ssl.SSLConnectionSocketFactory;
-import org.apache.http.conn.ssl.SSLContextBuilder;
-import org.apache.http.conn.ssl.TrustSelfSignedStrategy;
-import org.apache.http.impl.auth.BasicScheme;
-import org.apache.http.impl.client.BasicAuthCache;
-import org.apache.http.impl.client.BasicCredentialsProvider;
-import org.apache.http.impl.client.CloseableHttpClient;
-import org.apache.http.impl.client.HttpClientBuilder;
-import org.apache.http.impl.conn.PoolingHttpClientConnectionManager;
-import org.apache.log4j.Logger;
-import org.apache.metron.dataloads.extractor.Extractor;
-import org.apache.metron.hbase.converters.threatintel.ThreatIntelConverter;
-import org.apache.metron.hbase.converters.threatintel.ThreatIntelKey;
-import org.apache.metron.hbase.converters.threatintel.ThreatIntelValue;
-import org.apache.metron.reference.lookup.LookupKV;
-import org.mitre.taxii.client.HttpClient;
-import org.mitre.taxii.messages.xml11.*;
-import org.w3c.dom.Document;
-import org.w3c.dom.Element;
-
-import javax.xml.bind.JAXBException;
-import javax.xml.datatype.DatatypeConfigurationException;
-import javax.xml.datatype.DatatypeFactory;
-import javax.xml.datatype.XMLGregorianCalendar;
-import javax.xml.transform.Transformer;
-import javax.xml.transform.TransformerException;
-import javax.xml.transform.TransformerFactory;
-import javax.xml.transform.dom.DOMSource;
-import javax.xml.transform.stream.StreamResult;
-import java.io.*;
-import java.net.URI;
-import java.net.URISyntaxException;
-import java.net.URL;
-import java.text.SimpleDateFormat;
-import java.util.*;
-
-public class TaxiiHandler extends TimerTask {
-    private static final Logger LOG = Logger.getLogger(TaxiiHandler.class);
-
-    private static ThreadLocal<TaxiiXmlFactory> xmlFactory = new ThreadLocal<TaxiiXmlFactory>() {
-        @Override
-        protected TaxiiXmlFactory initialValue() {
-            return new TaxiiXmlFactory();
-        }
-    };
-    private static ThreadLocal<ObjectFactory> messageFactory = new ThreadLocal<ObjectFactory>() {
-        @Override
-        protected ObjectFactory initialValue() {
-            return new ObjectFactory();
-        }
-    };
-
-    private HttpClient taxiiClient;
-    private URL endpoint;
-    private Extractor extractor;
-    private Map<String, TableInfo> tableMap;
-    private Map<TableInfo, HTableInterface> connectionCache = new HashMap<>();
-    private HttpClientContext context;
-    private String collection;
-    private String subscriptionId;
-    private ThreatIntelConverter converter = new ThreatIntelConverter();
-    private Date beginTime;
-    private Configuration config;
-    private boolean inProgress = false;
-    public TaxiiHandler( TaxiiConnectionConfig connectionConfig
-                       , Extractor extractor
-                       , Configuration config
-                       ) throws Exception
-    {
-        LOG.info("Loading configuration: " + connectionConfig);
-        this.extractor = extractor;
-        this.collection = connectionConfig.getCollection();
-        this.subscriptionId = connectionConfig.getSubscriptionId();
-        this.tableMap = connectionConfig.getTableMap();
-        this.beginTime = connectionConfig.getBeginTime();
-        this.config = config;
-        initializeClient(connectionConfig);
-        LOG.info("Configured, starting polling " + endpoint + " for " + collection);
-    }
-
-    protected synchronized HTableInterface getTable(TableInfo tableInfo) throws IOException {
-        HTableInterface ret = connectionCache.get(tableInfo);
-        if(ret == null) {
-            ret = createHTable(tableInfo);
-            connectionCache.put(tableInfo, ret);
-        }
-        return ret;
-    }
-
-    protected synchronized HTableInterface createHTable(TableInfo tableInfo) throws IOException {
-        return new HTable(config, tableInfo.getTableName());
-    }
-    /**
-     * The action to be performed by this timer task.
-     */
-    @Override
-    public void run() {
-        if(inProgress) {
-            return;
-        }
-        Date ts = new Date();
-        LOG.info("Polling..." + new SimpleDateFormat().format(ts));
-        try {
-            inProgress = true;
-            // Prepare the message to send.
-            String sessionID = MessageHelper.generateMessageId();
-            PollRequest request = messageFactory.get().createPollRequest()
-                    .withMessageId(sessionID)
-                    .withCollectionName(collection);
-            if (subscriptionId != null) {
-                request = request.withSubscriptionID(subscriptionId);
-            } else {
-                request = request.withPollParameters(messageFactory.get().createPollParametersType());
-            }
-            if (beginTime != null) {
-                Calendar gc = GregorianCalendar.getInstance();
-                gc.setTime(beginTime);
-                XMLGregorianCalendar gTime = null;
-                try {
-                    gTime = DatatypeFactory.newInstance().newXMLGregorianCalendar((GregorianCalendar) gc).normalize();
-                } catch (DatatypeConfigurationException e) {
-                    LOG.error("Unable to set the begin time", e);
-                }
-                gTime.setFractionalSecond(null);
-                LOG.info("Begin Time: " + gTime);
-                request.setExclusiveBeginTimestamp(gTime);
-            }
-
-            try {
-                PollResponse response = call(request, PollResponse.class);
-                LOG.info("Got Poll Response with " + response.getContentBlocks().size() + " blocks");
-                int numProcessed = 0;
-                long avgTimeMS = 0;
-                long timeStartedBlock = System.currentTimeMillis();
-                for (ContentBlock block : response.getContentBlocks()) {
-                    AnyMixedContentType content = block.getContent();
-                    for (Object o : content.getContent()) {
-                        numProcessed++;
-                        long timeS = System.currentTimeMillis();
-                        String xml = null;
-                        if (o instanceof Element) {
-                            Element element = (Element) o;
-                            xml = getStringFromDocument(element.getOwnerDocument());
-                            if(LOG.isDebugEnabled() && Math.random() < 0.01) {
-                                LOG.debug("Random Stix doc: " + xml);
-                            }
-                            for (LookupKV<ThreatIntelKey, ThreatIntelValue> kv : extractor.extract(xml)) {
-                                String indicatorType = kv.getValue().getMetadata().get("indicator-type");
-                                TableInfo tableInfo = tableMap.get(indicatorType);
-                                boolean persisted = false;
-                                if (tableInfo != null) {
-                                    kv.getValue().getMetadata().put("source_type", "taxii");
-                                    kv.getValue().getMetadata().put("taxii_url", endpoint.toString());
-                                    kv.getValue().getMetadata().put("taxii_collection", collection);
-                                    Put p = converter.toPut(tableInfo.getColumnFamily(), kv.getKey(), kv.getValue());
-                                    HTableInterface table = getTable(tableInfo);
-                                    table.put(p);
-                                    persisted = true;
-                                }
-                                LOG.info("Found Threat Intel: " + persisted + ", " + kv.getKey() + " => " + kv.getValue());
-                            }
-                        }
-                        avgTimeMS += System.currentTimeMillis() - timeS;
-                    }
-                    if( (numProcessed + 1) % 100 == 0) {
-                        LOG.info("Processed " + numProcessed + " in " + (System.currentTimeMillis() - timeStartedBlock) + " ms, avg time: " + avgTimeMS / content.getContent().size());
-                        timeStartedBlock = System.currentTimeMillis();
-                        avgTimeMS = 0;
-                        numProcessed = 0;
-                    }
-                }
-            } catch (Exception e) {
-                LOG.error(e.getMessage(), e);
-                throw new RuntimeException("Unable to make request", e);
-            }
-        }
-        finally {
-            inProgress = false;
-            beginTime = ts;
-        }
-    }
-    public String getStringFromDocument(Document doc)
-    {
-        try
-        {
-            DOMSource domSource = new DOMSource(doc);
-            StringWriter writer = new StringWriter();
-            StreamResult result = new StreamResult(writer);
-            TransformerFactory tf = TransformerFactory.newInstance();
-            Transformer transformer = tf.newTransformer();
-            transformer.transform(domSource, result);
-            return writer.toString();
-        }
-        catch(TransformerException ex)
-        {
-            ex.printStackTrace();
-            return null;
-        }
-    }
-    private <RESPONSE_T> RESPONSE_T call( Object request, Class<RESPONSE_T> responseClazz) throws URISyntaxException, JAXBException, IOException {
-        return call(taxiiClient, endpoint.toURI(), request, context, responseClazz);
-    }
-
-    private void initializeClient(TaxiiConnectionConfig config) throws Exception {
-        LOG.info("Initializing client..");
-        if(context == null) {
-            context = createContext(config.getEndpoint(), config.getUsername(), config.getPassword(), config.getPort());
-        }
-        URL endpoint = config.getEndpoint();
-        if(config.getType() == ConnectionType.DISCOVER) {
-            LOG.info("Discovering endpoint");
-            endpoint = discoverPollingClient(config.getProxy(), endpoint, config.getUsername(), config.getPassword(), context, collection).pollEndpoint;
-            this.endpoint = endpoint;
-            LOG.info("Discovered endpoint as " + endpoint);
-        }
-        taxiiClient = buildClient(config.getProxy(), config.getUsername(), config.getPassword());
-    }
-
-    private static class DiscoveryResults {
-        URL pollEndpoint;
-        URL collectionManagementEndpoint;
-        List<String> collections = new ArrayList<>();
-    }
-    private static DiscoveryResults discoverPollingClient(URL proxy, URL endpoint, String username, String password, HttpClientContext context, String defaultCollection) throws Exception {
-
-        DiscoveryResults results = new DiscoveryResults();
-        {
-            HttpClient discoverClient = buildClient(proxy, username, password);
-            String sessionID = MessageHelper.generateMessageId();
-            // Prepare the message to send.
-            DiscoveryRequest request = messageFactory.get().createDiscoveryRequest()
-                    .withMessageId(sessionID);
-            DiscoveryResponse response = call(discoverClient, endpoint.toURI(), request, context, DiscoveryResponse.class);
-            for (ServiceInstanceType serviceInstance : response.getServiceInstances()) {
-                if (serviceInstance.isAvailable() && serviceInstance.getServiceType() == ServiceTypeEnum.POLL) {
-                    results.pollEndpoint = new URL(serviceInstance.getAddress());
-                }
-                else if(serviceInstance.isAvailable() && serviceInstance.getServiceType() == ServiceTypeEnum.COLLECTION_MANAGEMENT) {
-                    results.collectionManagementEndpoint= new URL(serviceInstance.getAddress());
-                }
-            }
-            if (results.pollEndpoint == null) {
-                throw new RuntimeException("Unable to discover a poll TAXII feed");
-            }
-        }
-        if(defaultCollection == null)
-        //get collections
-        {
-            HttpClient discoverClient = buildClient(proxy, username, password);
-            String sessionID = MessageHelper.generateMessageId();
-            CollectionInformationRequest request = messageFactory.get().createCollectionInformationRequest()
-                                                                 .withMessageId(sessionID);
-            CollectionInformationResponse response = call(discoverClient, results.collectionManagementEndpoint.toURI(), request, context, CollectionInformationResponse.class);
-            LOG.info("Unable to find the default collection; available collections are:");
-            for(CollectionRecordType c : response.getCollections()) {
-                LOG.info(c.getCollectionName());
-                results.collections.add(c.getCollectionName());
-            }
-            System.exit(0);
-        }
-        return results;
-    }
-
-    private static HttpClientContext createContext(URL endpoint, String username, String password, int port) {
-        HttpClientContext context = null;
-        HttpHost target = new HttpHost(endpoint.getHost(), port, endpoint.getProtocol());
-        if (username != null && password != null) {
-
-            CredentialsProvider credsProvider = new BasicCredentialsProvider();
-            credsProvider.setCredentials(
-                    new AuthScope(target.getHostName(), target.getPort()),
-                    new UsernamePasswordCredentials(username, password));
-
-            // http://hc.apache.org/httpcomponents-client-ga/tutorial/html/authentication.html
-            AuthCache authCache = new BasicAuthCache();
-            authCache.put(target, new BasicScheme());
-
-            // Add AuthCache to the execution context
-            context = HttpClientContext.create();
-            context.setCredentialsProvider(credsProvider);
-            context.setAuthCache(authCache);
-        } else {
-            context = null;
-        }
-        return context;
-    }
-
-
-    public static <RESPONSE_T, REQUEST_T> RESPONSE_T call( HttpClient taxiiClient
-            , URI endpoint
-            , REQUEST_T request
-            , HttpClientContext context
-            , Class<RESPONSE_T> responseClazz
-    ) throws JAXBException, IOException {
-        //TaxiiXml taxiiXml = xmlFactory.get().createTaxiiXml();
-        //String req = taxiiXml.marshalToString(request, true);
-        // Call the service
-        Object responseObj =  taxiiClient.callTaxiiService(endpoint, request, context);
-        LOG.info("Request made : " + request.getClass().getCanonicalName() + " => " + responseObj.getClass().getCanonicalName() + " (expected " + responseClazz.getCanonicalName() + ")");
-        //String resp = taxiiXml.marshalToString(responseObj, true);
-        try {
-            return responseClazz.cast(responseObj);
-        }
-        catch(ClassCastException cce) {
-            TaxiiXml taxiiXml = xmlFactory.get().createTaxiiXml();
-            String resp = taxiiXml.marshalToString(responseObj, true);
-            String msg = "Didn't return the response we expected: " + responseObj.getClass() + " \n" + resp;
-            LOG.error(msg, cce);
-            throw new RuntimeException(msg, cce);
-        }
-    }
-    private static HttpClient buildClient(URL proxy, String username, String password) throws Exception
-    {
-        HttpClient client = new HttpClient(); // Start with a default TAXII HTTP client.
-
-        // Create an Apache HttpClientBuilder to be customized by the command line arguments.
-        HttpClientBuilder builder = HttpClientBuilder.create().useSystemProperties();
-
-        // Proxy
-        if (proxy != null) {
-            HttpHost proxyHost = new HttpHost(proxy.getHost(), proxy.getPort(), proxy.getProtocol());
-            builder.setProxy(proxyHost);
-        }
-
-        // Basic authentication. User & Password
-        if (username != null ^ password != null) {
-            throw new Exception("'username' and 'password' arguments are required to appear together.");
-        }
-
-
-        // from:  http://stackoverflow.com/questions/19517538/ignoring-ssl-certificate-in-apache-httpclient-4-3
-        SSLContextBuilder ssbldr = new SSLContextBuilder();
-        ssbldr.loadTrustMaterial(null, new TrustSelfSignedStrategy());
-        SSLConnectionSocketFactory sslsf = new SSLConnectionSocketFactory(ssbldr.build(),SSLConnectionSocketFactory.BROWSER_COMPATIBLE_HOSTNAME_VERIFIER);
-
-
-        Registry<ConnectionSocketFactory> registry = RegistryBuilder.<ConnectionSocketFactory>create()
-                .register("http", new PlainConnectionSocketFactory())
-                .register("https", sslsf)
-                .build();
-
-
-        PoolingHttpClientConnectionManager cm = new PoolingHttpClientConnectionManager(registry);
-        cm.setMaxTotal(20);//max connection
-
-        System.setProperty("jsse.enableSNIExtension", "false"); //""
-        CloseableHttpClient httpClient = builder
-                .setSSLSocketFactory(sslsf)
-                .setConnectionManager(cm)
-                .build();
-
-        client.setHttpclient(httpClient);
-        return client;
-    }
-    public static void main(String... argv) throws Exception {
-        URL endpoint = new URL("http://hailataxii.com/taxii-discovery-service");
-        String username = "guest";
-        String password = "guest";
-        TaxiiConnectionConfig config = new TaxiiConnectionConfig();
-        config = config.withConnectionType(ConnectionType.DISCOVER)
-                       .withEndpoint(endpoint)
-                       .withUsername(username)
-                       .withCollection("guest.Abuse_ch")
-                       .withPassword(password);
-        //TaxiiHandler handler = new TaxiiHandler(config, null);
-        //handler.run();
-        //discoverPollingClient(null, endpoint, username, password, context);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/9621c55e/metron-streaming/Metron-DataLoads/src/main/java/org/apache/metron/dataloads/taxii/TaxiiLoader.java
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-DataLoads/src/main/java/org/apache/metron/dataloads/taxii/TaxiiLoader.java b/metron-streaming/Metron-DataLoads/src/main/java/org/apache/metron/dataloads/taxii/TaxiiLoader.java
deleted file mode 100644
index 4f307c5..0000000
--- a/metron-streaming/Metron-DataLoads/src/main/java/org/apache/metron/dataloads/taxii/TaxiiLoader.java
+++ /dev/null
@@ -1,180 +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.dataloads.taxii;
-
-import com.google.common.base.Function;
-import com.google.common.base.Joiner;
-import org.apache.commons.cli.*;
-import org.apache.commons.cli.ParseException;
-import org.apache.commons.io.FileUtils;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.HBaseConfiguration;
-import org.apache.hadoop.util.GenericOptionsParser;
-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 javax.annotation.Nullable;
-import java.io.File;
-import java.io.IOException;
-import java.text.*;
-import java.util.Date;
-import java.util.Map;
-import java.util.Timer;
-
-public class TaxiiLoader {
-private static abstract class OptionHandler implements Function<String, Option> {}
-    private enum TaxiiOptions {
-        HELP("h", new OptionHandler() {
-
-            @Nullable
-            @Override
-            public Option apply(@Nullable String s) {
-                return new Option(s, "help", false, "Generate Help screen");
-            }
-        })
-        ,EXTRACTOR_CONFIG("e", new OptionHandler() {
-            @Nullable
-            @Override
-            public Option apply(@Nullable String s) {
-                Option o = new Option(s, "extractor_config", true, "JSON Document describing the extractor for this input data source");
-                o.setArgName("JSON_FILE");
-                o.setRequired(true);
-                return o;
-            }
-        })
-        ,CONNECTION_CONFIG("c", new OptionHandler() {
-            @Nullable
-            @Override
-            public Option apply(@Nullable String s) {
-                Option o = new Option(s, "taxii_connection_config", true, "The JSON config file to configure the connection");
-                o.setArgName("config_file");
-                o.setRequired(true);
-                return o;
-            }
-        })
-        ,TIME_BETWEEN_POLLS("p", new OptionHandler() {
-            @Nullable
-            @Override
-            public Option apply(@Nullable String s) {
-                Option o = new Option(s, "time_between_polls", true, "The time between polls (in ms)");
-                o.setArgName("MS");
-                o.setRequired(false);
-                return o;
-            }
-        })
-        ,BEGIN_TIME("b", new OptionHandler() {
-            @Nullable
-            @Override
-            public Option apply(@Nullable String s) {
-                Option o = new Option(s, "begin_time", true, "Start time to poll the Taxii server (all data from that point will be gathered in the first pull).");
-                o.setArgName(DATE_FORMAT.toPattern());
-                o.setRequired(false);
-                return o;
-            }
-        })
-        ,LOG4J_PROPERTIES("l", new OptionHandler() {
-            @Nullable
-            @Override
-            public Option apply(@Nullable String s) {
-                Option o = new Option(s, "log4j", true, "The log4j properties file to load");
-                o.setArgName("FILE");
-                o.setRequired(false);
-                return o;
-            }
-        })
-        ;
-        Option option;
-        String shortCode;
-        TaxiiOptions(String shortCode, OptionHandler 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(TaxiiOptions.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( "TaxiiLoader", getOptions());
-        }
-
-        public static Options getOptions() {
-            Options ret = new Options();
-            for(TaxiiOptions o : TaxiiOptions.values()) {
-               ret.addOption(o.option);
-            }
-            return ret;
-        }
-    }
-    public static final SimpleDateFormat DATE_FORMAT = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss");
-    public static final long ONE_HR_IN_MS = 60*60*1000;
-    public static final long DEFAULT_TIME_BETWEEN_POLLS = ONE_HR_IN_MS;
-    public static void main(String... argv) throws Exception {
-        Configuration conf = HBaseConfiguration.create();
-        String[] otherArgs = new GenericOptionsParser(conf, argv).getRemainingArgs();
-
-        CommandLine cli = TaxiiOptions.parse(new PosixParser(), otherArgs);
-        if(TaxiiOptions.LOG4J_PROPERTIES.has(cli)) {
-            PropertyConfigurator.configure(TaxiiOptions.LOG4J_PROPERTIES.get(cli));
-        }
-        ExtractorHandler handler = ExtractorHandler.load(FileUtils.readFileToString(new File(TaxiiOptions.EXTRACTOR_CONFIG.get(cli))));
-        Extractor e = handler.getExtractor();
-        Timer timer = new Timer();
-        if(e instanceof StixExtractor) {
-            StixExtractor extractor = (StixExtractor)e;
-            TaxiiConnectionConfig connectionConfig = TaxiiConnectionConfig.load(FileUtils.readFileToString(new File(TaxiiOptions.CONNECTION_CONFIG.get(cli))));
-            if(TaxiiOptions.BEGIN_TIME.has(cli)) {
-                Date d = DATE_FORMAT.parse(TaxiiOptions.BEGIN_TIME.get(cli));
-                connectionConfig.withBeginTime(d);
-            }
-            long timeBetween = DEFAULT_TIME_BETWEEN_POLLS;
-            if(TaxiiOptions.TIME_BETWEEN_POLLS.has(cli)) {
-                timeBetween = Long.parseLong(TaxiiOptions.TIME_BETWEEN_POLLS.get(cli));
-            }
-            timer.scheduleAtFixedRate(new TaxiiHandler(connectionConfig, extractor, conf), 0, timeBetween);
-        }
-        else {
-            throw new IllegalStateException("Extractor must be a STIX Extractor");
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/9621c55e/metron-streaming/Metron-DataLoads/src/test/java/org/apache/metron/dataloads/extractor/ExtractorTest.java
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-DataLoads/src/test/java/org/apache/metron/dataloads/extractor/ExtractorTest.java b/metron-streaming/Metron-DataLoads/src/test/java/org/apache/metron/dataloads/extractor/ExtractorTest.java
index a124710..a999080 100644
--- a/metron-streaming/Metron-DataLoads/src/test/java/org/apache/metron/dataloads/extractor/ExtractorTest.java
+++ b/metron-streaming/Metron-DataLoads/src/test/java/org/apache/metron/dataloads/extractor/ExtractorTest.java
@@ -18,10 +18,9 @@
 package org.apache.metron.dataloads.extractor;
 
 import com.google.common.collect.Iterables;
-import org.apache.metron.hbase.converters.threatintel.ThreatIntelKey;
-import org.apache.metron.hbase.converters.threatintel.ThreatIntelValue;
+import org.apache.metron.hbase.converters.enrichment.EnrichmentKey;
+import org.apache.metron.hbase.converters.enrichment.EnrichmentValue;
 import org.apache.metron.reference.lookup.LookupKV;
-import org.apache.metron.threatintel.ThreatIntelResults;
 import org.junit.Assert;
 import org.junit.Test;
 
@@ -36,11 +35,12 @@ public class ExtractorTest {
 
         @Override
         public Iterable<LookupKV> extract(String line) throws IOException {
-            ThreatIntelKey key = new ThreatIntelKey();
+            EnrichmentKey key = new EnrichmentKey();
             key.indicator = "dummy";
+            key.type = "type";
             Map<String, String> value = new HashMap<>();
             value.put("indicator", "dummy");
-            return Arrays.asList(new LookupKV(key, new ThreatIntelValue(value)));
+            return Arrays.asList(new LookupKV(key, new EnrichmentValue(value)));
         }
 
         @Override
@@ -52,9 +52,10 @@ public class ExtractorTest {
     public void testDummyExtractor() throws IllegalAccessException, InstantiationException, ClassNotFoundException, IOException {
         Extractor extractor = Extractors.create(DummyExtractor.class.getName());
         LookupKV results = Iterables.getFirst(extractor.extract(null), null);
-        ThreatIntelKey key = (ThreatIntelKey) results.getKey();
-        ThreatIntelValue value = (ThreatIntelValue) results.getValue();
+        EnrichmentKey key = (EnrichmentKey) results.getKey();
+        EnrichmentValue value = (EnrichmentValue) results.getValue();
         Assert.assertEquals("dummy", key.indicator);
+        Assert.assertEquals("type", key.type);
         Assert.assertEquals("dummy", value.getMetadata().get("indicator"));
     }
 
@@ -73,9 +74,10 @@ public class ExtractorTest {
                 "         }";
         ExtractorHandler handler = ExtractorHandler.load(config);
         LookupKV results = Iterables.getFirst(handler.getExtractor().extract(null), null);
-        ThreatIntelKey key = (ThreatIntelKey) results.getKey();
-        ThreatIntelValue value = (ThreatIntelValue) results.getValue();
+        EnrichmentKey key = (EnrichmentKey) results.getKey();
+        EnrichmentValue value = (EnrichmentValue) results.getValue();
         Assert.assertEquals("dummy", key.indicator);
+        Assert.assertEquals("type", key.type);
         Assert.assertEquals("dummy", value.getMetadata().get("indicator"));
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/9621c55e/metron-streaming/Metron-DataLoads/src/test/java/org/apache/metron/dataloads/extractor/csv/CSVExtractorTest.java
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-DataLoads/src/test/java/org/apache/metron/dataloads/extractor/csv/CSVExtractorTest.java b/metron-streaming/Metron-DataLoads/src/test/java/org/apache/metron/dataloads/extractor/csv/CSVExtractorTest.java
index 682b230..f01f619 100644
--- a/metron-streaming/Metron-DataLoads/src/test/java/org/apache/metron/dataloads/extractor/csv/CSVExtractorTest.java
+++ b/metron-streaming/Metron-DataLoads/src/test/java/org/apache/metron/dataloads/extractor/csv/CSVExtractorTest.java
@@ -18,104 +18,56 @@
 package org.apache.metron.dataloads.extractor.csv;
 
 import com.google.common.collect.Iterables;
+import org.adrianwalker.multilinestring.Multiline;
 import org.apache.metron.dataloads.extractor.ExtractorHandler;
-import org.apache.metron.hbase.converters.threatintel.ThreatIntelKey;
-import org.apache.metron.hbase.converters.threatintel.ThreatIntelValue;
+import org.apache.metron.hbase.converters.enrichment.EnrichmentKey;
+import org.apache.metron.hbase.converters.enrichment.EnrichmentValue;
 import org.apache.metron.reference.lookup.LookupKV;
-import org.apache.metron.threatintel.ThreatIntelResults;
 import org.junit.Assert;
 import org.junit.Test;
 
 import java.io.IOException;
 
 public class CSVExtractorTest {
-    @Test
-    public void testCSVExtractorMapColumns() throws Exception {
-        /**
-         {
-            "config" : {
-                        "columns" : {
-                                "host" : 0
-                                ,"meta" : 2
-                                    }
-                       ,"indicator_column" : "host"
-                       ,"separator" : ","
-                       }
-            ,"extractor" : "CSV"
-         }
-         */
-        String config = "{\n" +
-                "            \"config\" : {\n" +
-                "                        \"columns\" : [\"host:0\",\"meta:2\"]\n" +
-                "                       ,\"indicator_column\" : \"host\"\n" +
-                "                       ,\"separator\" : \",\"\n" +
-                "                       }\n" +
-                "            ,\"extractor\" : \"CSV\"\n" +
-                "         }";
-        ExtractorHandler handler = ExtractorHandler.load(config);
-        validate(handler);
-    }
-    @Test
-    public void testCSVExtractorListColumns() throws Exception {
-        /**
-         {
-            "config" : {
-                        "columns" : ["host:0","meta:2"]
-                       ,"indicator_column" : "host"
-                       ,"separator" : ","
-                       }
-            ,"extractor" : "CSV"
-         }
-         */
-        String config = "{\n" +
-                "            \"config\" : {\n" +
-                "                        \"columns\" : [\"host:0\",\"meta:2\"]\n" +
-                "                       ,\"indicator_column\" : \"host\"\n" +
-                "                       ,\"separator\" : \",\"\n" +
-                "                       }\n" +
-                "            ,\"extractor\" : \"CSV\"\n" +
-                "         }";
-        ExtractorHandler handler = ExtractorHandler.load(config);
-        validate(handler);
-    }
 
-    @Test
-    public void testCSVExtractor() throws Exception {
-        /**
-         {
-            "config" : {
-                        "columns" : "host:0,meta:2"
-                       ,"indicator_column" : "host"
-                       ,"separator" : ","
-                       }
-            ,"extractor" : "CSV"
-         }
-         */
-        String config = "{\n" +
-                "            \"config\" : {\n" +
-                "                        \"columns\" : \"host:0,meta:2\"\n" +
-                "                       ,\"indicator_column\" : \"host\"\n" +
-                "                       ,\"separator\" : \",\"\n" +
-                "                       }\n" +
-                "            ,\"extractor\" : \"CSV\"\n" +
-                "         }";
-        ExtractorHandler handler = ExtractorHandler.load(config);
-        validate(handler);
-    }
+  /**
+   {
+     "config" : {
+        "columns" : {
+            "host" : 0
+           ,"meta" : 2
+                    }
+       ,"indicator_column" : "host"
+       ,"type" : "threat"
+       ,"separator" : ","
+               }
+     ,"extractor" : "CSV"
+   }
+   */
+  @Multiline
+  static String testCSVConfig;
 
-    public void validate(ExtractorHandler handler) throws IOException {
-        {
-            LookupKV results = Iterables.getFirst(handler.getExtractor().extract("google.com,1.0,foo"), null);
-            ThreatIntelKey key = (ThreatIntelKey) results.getKey();
-            ThreatIntelValue value = (ThreatIntelValue) results.getValue();
-            Assert.assertEquals("google.com", key.indicator);
-            Assert.assertEquals("google.com", value.getMetadata().get("host"));
-            Assert.assertEquals("foo", value.getMetadata().get("meta"));
-            Assert.assertEquals(2, value.getMetadata().size());
-        }
-        {
-            Iterable<LookupKV> results = handler.getExtractor().extract("#google.com,1.0,foo");
-            Assert.assertEquals(0, Iterables.size(results));
-        }
+  @Test
+  public void testCSVExtractor() throws Exception {
+
+    ExtractorHandler handler = ExtractorHandler.load(testCSVConfig);
+    validate(handler);
+  }
+
+  public void validate(ExtractorHandler handler) throws IOException {
+    {
+      LookupKV results = Iterables.getFirst(handler.getExtractor().extract("google.com,1.0,foo"), null);
+      EnrichmentKey key = (EnrichmentKey) results.getKey();
+      EnrichmentValue value = (EnrichmentValue) results.getValue();
+      Assert.assertEquals("google.com", key.indicator);
+      Assert.assertEquals("threat", key.type);
+      Assert.assertEquals("google.com", value.getMetadata().get("host"));
+      Assert.assertEquals("foo", value.getMetadata().get("meta"));
+      Assert.assertEquals(2, value.getMetadata().size());
+    }
+    {
+      Iterable<LookupKV> results = handler.getExtractor().extract("#google.com,1.0,foo");
+      Assert.assertEquals(0, Iterables.size(results));
     }
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/9621c55e/metron-streaming/Metron-DataLoads/src/test/java/org/apache/metron/dataloads/extractor/stix/StixExtractorTest.java
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-DataLoads/src/test/java/org/apache/metron/dataloads/extractor/stix/StixExtractorTest.java b/metron-streaming/Metron-DataLoads/src/test/java/org/apache/metron/dataloads/extractor/stix/StixExtractorTest.java
index eaf6677..b348c61 100644
--- a/metron-streaming/Metron-DataLoads/src/test/java/org/apache/metron/dataloads/extractor/stix/StixExtractorTest.java
+++ b/metron-streaming/Metron-DataLoads/src/test/java/org/apache/metron/dataloads/extractor/stix/StixExtractorTest.java
@@ -18,17 +18,16 @@
 package org.apache.metron.dataloads.extractor.stix;
 
 import com.google.common.collect.Iterables;
+import org.adrianwalker.multilinestring.Multiline;
 import org.apache.metron.dataloads.extractor.Extractor;
 import org.apache.metron.dataloads.extractor.ExtractorHandler;
-import org.apache.metron.hbase.converters.threatintel.ThreatIntelKey;
+import org.apache.metron.hbase.converters.enrichment.EnrichmentKey;
 import org.apache.metron.reference.lookup.LookupKV;
 import org.junit.Assert;
 import org.junit.Test;
 
 public class StixExtractorTest {
-    @Test
-    public void testStixAddresses() throws Exception {
-        /**
+    /**
          <!--
          STIX IP Watchlist Example
 
@@ -77,119 +76,64 @@ public class StixExtractorTest {
          </stix:Indicator>
          </stix:Indicators>
          </stix:STIX_Package>
-
-
          */
-        String stixDoc = "<!--\n" +
-                "STIX IP Watchlist Example\n" +
-                "\n" +
-                "Copyright (c) 2015, The MITRE Corporation. All rights reserved.\n" +
-                "The contents of this file are subject to the terms of the STIX License located at http://stix.mitre.org/about/termsofuse.html.\n" +
-                "\n" +
-                "This example demonstrates a simple usage of STIX to represent a list of IP address indicators (watchlist of IP addresses). Cyber operations and malware analysis centers often share a list of suspected malicious IP addresses with information about what those IPs might indicate. This STIX package represents a list of three IP addresses with a short dummy description of what they represent.\n" +
-                "\n" +
-                "It demonstrates the use of:\n" +
-                "\n" +
-                "* STIX Indicators\n" +
-                "* CybOX within STIX\n" +
-                "* The CybOX Address Object (IP)\n" +
-                "* CybOX Patterns (apply_condition=\"ANY\")\n" +
-                "* Controlled vocabularies\n" +
-                "\n" +
-                "Created by Mark Davidson\n" +
-                "-->\n" +
-                "<stix:STIX_Package\n" +
-                "    xmlns:xsi=\"http://www.w3.org/2001/XMLSchema-instance\"\n" +
-                "    xmlns:stix=\"http://stix.mitre.org/stix-1\"\n" +
-                "    xmlns:indicator=\"http://stix.mitre.org/Indicator-2\"\n" +
-                "    xmlns:cybox=\"http://cybox.mitre.org/cybox-2\"\n" +
-                "    xmlns:AddressObject=\"http://cybox.mitre.org/objects#AddressObject-2\"\n" +
-                "    xmlns:cyboxVocabs=\"http://cybox.mitre.org/default_vocabularies-2\"\n" +
-                "    xmlns:stixVocabs=\"http://stix.mitre.org/default_vocabularies-1\"\n" +
-                "    xmlns:example=\"http://example.com/\"\n" +
-                "    id=\"example:STIXPackage-33fe3b22-0201-47cf-85d0-97c02164528d\"\n" +
-                "    timestamp=\"2014-05-08T09:00:00.000000Z\"\n" +
-                "    version=\"1.2\">\n" +
-                "    <stix:STIX_Header>\n" +
-                "        <stix:Title>Example watchlist that contains IP information.</stix:Title>\n" +
-                "        <stix:Package_Intent xsi:type=\"stixVocabs:PackageIntentVocab-1.0\">Indicators - Watchlist</stix:Package_Intent>\n" +
-                "    </stix:STIX_Header>\n" +
-                "    <stix:Indicators>\n" +
-                "        <stix:Indicator xsi:type=\"indicator:IndicatorType\" id=\"example:Indicator-33fe3b22-0201-47cf-85d0-97c02164528d\" timestamp=\"2014-05-08T09:00:00.000000Z\">\n" +
-                "            <indicator:Type xsi:type=\"stixVocabs:IndicatorTypeVocab-1.1\">IP Watchlist</indicator:Type>\n" +
-                "            <indicator:Description>Sample IP Address Indicator for this watchlist. This contains one indicator with a set of three IP addresses in the watchlist.</indicator:Description>\n" +
-                "            <indicator:Observable  id=\"example:Observable-1c798262-a4cd-434d-a958-884d6980c459\">\n" +
-                "                <cybox:Object id=\"example:Object-1980ce43-8e03-490b-863a-ea404d12242e\">\n" +
-                "                    <cybox:Properties xsi:type=\"AddressObject:AddressObjectType\" category=\"ipv4-addr\">\n" +
-                "                        <AddressObject:Address_Value condition=\"Equals\" apply_condition=\"ANY\">10.0.0.0##comma##10.0.0.1##comma##10.0.0.2</AddressObject:Address_Value>\n" +
-                "                    </cybox:Properties>\n" +
-                "                </cybox:Object>\n" +
-                "            </indicator:Observable>\n" +
-                "        </stix:Indicator>\n" +
-                "    </stix:Indicators>\n" +
-                "</stix:STIX_Package>\n" +
-                "\n";
-        {
-            /**
-             {
-             "config" : {
+    @Multiline
+    private static String stixDoc;
+
+    /**
+    {
+        "config" : {
              "stix_address_categories" : "IPV_4_ADDR"
-             }
-             ,"extractor" : "STIX"
-             }
-             */
-            String config = "{\n" +
-                    "            \"config\" : {\n" +
-                    "                       \"stix_address_categories\" : \"IPV_4_ADDR\"\n" +
-                    "                       }\n" +
-                    "            ,\"extractor\" : \"STIX\"\n" +
-                    "         }";
-            ExtractorHandler handler = ExtractorHandler.load(config);
+        }
+        ,"extractor" : "STIX"
+    }
+    */
+    @Multiline
+    private static String stixConfigOnlyIPV4;
+    /**
+    {
+        "config" : {
+             "stix_address_categories" : "IPV_6_ADDR"
+        }
+        ,"extractor" : "STIX"
+    }
+    */
+    @Multiline
+    private static String stixConfigOnlyIPV6;
+    /**
+    {
+        "config" : {
+        }
+        ,"extractor" : "STIX"
+    }
+    */
+    @Multiline
+    private static String stixConfig;
+    @Test
+    public void testStixAddresses() throws Exception {
+        {
+            ExtractorHandler handler = ExtractorHandler.load(stixConfigOnlyIPV4);
             Extractor extractor = handler.getExtractor();
             Iterable<LookupKV> results = extractor.extract(stixDoc);
 
             Assert.assertEquals(3, Iterables.size(results));
-            Assert.assertEquals("10.0.0.0", ((ThreatIntelKey)(Iterables.get(results, 0).getKey())).indicator);
-            Assert.assertEquals("10.0.0.1", ((ThreatIntelKey)(Iterables.get(results, 1).getKey())).indicator);
-            Assert.assertEquals("10.0.0.2", ((ThreatIntelKey)(Iterables.get(results, 2).getKey())).indicator);
+            Assert.assertEquals("10.0.0.0", ((EnrichmentKey)(Iterables.get(results, 0).getKey())).indicator);
+            Assert.assertEquals("10.0.0.1", ((EnrichmentKey)(Iterables.get(results, 1).getKey())).indicator);
+            Assert.assertEquals("10.0.0.2", ((EnrichmentKey)(Iterables.get(results, 2).getKey())).indicator);
         }
         {
-            /**
-             {
-             "config" : {
-             }
-             ,"extractor" : "STIX"
-             }
-             */
-            String config = "{\n" +
-                    "            \"config\" : {\n" +
-                    "                       }\n" +
-                    "            ,\"extractor\" : \"STIX\"\n" +
-                    "         }";
-            ExtractorHandler handler = ExtractorHandler.load(config);
+
+            ExtractorHandler handler = ExtractorHandler.load(stixConfig);
             Extractor extractor = handler.getExtractor();
             Iterable<LookupKV> results = extractor.extract(stixDoc);
             Assert.assertEquals(3, Iterables.size(results));
-            Assert.assertEquals("10.0.0.0", ((ThreatIntelKey)(Iterables.get(results, 0).getKey())).indicator);
-            Assert.assertEquals("10.0.0.1", ((ThreatIntelKey)(Iterables.get(results, 1).getKey())).indicator);
-            Assert.assertEquals("10.0.0.2", ((ThreatIntelKey)(Iterables.get(results, 2).getKey())).indicator);
+            Assert.assertEquals("10.0.0.0", ((EnrichmentKey)(Iterables.get(results, 0).getKey())).indicator);
+            Assert.assertEquals("10.0.0.1", ((EnrichmentKey)(Iterables.get(results, 1).getKey())).indicator);
+            Assert.assertEquals("10.0.0.2", ((EnrichmentKey)(Iterables.get(results, 2).getKey())).indicator);
         }
         {
-            /**
-             {
-             "config" : {
-                "stix_address_categories" : "IPV_6_ADDR"
-             }
-             ,"extractor" : "STIX"
-             }
-             */
-            String config = "{\n" +
-                    "            \"config\" : {\n" +
-                    "                       \"stix_address_categories\" : \"IPV_6_ADDR\"\n" +
-                    "                       }\n" +
-                    "            ,\"extractor\" : \"STIX\"\n" +
-                    "         }";
-            ExtractorHandler handler = ExtractorHandler.load(config);
+
+            ExtractorHandler handler = ExtractorHandler.load(stixConfigOnlyIPV6);
             Extractor extractor = handler.getExtractor();
             Iterable<LookupKV> results = extractor.extract(stixDoc);
             Assert.assertEquals(0, Iterables.size(results));

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/9621c55e/metron-streaming/Metron-DataLoads/src/test/java/org/apache/metron/dataloads/hbase/HBaseEnrichmentConverterTest.java
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-DataLoads/src/test/java/org/apache/metron/dataloads/hbase/HBaseEnrichmentConverterTest.java b/metron-streaming/Metron-DataLoads/src/test/java/org/apache/metron/dataloads/hbase/HBaseEnrichmentConverterTest.java
new file mode 100644
index 0000000..ec92350
--- /dev/null
+++ b/metron-streaming/Metron-DataLoads/src/test/java/org/apache/metron/dataloads/hbase/HBaseEnrichmentConverterTest.java
@@ -0,0 +1,74 @@
+/**
+ * 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.dataloads.hbase;
+
+import org.apache.hadoop.hbase.client.Get;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.metron.hbase.converters.HbaseConverter;
+import org.apache.metron.hbase.converters.enrichment.EnrichmentConverter;
+import org.apache.metron.hbase.converters.enrichment.EnrichmentKey;
+import org.apache.metron.hbase.converters.enrichment.EnrichmentValue;
+import org.apache.metron.reference.lookup.LookupKV;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.util.HashMap;
+
+
+public class HBaseEnrichmentConverterTest {
+    EnrichmentKey key = new EnrichmentKey("domain", "google");
+    EnrichmentValue value = new EnrichmentValue(
+            new HashMap<String, String>() {{
+                put("foo", "bar");
+                put("grok", "baz");
+            }});
+    LookupKV<EnrichmentKey, EnrichmentValue> results = new LookupKV(key, value);
+    @Test
+    public void testKeySerialization() {
+        byte[] serialized = key.toBytes();
+
+        EnrichmentKey deserialized = new EnrichmentKey();
+        deserialized.fromBytes(serialized);
+        Assert.assertEquals(key, deserialized);
+    }
+
+    @Test
+    public void testPut() throws IOException {
+        HbaseConverter<EnrichmentKey, EnrichmentValue> converter = new EnrichmentConverter();
+        Put put = converter.toPut("cf", key, value);
+        LookupKV<EnrichmentKey, EnrichmentValue> converted= converter.fromPut(put, "cf");
+        Assert.assertEquals(results, converted);
+    }
+    @Test
+    public void testResult() throws IOException {
+        HbaseConverter<EnrichmentKey, EnrichmentValue> converter = new EnrichmentConverter();
+        Result r = converter.toResult("cf", key, value);
+        LookupKV<EnrichmentKey, EnrichmentValue> converted= converter.fromResult(r, "cf");
+        Assert.assertEquals(results, converted);
+    }
+
+    @Test
+    public void testGet() throws Exception {
+        HbaseConverter<EnrichmentKey, EnrichmentValue> converter = new EnrichmentConverter();
+        Get get = converter.toGet("cf", key);
+        Assert.assertArrayEquals(key.toBytes(), get.getRow());
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/9621c55e/metron-streaming/Metron-DataLoads/src/test/java/org/apache/metron/dataloads/hbase/HBaseThreatIntelConverterTest.java
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-DataLoads/src/test/java/org/apache/metron/dataloads/hbase/HBaseThreatIntelConverterTest.java b/metron-streaming/Metron-DataLoads/src/test/java/org/apache/metron/dataloads/hbase/HBaseThreatIntelConverterTest.java
deleted file mode 100644
index 21a1d26..0000000
--- a/metron-streaming/Metron-DataLoads/src/test/java/org/apache/metron/dataloads/hbase/HBaseThreatIntelConverterTest.java
+++ /dev/null
@@ -1,76 +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.dataloads.hbase;
-
-import org.apache.hadoop.hbase.client.Get;
-import org.apache.hadoop.hbase.client.Put;
-import org.apache.hadoop.hbase.client.Result;
-import org.apache.metron.hbase.converters.HbaseConverter;
-import org.apache.metron.hbase.converters.threatintel.ThreatIntelConverter;
-import org.apache.metron.hbase.converters.threatintel.ThreatIntelKey;
-import org.apache.metron.hbase.converters.threatintel.ThreatIntelValue;
-import org.apache.metron.reference.lookup.LookupKV;
-import org.junit.Assert;
-import org.junit.Test;
-
-import java.io.IOException;
-import java.util.HashMap;
-
-/**
- * Created by cstella on 2/3/16.
- */
-public class HBaseThreatIntelConverterTest {
-    ThreatIntelKey key = new ThreatIntelKey("google");
-    ThreatIntelValue value = new ThreatIntelValue(
-    new HashMap<String, String>() {{
-        put("foo", "bar");
-        put("grok", "baz");
-    }});
-    LookupKV<ThreatIntelKey, ThreatIntelValue> results = new LookupKV(key, value);
-    @Test
-    public void testKeySerialization() {
-        byte[] serialized = key.toBytes();
-
-        ThreatIntelKey deserialized = new ThreatIntelKey();
-        deserialized.fromBytes(serialized);
-        Assert.assertEquals(key, deserialized);
-    }
-
-    @Test
-    public void testPut() throws IOException {
-        HbaseConverter<ThreatIntelKey, ThreatIntelValue> converter = new ThreatIntelConverter();
-        Put put = converter.toPut("cf", key, value);
-        LookupKV<ThreatIntelKey, ThreatIntelValue> converted= converter.fromPut(put, "cf");
-        Assert.assertEquals(results, converted);
-    }
-    @Test
-    public void testResult() throws IOException {
-        HbaseConverter<ThreatIntelKey, ThreatIntelValue> converter = new ThreatIntelConverter();
-        Result r = converter.toResult("cf", key, value);
-        LookupKV<ThreatIntelKey, ThreatIntelValue> converted= converter.fromResult(r, "cf");
-        Assert.assertEquals(results, converted);
-    }
-
-    @Test
-    public void testGet() throws Exception {
-        HbaseConverter<ThreatIntelKey, ThreatIntelValue> converter = new ThreatIntelConverter();
-        Get get = converter.toGet("cf", key);
-        Assert.assertArrayEquals(key.toBytes(), get.getRow());
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/9621c55e/metron-streaming/Metron-DataLoads/src/test/java/org/apache/metron/dataloads/hbase/mr/BulkLoadMapperIntegrationTest.java
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-DataLoads/src/test/java/org/apache/metron/dataloads/hbase/mr/BulkLoadMapperIntegrationTest.java b/metron-streaming/Metron-DataLoads/src/test/java/org/apache/metron/dataloads/hbase/mr/BulkLoadMapperIntegrationTest.java
index be9ac90..c2f0882 100644
--- a/metron-streaming/Metron-DataLoads/src/test/java/org/apache/metron/dataloads/hbase/mr/BulkLoadMapperIntegrationTest.java
+++ b/metron-streaming/Metron-DataLoads/src/test/java/org/apache/metron/dataloads/hbase/mr/BulkLoadMapperIntegrationTest.java
@@ -17,6 +17,7 @@
  */
 package org.apache.metron.dataloads.hbase.mr;
 
+import org.adrianwalker.multilinestring.Multiline;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -27,10 +28,10 @@ import org.apache.hadoop.hbase.client.ResultScanner;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.mapreduce.Job;
 import org.apache.metron.dataloads.bulk.ThreatIntelBulkLoader;
-import org.apache.metron.hbase.converters.threatintel.ThreatIntelKey;
-import org.apache.metron.hbase.converters.threatintel.ThreatIntelValue;
+import org.apache.metron.hbase.converters.enrichment.EnrichmentConverter;
+import org.apache.metron.hbase.converters.enrichment.EnrichmentKey;
+import org.apache.metron.hbase.converters.enrichment.EnrichmentValue;
 import org.apache.metron.reference.lookup.LookupKV;
-import org.apache.metron.hbase.converters.threatintel.ThreatIntelConverter;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
@@ -62,9 +63,6 @@ public class BulkLoadMapperIntegrationTest {
   public void teardown() throws Exception {
     HBaseUtil.INSTANCE.teardown(testUtil);
   }
-
-  @Test
-  public void test() throws IOException, ClassNotFoundException, InterruptedException {
  /**
          {
             "config" : {
@@ -74,32 +72,32 @@ public class BulkLoadMapperIntegrationTest {
                                     }
                        ,"indicator_column" : "host"
                        ,"separator" : ","
+                       ,"type" : "threat"
                        }
             ,"extractor" : "CSV"
          }
          */
-        final String extractorConfig = "{\n" +
-                "            \"config\" : {\n" +
-                "                        \"columns\" : [\"host:0\",\"meta:2\"]\n" +
-                "                       ,\"indicator_column\" : \"host\"\n" +
-                "                       ,\"separator\" : \",\"\n" +
-                "                       }\n" +
-                "            ,\"extractor\" : \"CSV\"\n" +
-                "         }";
+  @Multiline
+  private static String extractorConfig;
+
+  @Test
+  public void test() throws IOException, ClassNotFoundException, InterruptedException {
+
     Assert.assertNotNull(testTable);
     FileSystem fs = FileSystem.get(config);
     String contents = "google.com,1,foo";
-    ThreatIntelConverter converter = new ThreatIntelConverter();
+    EnrichmentConverter converter = new EnrichmentConverter();
     HBaseUtil.INSTANCE.writeFile(contents, new Path("input.csv"), fs);
-    Job job = ThreatIntelBulkLoader.createJob(config, "input.csv", tableName, cf, extractorConfig, 0L, new ThreatIntelConverter());
+    Job job = ThreatIntelBulkLoader.createJob(config, "input.csv", tableName, cf, extractorConfig, 0L, new EnrichmentConverter());
     Assert.assertTrue(job.waitForCompletion(true));
     ResultScanner scanner = testTable.getScanner(Bytes.toBytes(cf));
-    List<LookupKV<ThreatIntelKey, ThreatIntelValue>> results = new ArrayList<>();
+    List<LookupKV<EnrichmentKey, EnrichmentValue>> results = new ArrayList<>();
     for(Result r : scanner) {
       results.add(converter.fromResult(r, cf));
     }
     Assert.assertEquals(1, results.size());
     Assert.assertEquals(results.get(0).getKey().indicator, "google.com");
+    Assert.assertEquals(results.get(0).getKey().type, "threat");
     Assert.assertEquals(results.get(0).getValue().getMetadata().size(), 2);
     Assert.assertEquals(results.get(0).getValue().getMetadata().get("meta"), "foo");
     Assert.assertEquals(results.get(0).getValue().getMetadata().get("host"), "google.com");

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/9621c55e/metron-streaming/Metron-DataLoads/src/test/java/org/apache/metron/dataloads/hbase/mr/BulkLoadMapperTest.java
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-DataLoads/src/test/java/org/apache/metron/dataloads/hbase/mr/BulkLoadMapperTest.java b/metron-streaming/Metron-DataLoads/src/test/java/org/apache/metron/dataloads/hbase/mr/BulkLoadMapperTest.java
index 8ace7bc..2134b57 100644
--- a/metron-streaming/Metron-DataLoads/src/test/java/org/apache/metron/dataloads/hbase/mr/BulkLoadMapperTest.java
+++ b/metron-streaming/Metron-DataLoads/src/test/java/org/apache/metron/dataloads/hbase/mr/BulkLoadMapperTest.java
@@ -17,15 +17,15 @@
  */
 package org.apache.metron.dataloads.hbase.mr;
 
+import org.adrianwalker.multilinestring.Multiline;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.hadoop.io.Text;
-import org.apache.metron.hbase.converters.threatintel.ThreatIntelValue;
+import org.apache.metron.hbase.converters.enrichment.EnrichmentConverter;
+import org.apache.metron.hbase.converters.enrichment.EnrichmentKey;
+import org.apache.metron.hbase.converters.enrichment.EnrichmentValue;
 import org.apache.metron.reference.lookup.LookupKV;
-import org.apache.metron.threatintel.ThreatIntelResults;
-import org.apache.metron.hbase.converters.threatintel.ThreatIntelConverter;
-import org.apache.metron.hbase.converters.threatintel.ThreatIntelKey;
 import org.junit.Assert;
 import org.junit.Test;
 
@@ -34,9 +34,7 @@ import java.util.HashMap;
 import java.util.Map;
 
 public class BulkLoadMapperTest {
-    @Test
-    public void testMapper() throws IOException, InterruptedException {
-        /**
+    /**
          {
             "config" : {
                         "columns" : {
@@ -44,19 +42,16 @@ public class BulkLoadMapperTest {
                                 ,"meta" : 2
                                     }
                        ,"indicator_column" : "host"
+                       ,"type" : "threat"
                        ,"separator" : ","
                        }
             ,"extractor" : "CSV"
          }
          */
-        final String extractorConfig = "{\n" +
-                "            \"config\" : {\n" +
-                "                        \"columns\" : [\"host:0\",\"meta:2\"]\n" +
-                "                       ,\"indicator_column\" : \"host\"\n" +
-                "                       ,\"separator\" : \",\"\n" +
-                "                       }\n" +
-                "            ,\"extractor\" : \"CSV\"\n" +
-                "         }";
+    @Multiline
+    private static String extractorConfig;
+    @Test
+    public void testMapper() throws IOException, InterruptedException {
 
         final Map<ImmutableBytesWritable, Put> puts = new HashMap<>();
         BulkLoadMapper mapper = new BulkLoadMapper() {
@@ -69,7 +64,7 @@ public class BulkLoadMapperTest {
             set(BulkLoadMapper.COLUMN_FAMILY_KEY, "cf");
             set(BulkLoadMapper.CONFIG_KEY, extractorConfig);
             set(BulkLoadMapper.LAST_SEEN_KEY, "0");
-            set(BulkLoadMapper.CONVERTER_KEY, ThreatIntelConverter.class.getName());
+            set(BulkLoadMapper.CONVERTER_KEY, EnrichmentConverter.class.getName());
         }});
         {
             mapper.map(null, new Text("#google.com,1,foo"), null);
@@ -78,13 +73,14 @@ public class BulkLoadMapperTest {
         {
             mapper.map(null, new Text("google.com,1,foo"), null);
             Assert.assertTrue(puts.size() == 1);
-            ThreatIntelKey expectedKey = new ThreatIntelKey() {{
+            EnrichmentKey expectedKey = new EnrichmentKey() {{
                 indicator = "google.com";
+                type = "threat";
             }};
-            ThreatIntelConverter converter = new ThreatIntelConverter();
+            EnrichmentConverter converter = new EnrichmentConverter();
             Put put = puts.get(new ImmutableBytesWritable(expectedKey.toBytes()));
             Assert.assertNotNull(puts);
-            LookupKV<ThreatIntelKey, ThreatIntelValue> results = converter.fromPut(put, "cf");
+            LookupKV<EnrichmentKey, EnrichmentValue> results = converter.fromPut(put, "cf");
             Assert.assertEquals(results.getKey().indicator, "google.com");
             Assert.assertEquals(results.getValue().getMetadata().size(), 2);
             Assert.assertEquals(results.getValue().getMetadata().get("meta"), "foo");

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/9621c55e/metron-streaming/Metron-DataLoads/src/test/java/org/apache/metron/dataloads/hbase/mr/LeastRecentlyUsedPrunerIntegrationTest.java
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-DataLoads/src/test/java/org/apache/metron/dataloads/hbase/mr/LeastRecentlyUsedPrunerIntegrationTest.java b/metron-streaming/Metron-DataLoads/src/test/java/org/apache/metron/dataloads/hbase/mr/LeastRecentlyUsedPrunerIntegrationTest.java
index a7d28fb..e58354d 100644
--- a/metron-streaming/Metron-DataLoads/src/test/java/org/apache/metron/dataloads/hbase/mr/LeastRecentlyUsedPrunerIntegrationTest.java
+++ b/metron-streaming/Metron-DataLoads/src/test/java/org/apache/metron/dataloads/hbase/mr/LeastRecentlyUsedPrunerIntegrationTest.java
@@ -24,13 +24,13 @@ import org.apache.hadoop.hbase.client.HTable;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.mapreduce.Job;
 import org.apache.metron.dataloads.bulk.LeastRecentlyUsedPruner;
-import org.apache.metron.hbase.converters.threatintel.ThreatIntelValue;
+import org.apache.metron.hbase.converters.enrichment.EnrichmentConverter;
+import org.apache.metron.hbase.converters.enrichment.EnrichmentKey;
+import org.apache.metron.hbase.converters.enrichment.EnrichmentValue;
+import org.apache.metron.hbase.lookup.EnrichmentLookup;
 import org.apache.metron.reference.lookup.LookupKey;
 import org.apache.metron.reference.lookup.accesstracker.BloomAccessTracker;
 import org.apache.metron.reference.lookup.accesstracker.PersistentAccessTracker;
-import org.apache.metron.hbase.converters.threatintel.ThreatIntelKey;
-import org.apache.metron.hbase.converters.threatintel.ThreatIntelConverter;
-import org.apache.metron.threatintel.hbase.ThreatIntelLookup;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
@@ -68,7 +68,7 @@ public class LeastRecentlyUsedPrunerIntegrationTest {
     public List<LookupKey> getKeys(int start, int end) {
         List<LookupKey> keys = new ArrayList<>();
         for(int i = start;i < end;++i) {
-            keys.add(new ThreatIntelKey("key-" + i));
+            keys.add(new EnrichmentKey("type", "key-" + i));
         }
         return keys;
     }
@@ -77,33 +77,33 @@ public class LeastRecentlyUsedPrunerIntegrationTest {
         long ts = System.currentTimeMillis();
         BloomAccessTracker bat = new BloomAccessTracker("tracker1", 100, 0.03);
         PersistentAccessTracker pat = new PersistentAccessTracker(tableName, "0", atTable, atCF, bat, 0L);
-        ThreatIntelLookup lookup = new ThreatIntelLookup(testTable, cf, pat);
+        EnrichmentLookup lookup = new EnrichmentLookup(testTable, cf, pat);
         List<LookupKey> goodKeysHalf = getKeys(0, 5);
         List<LookupKey> goodKeysOtherHalf = getKeys(5, 10);
         Iterable<LookupKey> goodKeys = Iterables.concat(goodKeysHalf, goodKeysOtherHalf);
         List<LookupKey> badKey = getKeys(10, 11);
-        ThreatIntelConverter converter = new ThreatIntelConverter();
+        EnrichmentConverter converter = new EnrichmentConverter();
         for(LookupKey k : goodKeysHalf) {
-            testTable.put(converter.toPut(cf, (ThreatIntelKey) k
-                                            , new ThreatIntelValue(
+            testTable.put(converter.toPut(cf, (EnrichmentKey) k
+                                            , new EnrichmentValue(
                                                   new HashMap<String, String>() {{
                                                     put("k", "dummy");
                                                     }}
                                                   )
                                           )
                          );
-            Assert.assertTrue(lookup.exists((ThreatIntelKey)k, testTable, true));
+            Assert.assertTrue(lookup.exists((EnrichmentKey)k, testTable, true));
         }
         pat.persist(true);
         for(LookupKey k : goodKeysOtherHalf) {
-            testTable.put(converter.toPut(cf, (ThreatIntelKey) k
-                                            , new ThreatIntelValue(new HashMap<String, String>() {{
+            testTable.put(converter.toPut(cf, (EnrichmentKey) k
+                                            , new EnrichmentValue(new HashMap<String, String>() {{
                                                     put("k", "dummy");
                                                     }}
                                                                   )
                                          )
                          );
-            Assert.assertTrue(lookup.exists((ThreatIntelKey)k, testTable, true));
+            Assert.assertTrue(lookup.exists((EnrichmentKey)k, testTable, true));
         }
         testUtil.flush();
         Assert.assertFalse(lookup.getAccessTracker().hasSeen(goodKeysHalf.get(0)));
@@ -112,8 +112,8 @@ public class LeastRecentlyUsedPrunerIntegrationTest {
         }
         pat.persist(true);
         {
-            testTable.put(converter.toPut(cf, (ThreatIntelKey) badKey.get(0)
-                    , new ThreatIntelValue(new HashMap<String, String>() {{
+            testTable.put(converter.toPut(cf, (EnrichmentKey) badKey.get(0)
+                    , new EnrichmentValue(new HashMap<String, String>() {{
                         put("k", "dummy");
                     }}
                     )
@@ -127,10 +127,10 @@ public class LeastRecentlyUsedPrunerIntegrationTest {
         Job job = LeastRecentlyUsedPruner.createJob(config, tableName, cf, atTableName, atCF, ts);
         Assert.assertTrue(job.waitForCompletion(true));
         for(LookupKey k : goodKeys) {
-            Assert.assertTrue(lookup.exists((ThreatIntelKey)k, testTable, true));
+            Assert.assertTrue(lookup.exists((EnrichmentKey)k, testTable, true));
         }
         for(LookupKey k : badKey) {
-            Assert.assertFalse(lookup.exists((ThreatIntelKey)k, testTable, true));
+            Assert.assertFalse(lookup.exists((EnrichmentKey)k, testTable, true));
         }
 
     }

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/9621c55e/metron-streaming/Metron-DataLoads/src/test/java/org/apache/metron/dataloads/nonbulk/taxii/MockTaxiiService.java
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-DataLoads/src/test/java/org/apache/metron/dataloads/nonbulk/taxii/MockTaxiiService.java b/metron-streaming/Metron-DataLoads/src/test/java/org/apache/metron/dataloads/nonbulk/taxii/MockTaxiiService.java
new file mode 100644
index 0000000..82ad7f0
--- /dev/null
+++ b/metron-streaming/Metron-DataLoads/src/test/java/org/apache/metron/dataloads/nonbulk/taxii/MockTaxiiService.java
@@ -0,0 +1,94 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.metron.dataloads.nonbulk.taxii;
+
+
+import com.sun.net.httpserver.HttpHandler;
+import com.sun.net.httpserver.HttpServer;
+import org.apache.commons.io.FileUtils;
+import org.apache.metron.integration.util.UnitTestHelper;
+
+import javax.ws.rs.ApplicationPath;
+import javax.ws.rs.POST;
+import javax.ws.rs.Path;
+import javax.ws.rs.core.*;
+import javax.ws.rs.ext.RuntimeDelegate;
+import java.io.File;
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.net.URI;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Set;
+
+@Path("/")
+public class MockTaxiiService {
+    static String discoveryMsg;
+    static String pollMsg;
+    static {
+        try {
+            String baseDir = UnitTestHelper.findDir("taxii-messages");
+            discoveryMsg = FileUtils.readFileToString(new File(new File(baseDir), "message.discovery"));
+            pollMsg= FileUtils.readFileToString(new File(new File(baseDir), "messages.poll"));
+        } catch (IOException e) {
+            throw new RuntimeException("Unable to read discovery message", e);
+        }
+    }
+
+
+    @POST
+    @Path("/taxii-discovery-service")
+    public Response getDiscovery() {
+        return Response.ok(discoveryMsg, MediaType.APPLICATION_XML_TYPE).header("x-taxii-content-type", "urn:taxii.mitre.org:message:xml:1.1").build();
+    }
+    @POST
+    @Path("/taxii-data")
+    public Response getData() {
+        return Response.ok(pollMsg).type(MediaType.APPLICATION_XML_TYPE).header("x-taxii-content-type", "urn:taxii.mitre.org:message:xml:1.1").build();
+    }
+
+    @ApplicationPath("rs")
+    public static class ApplicationConfig extends Application{
+        private final Set<Class<?>> classes;
+        public ApplicationConfig() {
+            HashSet<Class<?>> c = new HashSet<>();
+            c.add(MockTaxiiService.class);
+            classes = Collections.unmodifiableSet(c);
+        }
+        @Override
+        public Set<Class<?>> getClasses() {
+            return classes;
+        }
+    }
+    private static HttpServer server;
+    public static void start(int port) throws IOException {
+        // Create an HTTP server listening at port 8282
+        URI uri = UriBuilder.fromUri("http://localhost/").port(port).build();
+        server = HttpServer.create(new InetSocketAddress(uri.getPort()), 0);
+        HttpHandler handler = RuntimeDelegate.getInstance().createEndpoint(new ApplicationConfig(), HttpHandler.class);
+        server.createContext(uri.getPath(), handler);
+        discoveryMsg = discoveryMsg.replaceAll("PORT", "" + uri.getPort());
+        server.start();
+    }
+
+    public static void shutdown() {
+        if(server != null) {
+            server.stop(0);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/9621c55e/metron-streaming/Metron-DataLoads/src/test/java/org/apache/metron/dataloads/nonbulk/taxii/TaxiiIntegrationTest.java
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-DataLoads/src/test/java/org/apache/metron/dataloads/nonbulk/taxii/TaxiiIntegrationTest.java b/metron-streaming/Metron-DataLoads/src/test/java/org/apache/metron/dataloads/nonbulk/taxii/TaxiiIntegrationTest.java
new file mode 100644
index 0000000..fbdb987
--- /dev/null
+++ b/metron-streaming/Metron-DataLoads/src/test/java/org/apache/metron/dataloads/nonbulk/taxii/TaxiiIntegrationTest.java
@@ -0,0 +1,119 @@
+/**
+ * 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.dataloads.nonbulk.taxii;
+
+import com.google.common.base.Splitter;
+import org.adrianwalker.multilinestring.Multiline;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.client.HTableInterface;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.metron.dataloads.extractor.stix.StixExtractor;
+import org.apache.metron.hbase.converters.enrichment.EnrichmentConverter;
+import org.apache.metron.hbase.converters.enrichment.EnrichmentKey;
+import org.apache.metron.hbase.converters.enrichment.EnrichmentValue;
+import org.apache.metron.integration.util.mock.MockHTable;
+import org.apache.metron.reference.lookup.LookupKV;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.util.HashSet;
+import java.util.Set;
+
+public class TaxiiIntegrationTest {
+
+    @Before
+    public void setup() throws IOException {
+        MockTaxiiService.start(8282);
+    }
+
+    @After
+    public void teardown() {
+        MockTaxiiService.shutdown();
+        MockHTable.Provider.clear();
+    }
+
+    /**
+         {
+            "endpoint" : "http://localhost:8282/taxii-discovery-service"
+           ,"type" : "DISCOVER"
+           ,"collection" : "guest.Abuse_ch"
+           ,"table" : "threat_intel"
+           ,"columnFamily" : "cf"
+           ,"allowedIndicatorTypes" : [ "domainname:FQDN", "address:IPV_4_ADDR" ]
+         }
+    */
+    @Multiline
+    static String taxiiConnectionConfig;
+
+    @Test
+    public void testTaxii() throws Exception {
+
+        final MockHTable.Provider provider = new MockHTable.Provider();
+        final Configuration config = HBaseConfiguration.create();
+        TaxiiHandler handler = new TaxiiHandler(TaxiiConnectionConfig.load(taxiiConnectionConfig), new StixExtractor(), config ) {
+            @Override
+            protected synchronized HTableInterface createHTable(String tableInfo) throws IOException {
+                return provider.addToCache("threat_intel", "cf");
+            }
+        };
+        //UnitTestHelper.verboseLogging();
+        handler.run();
+        Set<String> maliciousDomains;
+        {
+            MockHTable table = (MockHTable) provider.getTable(config, "threat_intel");
+            maliciousDomains = getIndicators("domainname:FQDN", table.getPutLog(), "cf");
+        }
+        Assert.assertTrue(maliciousDomains.contains("www.office-112.com"));
+        Assert.assertEquals(numStringsMatch(MockTaxiiService.pollMsg, "DomainNameObj:Value condition=\"Equals\""), maliciousDomains.size());
+        Set<String> maliciousAddresses;
+        {
+            MockHTable table = (MockHTable) provider.getTable(config, "threat_intel");
+            maliciousAddresses= getIndicators("address:IPV_4_ADDR", table.getPutLog(), "cf");
+        }
+        Assert.assertTrue(maliciousAddresses.contains("94.102.53.142"));
+        Assert.assertEquals(numStringsMatch(MockTaxiiService.pollMsg, "AddressObj:Address_Value condition=\"Equal\""), maliciousAddresses.size());
+        MockHTable.Provider.clear();
+    }
+
+    private static int numStringsMatch(String xmlBundle, String text) {
+        int cnt = 0;
+        for(String line : Splitter.on("\n").split(xmlBundle)) {
+            if(line.contains(text)) {
+                cnt++;
+            }
+        }
+        return cnt;
+    }
+
+    private static Set<String> getIndicators(String indicatorType, Iterable<Put> puts, String cf) throws IOException {
+        EnrichmentConverter converter = new EnrichmentConverter();
+        Set<String> ret = new HashSet<>();
+        for(Put p : puts) {
+            LookupKV<EnrichmentKey, EnrichmentValue> kv = converter.fromPut(p, cf);
+            if (kv.getKey().type.equals(indicatorType)) {
+                ret.add(kv.getKey().indicator);
+            }
+        }
+        return ret;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/9621c55e/metron-streaming/Metron-DataLoads/src/test/java/org/apache/metron/dataloads/taxii/MockTaxiiService.java
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-DataLoads/src/test/java/org/apache/metron/dataloads/taxii/MockTaxiiService.java b/metron-streaming/Metron-DataLoads/src/test/java/org/apache/metron/dataloads/taxii/MockTaxiiService.java
deleted file mode 100644
index 64e4618..0000000
--- a/metron-streaming/Metron-DataLoads/src/test/java/org/apache/metron/dataloads/taxii/MockTaxiiService.java
+++ /dev/null
@@ -1,99 +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.dataloads.taxii;
-
-
-import com.sun.net.httpserver.HttpHandler;
-import com.sun.net.httpserver.HttpServer;
-import org.apache.commons.io.FileUtils;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.metron.integration.util.UnitTestHelper;
-
-import javax.servlet.ServletContext;
-import javax.servlet.http.HttpServletResponse;
-import javax.ws.rs.ApplicationPath;
-import javax.ws.rs.POST;
-import javax.ws.rs.Path;
-import javax.ws.rs.WebApplicationException;
-import javax.ws.rs.core.*;
-import javax.ws.rs.ext.RuntimeDelegate;
-import java.io.File;
-import java.io.IOException;
-import java.io.OutputStream;
-import java.net.InetSocketAddress;
-import java.net.URI;
-import java.util.Collections;
-import java.util.HashSet;
-import java.util.Set;
-
-@Path("/")
-public class MockTaxiiService {
-    static String discoveryMsg;
-    static String pollMsg;
-    static {
-        try {
-            String baseDir = UnitTestHelper.findDir("taxii-messages");
-            discoveryMsg = FileUtils.readFileToString(new File(new File(baseDir), "message.discovery"));
-            pollMsg= FileUtils.readFileToString(new File(new File(baseDir), "messages.poll"));
-        } catch (IOException e) {
-            throw new RuntimeException("Unable to read discovery message", e);
-        }
-    }
-
-
-    @POST
-    @Path("/taxii-discovery-service")
-    public Response getDiscovery() {
-        return Response.ok(discoveryMsg, MediaType.APPLICATION_XML_TYPE).header("x-taxii-content-type", "urn:taxii.mitre.org:message:xml:1.1").build();
-    }
-    @POST
-    @Path("/taxii-data")
-    public Response getData() {
-        return Response.ok(pollMsg).type(MediaType.APPLICATION_XML_TYPE).header("x-taxii-content-type", "urn:taxii.mitre.org:message:xml:1.1").build();
-    }
-
-    @ApplicationPath("rs")
-    public static class ApplicationConfig extends Application{
-        private final Set<Class<?>> classes;
-        public ApplicationConfig() {
-            HashSet<Class<?>> c = new HashSet<>();
-            c.add(MockTaxiiService.class);
-            classes = Collections.unmodifiableSet(c);
-        }
-        @Override
-        public Set<Class<?>> getClasses() {
-            return classes;
-        }
-    }
-    private static HttpServer server;
-    public static void start(int port) throws IOException {
-        // Create an HTTP server listening at port 8282
-        URI uri = UriBuilder.fromUri("http://localhost/").port(port).build();
-        server = HttpServer.create(new InetSocketAddress(uri.getPort()), 0);
-        HttpHandler handler = RuntimeDelegate.getInstance().createEndpoint(new ApplicationConfig(), HttpHandler.class);
-        server.createContext(uri.getPath(), handler);
-        discoveryMsg = discoveryMsg.replaceAll("PORT", "" + uri.getPort());
-        server.start();
-    }
-
-    public static void shutdown() {
-        if(server != null) {
-            server.stop(0);
-        }
-    }
-}


[4/6] incubator-metron git commit: METRON-93: Generalize the HBase threat intel infrastructure to support enrichments closes apache/incubator-metron#64

Posted by ce...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/9621c55e/metron-streaming/Metron-DataLoads/src/main/java/org/apache/metron/dataloads/extractor/stix/types/DomainHandler.java
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-DataLoads/src/main/java/org/apache/metron/dataloads/extractor/stix/types/DomainHandler.java b/metron-streaming/Metron-DataLoads/src/main/java/org/apache/metron/dataloads/extractor/stix/types/DomainHandler.java
index 233550b..ae839d2 100644
--- a/metron-streaming/Metron-DataLoads/src/main/java/org/apache/metron/dataloads/extractor/stix/types/DomainHandler.java
+++ b/metron-streaming/Metron-DataLoads/src/main/java/org/apache/metron/dataloads/extractor/stix/types/DomainHandler.java
@@ -18,10 +18,9 @@
 package org.apache.metron.dataloads.extractor.stix.types;
 
 import org.apache.metron.dataloads.extractor.stix.StixExtractor;
-import org.apache.metron.hbase.converters.threatintel.ThreatIntelKey;
-import org.apache.metron.hbase.converters.threatintel.ThreatIntelValue;
+import org.apache.metron.hbase.converters.enrichment.EnrichmentKey;
+import org.apache.metron.hbase.converters.enrichment.EnrichmentValue;
 import org.apache.metron.reference.lookup.LookupKV;
-import org.apache.metron.threatintel.ThreatIntelResults;
 import org.mitre.cybox.common_2.StringObjectPropertyType;
 import org.mitre.cybox.objects.DomainName;
 import org.mitre.cybox.objects.DomainNameTypeEnum;
@@ -30,30 +29,49 @@ import java.io.IOException;
 import java.util.*;
 
 public class DomainHandler extends AbstractObjectTypeHandler<DomainName> {
-    EnumSet<DomainNameTypeEnum> SUPPORTED_TYPES = EnumSet.of(DomainNameTypeEnum.FQDN);
-    public DomainHandler() {
-        super(DomainName.class);
-    }
+  public static final String TYPE_CONFIG = "stix_domain_type";
+  EnumSet<DomainNameTypeEnum> SUPPORTED_TYPES = EnumSet.of(DomainNameTypeEnum.FQDN);
+  public DomainHandler() {
+    super(DomainName.class);
+  }
 
-    @Override
-    public Iterable<LookupKV> extract(final DomainName type, Map<String, Object> config) throws IOException {
-        List<LookupKV> ret = new ArrayList<>();
-        final DomainNameTypeEnum domainType = type.getType();
-        if(domainType == null || SUPPORTED_TYPES.contains(domainType)) {
-            StringObjectPropertyType value = type.getValue();
-            for (String token : StixExtractor.split(value)) {
-                LookupKV results = new LookupKV(new ThreatIntelKey(token)
-                                               , new ThreatIntelValue(
-                                                    new HashMap<String, String>() {{
-                                                        put("source-type", "STIX");
-                                                        put("indicator-type", type.getClass().getSimpleName() + ":" + DomainNameTypeEnum.FQDN);
-                                                        put("source", type.toXMLString());
-                                                    }}
-                                                                    )
-                                               );
-                ret.add(results);
-            }
-        }
-        return ret;
+  @Override
+  public Iterable<LookupKV> extract(final DomainName type, Map<String, Object> config) throws IOException {
+    List<LookupKV> ret = new ArrayList<>();
+    String typeStr = getType();
+    if(config != null) {
+      Object o = config.get(TYPE_CONFIG);
+      if(o != null) {
+        typeStr = o.toString();
+      }
+    }
+    final DomainNameTypeEnum domainType = type.getType();
+    if(domainType == null || SUPPORTED_TYPES.contains(domainType)) {
+      StringObjectPropertyType value = type.getValue();
+      for (String token : StixExtractor.split(value)) {
+        final String indicatorType = typeStr + ":" + DomainNameTypeEnum.FQDN;
+        LookupKV results = new LookupKV(new EnrichmentKey(indicatorType, token)
+                , new EnrichmentValue(
+                new HashMap<String, String>() {{
+                  put("source-type", "STIX");
+                  put("indicator-type", indicatorType);
+                  put("source", type.toXMLString());
+                }}
+        )
+        );
+        ret.add(results);
+      }
+    }
+    return ret;
+  }
+  @Override
+  public List<String> getPossibleTypes() {
+    String typeStr = getType();
+    List<String> ret = new ArrayList<>();
+    for(DomainNameTypeEnum e : SUPPORTED_TYPES)
+    {
+       ret.add(typeStr + ":" + e);
     }
+    return ret;
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/9621c55e/metron-streaming/Metron-DataLoads/src/main/java/org/apache/metron/dataloads/extractor/stix/types/HostnameHandler.java
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-DataLoads/src/main/java/org/apache/metron/dataloads/extractor/stix/types/HostnameHandler.java b/metron-streaming/Metron-DataLoads/src/main/java/org/apache/metron/dataloads/extractor/stix/types/HostnameHandler.java
index 702c440..ab02440 100644
--- a/metron-streaming/Metron-DataLoads/src/main/java/org/apache/metron/dataloads/extractor/stix/types/HostnameHandler.java
+++ b/metron-streaming/Metron-DataLoads/src/main/java/org/apache/metron/dataloads/extractor/stix/types/HostnameHandler.java
@@ -18,11 +18,11 @@
 
 package org.apache.metron.dataloads.extractor.stix.types;
 
+import com.google.common.collect.ImmutableList;
 import org.apache.metron.dataloads.extractor.stix.StixExtractor;
-import org.apache.metron.hbase.converters.threatintel.ThreatIntelKey;
-import org.apache.metron.hbase.converters.threatintel.ThreatIntelValue;
+import org.apache.metron.hbase.converters.enrichment.EnrichmentKey;
+import org.apache.metron.hbase.converters.enrichment.EnrichmentValue;
 import org.apache.metron.reference.lookup.LookupKV;
-import org.apache.metron.threatintel.ThreatIntelResults;
 import org.mitre.cybox.common_2.StringObjectPropertyType;
 import org.mitre.cybox.objects.Hostname;
 
@@ -33,25 +33,38 @@ import java.util.List;
 import java.util.Map;
 
 public class HostnameHandler  extends AbstractObjectTypeHandler<Hostname>{
-    public HostnameHandler() {
-        super(Hostname.class);
-    }
+  public static final String TYPE_CONFIG = "stix_hostname_type";
+  public HostnameHandler() {
+    super(Hostname.class);
+  }
 
-    @Override
-    public Iterable<LookupKV> extract(final Hostname type, Map<String, Object> config) throws IOException {
-        StringObjectPropertyType value = type.getHostnameValue();
-        List<LookupKV> ret = new ArrayList<>();
-        for(String token : StixExtractor.split(value)) {
-            LookupKV results = new LookupKV(new ThreatIntelKey(token)
-                                           , new ThreatIntelValue(new HashMap<String, String>() {{
-                                                                        put("source-type", "STIX");
-                                                                        put("indicator-type", type.getClass().getSimpleName());
-                                                                        put("source", type.toXMLString());
-                                                                    }}
-                                                                 )
-                                           );
-                ret.add(results);
-        }
-        return ret;
+  @Override
+  public Iterable<LookupKV> extract(final Hostname type, Map<String, Object> config) throws IOException {
+    StringObjectPropertyType value = type.getHostnameValue();
+    String typeStr = getType();
+    if(config != null) {
+      Object o = config.get(TYPE_CONFIG);
+      if(o != null) {
+        typeStr = o.toString();
+      }
+    }
+    List<LookupKV> ret = new ArrayList<>();
+    for(String token : StixExtractor.split(value)) {
+      final String indicatorType = typeStr;
+      LookupKV results = new LookupKV(new EnrichmentKey(indicatorType, token)
+              , new EnrichmentValue(new HashMap<String, String>() {{
+        put("source-type", "STIX");
+        put("indicator-type", indicatorType);
+        put("source", type.toXMLString());
+      }}
+      )
+      );
+      ret.add(results);
     }
+    return ret;
+  }
+  @Override
+  public List<String> getPossibleTypes() {
+    return ImmutableList.of(getType());
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/9621c55e/metron-streaming/Metron-DataLoads/src/main/java/org/apache/metron/dataloads/extractor/stix/types/ObjectTypeHandler.java
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-DataLoads/src/main/java/org/apache/metron/dataloads/extractor/stix/types/ObjectTypeHandler.java b/metron-streaming/Metron-DataLoads/src/main/java/org/apache/metron/dataloads/extractor/stix/types/ObjectTypeHandler.java
index e5a5296..57d72ff 100644
--- a/metron-streaming/Metron-DataLoads/src/main/java/org/apache/metron/dataloads/extractor/stix/types/ObjectTypeHandler.java
+++ b/metron-streaming/Metron-DataLoads/src/main/java/org/apache/metron/dataloads/extractor/stix/types/ObjectTypeHandler.java
@@ -17,15 +17,15 @@
  */
 package org.apache.metron.dataloads.extractor.stix.types;
 
-import org.apache.metron.dataloads.extractor.Extractor;
 import org.apache.metron.reference.lookup.LookupKV;
-import org.apache.metron.threatintel.ThreatIntelResults;
 import org.mitre.cybox.common_2.ObjectPropertiesType;
 
 import java.io.IOException;
+import java.util.List;
 import java.util.Map;
 
 public interface ObjectTypeHandler<T extends ObjectPropertiesType> {
-    Iterable<LookupKV> extract(T type, Map<String, Object> config) throws IOException;
-    Class<T> getTypeClass();
+  Iterable<LookupKV> extract(T type, Map<String, Object> config) throws IOException;
+  Class<T> getTypeClass();
+  List<String> getPossibleTypes();
 }

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/9621c55e/metron-streaming/Metron-DataLoads/src/main/java/org/apache/metron/dataloads/hbase/mr/BulkLoadMapper.java
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-DataLoads/src/main/java/org/apache/metron/dataloads/hbase/mr/BulkLoadMapper.java b/metron-streaming/Metron-DataLoads/src/main/java/org/apache/metron/dataloads/hbase/mr/BulkLoadMapper.java
index 04714d9..5baa3a5 100644
--- a/metron-streaming/Metron-DataLoads/src/main/java/org/apache/metron/dataloads/hbase/mr/BulkLoadMapper.java
+++ b/metron-streaming/Metron-DataLoads/src/main/java/org/apache/metron/dataloads/hbase/mr/BulkLoadMapper.java
@@ -26,8 +26,6 @@ import org.apache.metron.dataloads.extractor.Extractor;
 import org.apache.metron.dataloads.extractor.ExtractorHandler;
 import org.apache.metron.hbase.converters.HbaseConverter;
 import org.apache.metron.reference.lookup.LookupKV;
-import org.apache.metron.threatintel.ThreatIntelResults;
-import org.apache.metron.hbase.converters.threatintel.ThreatIntelConverter;
 
 import java.io.IOException;
 
@@ -39,7 +37,6 @@ public class BulkLoadMapper extends Mapper<Object, Text, ImmutableBytesWritable,
     public static final String CONVERTER_KEY = "bl_converter";
     Extractor extractor = null;
     String columnFamily = null;
-    Long lastSeen = null;
     HbaseConverter converter;
     @Override
     public void setup(Context context) throws IOException,
@@ -61,7 +58,6 @@ public class BulkLoadMapper extends Mapper<Object, Text, ImmutableBytesWritable,
         String configStr = configuration.get(CONFIG_KEY);
         extractor = ExtractorHandler.load(configStr).getExtractor();
         columnFamily = configuration.get(COLUMN_FAMILY_KEY);
-        lastSeen = Long.parseLong(configuration.get(LAST_SEEN_KEY));
         try {
             converter = (HbaseConverter) Class.forName(configuration.get(CONVERTER_KEY)).newInstance();
         } catch (InstantiationException e) {

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/9621c55e/metron-streaming/Metron-DataLoads/src/main/java/org/apache/metron/dataloads/hbase/mr/PrunerMapper.java
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-DataLoads/src/main/java/org/apache/metron/dataloads/hbase/mr/PrunerMapper.java b/metron-streaming/Metron-DataLoads/src/main/java/org/apache/metron/dataloads/hbase/mr/PrunerMapper.java
index bf33eed..b418a80 100644
--- a/metron-streaming/Metron-DataLoads/src/main/java/org/apache/metron/dataloads/hbase/mr/PrunerMapper.java
+++ b/metron-streaming/Metron-DataLoads/src/main/java/org/apache/metron/dataloads/hbase/mr/PrunerMapper.java
@@ -48,7 +48,7 @@ public class PrunerMapper extends TableMapper<ImmutableBytesWritable, Delete> {
         }
         try {
             tracker = AccessTrackerUtil.INSTANCE.loadAll(AccessTrackerUtil.INSTANCE.loadAll(table, atCF, atName, timestamp));
-        } catch (Exception e) {
+        } catch (Throwable e) {
             throw new IllegalStateException("Unable to load the accesstrackers from the directory", e);
         }
     }

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/9621c55e/metron-streaming/Metron-DataLoads/src/main/java/org/apache/metron/dataloads/nonbulk/flatfile/SimpleEnrichmentFlatFileLoader.java
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-DataLoads/src/main/java/org/apache/metron/dataloads/nonbulk/flatfile/SimpleEnrichmentFlatFileLoader.java b/metron-streaming/Metron-DataLoads/src/main/java/org/apache/metron/dataloads/nonbulk/flatfile/SimpleEnrichmentFlatFileLoader.java
new file mode 100644
index 0000000..816311f
--- /dev/null
+++ b/metron-streaming/Metron-DataLoads/src/main/java/org/apache/metron/dataloads/nonbulk/flatfile/SimpleEnrichmentFlatFileLoader.java
@@ -0,0 +1,263 @@
+/**
+ * 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.dataloads.nonbulk.flatfile;
+
+import com.google.common.base.Function;
+import com.google.common.base.Joiner;
+import com.google.common.collect.ImmutableList;
+import org.apache.commons.cli.*;
+import org.apache.commons.io.FileUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.client.HTableInterface;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.util.GenericOptionsParser;
+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.enrichment.EnrichmentConfig;
+import org.apache.metron.hbase.HTableProvider;
+import org.apache.metron.hbase.converters.HbaseConverter;
+import org.apache.metron.hbase.converters.enrichment.EnrichmentConverter;
+import org.apache.metron.hbase.converters.enrichment.EnrichmentKey;
+import org.apache.metron.hbase.converters.enrichment.EnrichmentValue;
+import org.apache.metron.reference.lookup.LookupKV;
+import org.apache.metron.utils.JSONUtils;
+
+import javax.annotation.Nullable;
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.FileReader;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Stack;
+
+public class SimpleEnrichmentFlatFileLoader {
+  private static abstract class OptionHandler implements Function<String, Option> {}
+  public static enum LoadOptions {
+    HELP("h", new OptionHandler() {
+
+      @Nullable
+      @Override
+      public Option apply(@Nullable String s) {
+        return new Option(s, "help", false, "Generate Help screen");
+      }
+    })
+    ,HBASE_TABLE("t", new OptionHandler() {
+      @Nullable
+      @Override
+      public Option apply(@Nullable String s) {
+        Option o = new Option(s, "hbase_table", true, "HBase table to ingest the data into.");
+        o.setArgName("TABLE");
+        o.setRequired(true);
+        return o;
+      }
+    })
+    ,HBASE_CF("c", new OptionHandler() {
+      @Nullable
+      @Override
+      public Option apply(@Nullable String s) {
+        Option o = new Option(s, "hbase_cf", true, "HBase column family to ingest the data into.");
+        o.setArgName("CF");
+        o.setRequired(true);
+        return o;
+      }
+    })
+    ,EXTRACTOR_CONFIG("e", new OptionHandler() {
+      @Nullable
+      @Override
+      public Option apply(@Nullable String s) {
+        Option o = new Option(s, "extractor_config", true, "JSON Document describing the extractor for this input data source");
+        o.setArgName("JSON_FILE");
+        o.setRequired(true);
+        return o;
+      }
+    })
+    ,ENRICHMENT_CONFIG("n", new OptionHandler() {
+      @Nullable
+      @Override
+      public Option apply(@Nullable String s) {
+        Option o = new Option(s, "enrichment_config", true
+                , "JSON Document describing the enrichment configuration details." +
+                "  This is used to associate an enrichment type with a field type in zookeeper."
+        );
+        o.setArgName("JSON_FILE");
+        o.setRequired(false);
+        return o;
+      }
+    })
+    ,LOG4J_PROPERTIES("l", new OptionHandler() {
+      @Nullable
+      @Override
+      public Option apply(@Nullable String s) {
+        Option o = new Option(s, "log4j", true, "The log4j properties file to load");
+        o.setArgName("FILE");
+        o.setRequired(false);
+        return o;
+      }
+    })
+    ,INPUT("i", new OptionHandler() {
+      @Nullable
+      @Override
+      public Option apply(@Nullable String s) {
+        Option o = new Option(s, "input", true, "The CSV File to load");
+        o.setArgName("FILE");
+        o.setRequired(true);
+        return o;
+      }
+    })
+    ;
+    Option option;
+    String shortCode;
+    LoadOptions(String shortCode, OptionHandler 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(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( "SimpleEnrichmentFlatFileLoader", getOptions());
+    }
+
+    public static Options getOptions() {
+      Options ret = new Options();
+      for(LoadOptions o : LoadOptions.values()) {
+        ret.addOption(o.option);
+      }
+      return ret;
+    }
+  }
+  public static List<File> getFiles(File root) {
+    if(!root.isDirectory())  {
+      return ImmutableList.of(root);
+    }
+    List<File> ret = new ArrayList<>();
+    Stack<File> stack = new Stack<File>();
+    stack.push(root);
+    while(!stack.isEmpty()) {
+      File f = stack.pop();
+      if(f.isDirectory()) {
+        for(File child : f.listFiles()) {
+          stack.push(child);
+        }
+      }
+      else {
+        ret.add(f);
+      }
+    }
+    return ret;
+  }
+
+  public HTableProvider getProvider() {
+    return new HTableProvider();
+  }
+
+  public List<Put> extract( String line
+                     , Extractor extractor
+                     , String cf
+                     , HbaseConverter converter
+                     ) throws IOException
+  {
+    List<Put> ret = new ArrayList<>();
+    Iterable<LookupKV> kvs = extractor.extract(line);
+    for(LookupKV kv : kvs) {
+      Put put = converter.toPut(cf, kv.getKey(), kv.getValue());
+      ret.add(put);
+    }
+    return ret;
+  }
+
+
+  public void loadFile( File inputFile
+                      , Extractor extractor
+                      , HTableInterface table
+                      , String cf
+                      , HbaseConverter converter
+                      , boolean lineByLine
+                      ) throws IOException
+  {
+    if(!lineByLine) {
+      table.put(extract(FileUtils.readFileToString(inputFile), extractor, cf, converter));
+    }
+    else {
+      BufferedReader br = new BufferedReader(new FileReader(inputFile));
+      for(String line = null;(line = br.readLine()) != null;) {
+        table.put(extract(line, extractor, cf, converter));
+      }
+    }
+  }
+  public static void main(String... argv) throws Exception {
+    Configuration conf = HBaseConfiguration.create();
+    String[] otherArgs = new GenericOptionsParser(conf, argv).getRemainingArgs();
+
+    CommandLine cli = LoadOptions.parse(new PosixParser(), otherArgs);
+    if(LoadOptions.LOG4J_PROPERTIES.has(cli)) {
+      PropertyConfigurator.configure(LoadOptions.LOG4J_PROPERTIES.get(cli));
+    }
+    ExtractorHandler handler = ExtractorHandler.load(
+            FileUtils.readFileToString(new File(LoadOptions.EXTRACTOR_CONFIG.get(cli)))
+    );
+    boolean lineByLine = !handler.getInputFormatHandler().getClass().equals(WholeFileFormat.class);
+    Extractor e = handler.getExtractor();
+    EnrichmentConfig enrichmentConfig = null;
+    if(LoadOptions.ENRICHMENT_CONFIG.has(cli)) {
+      enrichmentConfig = JSONUtils.INSTANCE.load( new File(LoadOptions.ENRICHMENT_CONFIG.get(cli))
+              , EnrichmentConfig.class
+      );
+    }
+    HbaseConverter converter = new EnrichmentConverter();
+    List<File> inputFiles = getFiles(new File(LoadOptions.INPUT.get(cli)));
+    SimpleEnrichmentFlatFileLoader loader = new SimpleEnrichmentFlatFileLoader();
+    HTableInterface table = loader.getProvider()
+            .getTable(conf, LoadOptions.HBASE_TABLE.get(cli));
+
+    for (File f : inputFiles) {
+      loader.loadFile(f, e, table, LoadOptions.HBASE_CF.get(cli), converter, lineByLine);
+    }
+    if(enrichmentConfig != null) {
+      enrichmentConfig.updateSensorConfigs();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/9621c55e/metron-streaming/Metron-DataLoads/src/main/java/org/apache/metron/dataloads/nonbulk/taxii/ConnectionType.java
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-DataLoads/src/main/java/org/apache/metron/dataloads/nonbulk/taxii/ConnectionType.java b/metron-streaming/Metron-DataLoads/src/main/java/org/apache/metron/dataloads/nonbulk/taxii/ConnectionType.java
new file mode 100644
index 0000000..77d1698
--- /dev/null
+++ b/metron-streaming/Metron-DataLoads/src/main/java/org/apache/metron/dataloads/nonbulk/taxii/ConnectionType.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.dataloads.nonbulk.taxii;
+
+public enum ConnectionType {
+   POLL, DISCOVER;
+}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/9621c55e/metron-streaming/Metron-DataLoads/src/main/java/org/apache/metron/dataloads/nonbulk/taxii/TableInfo.java
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-DataLoads/src/main/java/org/apache/metron/dataloads/nonbulk/taxii/TableInfo.java b/metron-streaming/Metron-DataLoads/src/main/java/org/apache/metron/dataloads/nonbulk/taxii/TableInfo.java
new file mode 100644
index 0000000..6bbf8e3
--- /dev/null
+++ b/metron-streaming/Metron-DataLoads/src/main/java/org/apache/metron/dataloads/nonbulk/taxii/TableInfo.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.dataloads.nonbulk.taxii;
+
+import com.google.common.base.Splitter;
+import com.google.common.collect.Iterables;
+
+public class TableInfo {
+    private String tableName;
+    private String columnFamily;
+    public TableInfo(String s) {
+        Iterable<String> i = Splitter.on(":").split(s);
+        if(Iterables.size(i) != 2) {
+            throw new IllegalStateException("Malformed table:cf => " + s);
+        }
+        tableName = Iterables.getFirst(i, null);
+        columnFamily = Iterables.getLast(i);
+    }
+
+    public String getTableName() {
+        return tableName;
+    }
+
+    public String getColumnFamily() {
+        return columnFamily;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (this == o) return true;
+        if (o == null || getClass() != o.getClass()) return false;
+
+        TableInfo tableInfo = (TableInfo) o;
+
+        if (getTableName() != null ? !getTableName().equals(tableInfo.getTableName()) : tableInfo.getTableName() != null)
+            return false;
+        return getColumnFamily() != null ? getColumnFamily().equals(tableInfo.getColumnFamily()) : tableInfo.getColumnFamily() == null;
+
+    }
+
+    @Override
+    public int hashCode() {
+        int result = getTableName() != null ? getTableName().hashCode() : 0;
+        result = 31 * result + (getColumnFamily() != null ? getColumnFamily().hashCode() : 0);
+        return result;
+    }
+
+    @Override
+    public String toString() {
+        return "TableInfo{" +
+                "tableName='" + tableName + '\'' +
+                ", columnFamily='" + columnFamily + '\'' +
+                '}';
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/9621c55e/metron-streaming/Metron-DataLoads/src/main/java/org/apache/metron/dataloads/nonbulk/taxii/TaxiiConnectionConfig.java
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-DataLoads/src/main/java/org/apache/metron/dataloads/nonbulk/taxii/TaxiiConnectionConfig.java b/metron-streaming/Metron-DataLoads/src/main/java/org/apache/metron/dataloads/nonbulk/taxii/TaxiiConnectionConfig.java
new file mode 100644
index 0000000..678f98b
--- /dev/null
+++ b/metron-streaming/Metron-DataLoads/src/main/java/org/apache/metron/dataloads/nonbulk/taxii/TaxiiConnectionConfig.java
@@ -0,0 +1,222 @@
+/**
+ * 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.dataloads.nonbulk.taxii;
+
+import com.google.common.base.Joiner;
+import org.apache.metron.dataloads.extractor.stix.types.ObjectTypeHandlers;
+import org.codehaus.jackson.map.ObjectMapper;
+
+import java.io.ByteArrayInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.net.MalformedURLException;
+import java.net.URL;
+import java.nio.charset.Charset;
+import java.text.DateFormat;
+import java.text.ParseException;
+import java.text.SimpleDateFormat;
+import java.util.*;
+
+public class TaxiiConnectionConfig {
+  final static ObjectMapper _mapper = new ObjectMapper();
+  private URL endpoint;
+  private int port = 443;
+  private URL proxy;
+  private String username;
+  private String password;
+  private ConnectionType type;
+  private String collection = "default";
+  private String subscriptionId = null;
+  private Date beginTime;
+  private String table;
+  private String columnFamily;
+  private Set<String> allowedIndicatorTypes = new HashSet<String>();
+
+  public TaxiiConnectionConfig withAllowedIndicatorTypes(List<String> indicatorTypes) {
+    allowedIndicatorTypes = new HashSet(indicatorTypes);
+    return this;
+  }
+
+  public TaxiiConnectionConfig withTable(String table) {
+    this.table = table;
+    return this;
+  }
+  public TaxiiConnectionConfig withColumnFamily(String cf) {
+    this.columnFamily = cf;
+    return this;
+  }
+  public TaxiiConnectionConfig withBeginTime(Date time) {
+    this.beginTime = time;
+    return this;
+  }
+  public TaxiiConnectionConfig withSubscriptionId(String subId) {
+    this.subscriptionId = subId;
+    return this;
+  }
+  public TaxiiConnectionConfig withCollection(String collection) {
+    this.collection = collection;
+    return this;
+  }
+
+  public TaxiiConnectionConfig withPort(int port) {
+    this.port = port;
+    return this;
+  }
+  public TaxiiConnectionConfig withEndpoint(URL endpoint) {
+    this.endpoint = endpoint;
+    return this;
+  }
+  public TaxiiConnectionConfig withProxy(URL proxy) {
+    this.proxy = proxy;
+    return this;
+  }
+  public TaxiiConnectionConfig withUsername(String username) {
+    this.username = username;
+    return this;
+  }
+  public TaxiiConnectionConfig withPassword(String password) {
+    this.password = password;
+    return this;
+  }
+  public TaxiiConnectionConfig withConnectionType(ConnectionType type) {
+    this.type= type;
+    return this;
+  }
+
+  public void setEndpoint(String endpoint) throws MalformedURLException {
+    this.endpoint = new URL(endpoint);
+  }
+
+  public void setPort(int port) {
+    this.port = port;
+  }
+
+  public void setProxy(String proxy) throws MalformedURLException {
+    this.proxy = new URL(proxy);
+  }
+
+  public void setUsername(String username) {
+    this.username = username;
+  }
+
+  public void setPassword(String password) {
+    this.password = password;
+  }
+
+  public void setType(ConnectionType type) {
+    this.type = type;
+  }
+
+  public void setCollection(String collection) {
+    this.collection = collection;
+  }
+
+  public void setSubscriptionId(String subscriptionId) {
+    this.subscriptionId = subscriptionId;
+  }
+
+  public void setBeginTime(String beginTime) throws ParseException {
+    SimpleDateFormat sdf = (SimpleDateFormat)DateFormat.getDateInstance(DateFormat.MEDIUM);
+    this.beginTime = sdf.parse(beginTime);
+  }
+
+  public String getTable() {
+    return table;
+  }
+
+  public void setTable(String table) {
+    this.table = table;
+  }
+
+  public String getColumnFamily() {
+    return columnFamily;
+  }
+
+  public void setColumnFamily(String columnFamily) {
+    this.columnFamily = columnFamily;
+  }
+
+  public Date getBeginTime() {
+    return beginTime;
+  }
+  public int getPort() {
+    return port;
+  }
+  public URL getEndpoint() {
+    return endpoint;
+  }
+
+  public URL getProxy() {
+    return proxy;
+  }
+
+  public String getUsername() {
+    return username;
+  }
+
+  public String getPassword() {
+    return password;
+  }
+
+  public ConnectionType getType() {
+    return type;
+  }
+
+  public String getCollection() {
+    return collection;
+  }
+  public String getSubscriptionId() {
+    return subscriptionId;
+  }
+
+  public void setAllowedIndicatorTypes(List<String> allowedIndicatorTypes) {
+    withAllowedIndicatorTypes(allowedIndicatorTypes);
+  }
+
+  public Set<String> getAllowedIndicatorTypes() {
+    return allowedIndicatorTypes;
+  }
+  public static synchronized TaxiiConnectionConfig load(InputStream is) throws IOException {
+    TaxiiConnectionConfig ret = _mapper.readValue(is, TaxiiConnectionConfig.class);
+    return ret;
+  }
+  public static synchronized TaxiiConnectionConfig load(String s, Charset c) throws IOException {
+    return load( new ByteArrayInputStream(s.getBytes(c)));
+  }
+  public static synchronized TaxiiConnectionConfig load(String s) throws IOException {
+    return load( s, Charset.defaultCharset());
+  }
+
+  @Override
+  public String toString() {
+    return "TaxiiConnectionConfig{" +
+            "endpoint=" + endpoint +
+            ", port=" + port +
+            ", proxy=" + proxy +
+            ", username='" + username + '\'' +
+            ", password=" + (password == null?"null" : "'******'") +
+            ", type=" + type +
+            ", allowedIndicatorTypes=" + Joiner.on(',').join(allowedIndicatorTypes)+
+            ", collection='" + collection + '\'' +
+            ", subscriptionId='" + subscriptionId + '\'' +
+            ", beginTime=" + beginTime +
+            ", table=" + table + ":" + columnFamily+
+            '}';
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/9621c55e/metron-streaming/Metron-DataLoads/src/main/java/org/apache/metron/dataloads/nonbulk/taxii/TaxiiHandler.java
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-DataLoads/src/main/java/org/apache/metron/dataloads/nonbulk/taxii/TaxiiHandler.java b/metron-streaming/Metron-DataLoads/src/main/java/org/apache/metron/dataloads/nonbulk/taxii/TaxiiHandler.java
new file mode 100644
index 0000000..1e45f94
--- /dev/null
+++ b/metron-streaming/Metron-DataLoads/src/main/java/org/apache/metron/dataloads/nonbulk/taxii/TaxiiHandler.java
@@ -0,0 +1,406 @@
+/**
+ * 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.dataloads.nonbulk.taxii;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.client.HTable;
+import org.apache.hadoop.hbase.client.HTableInterface;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.http.HttpHost;
+import org.apache.http.auth.AuthScope;
+import org.apache.http.auth.UsernamePasswordCredentials;
+import org.apache.http.client.AuthCache;
+import org.apache.http.client.CredentialsProvider;
+import org.apache.http.client.protocol.HttpClientContext;
+import org.apache.http.config.Registry;
+import org.apache.http.config.RegistryBuilder;
+import org.apache.http.conn.socket.ConnectionSocketFactory;
+import org.apache.http.conn.socket.PlainConnectionSocketFactory;
+import org.apache.http.conn.ssl.SSLConnectionSocketFactory;
+import org.apache.http.conn.ssl.SSLContextBuilder;
+import org.apache.http.conn.ssl.TrustSelfSignedStrategy;
+import org.apache.http.impl.auth.BasicScheme;
+import org.apache.http.impl.client.BasicAuthCache;
+import org.apache.http.impl.client.BasicCredentialsProvider;
+import org.apache.http.impl.client.CloseableHttpClient;
+import org.apache.http.impl.client.HttpClientBuilder;
+import org.apache.http.impl.conn.PoolingHttpClientConnectionManager;
+import org.apache.log4j.Logger;
+import org.apache.metron.dataloads.extractor.Extractor;
+import org.apache.metron.hbase.converters.enrichment.EnrichmentConverter;
+import org.apache.metron.hbase.converters.enrichment.EnrichmentKey;
+import org.apache.metron.hbase.converters.enrichment.EnrichmentValue;
+import org.apache.metron.reference.lookup.LookupKV;
+import org.mitre.taxii.client.HttpClient;
+import org.mitre.taxii.messages.xml11.*;
+import org.w3c.dom.Document;
+import org.w3c.dom.Element;
+
+import javax.xml.bind.JAXBException;
+import javax.xml.datatype.DatatypeConfigurationException;
+import javax.xml.datatype.DatatypeFactory;
+import javax.xml.datatype.XMLGregorianCalendar;
+import javax.xml.transform.Transformer;
+import javax.xml.transform.TransformerException;
+import javax.xml.transform.TransformerFactory;
+import javax.xml.transform.dom.DOMSource;
+import javax.xml.transform.stream.StreamResult;
+import java.io.*;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.net.URL;
+import java.text.SimpleDateFormat;
+import java.util.*;
+
+public class TaxiiHandler extends TimerTask {
+    private static final Logger LOG = Logger.getLogger(TaxiiHandler.class);
+
+    private static ThreadLocal<TaxiiXmlFactory> xmlFactory = new ThreadLocal<TaxiiXmlFactory>() {
+        @Override
+        protected TaxiiXmlFactory initialValue() {
+            return new TaxiiXmlFactory();
+        }
+    };
+    private static ThreadLocal<ObjectFactory> messageFactory = new ThreadLocal<ObjectFactory>() {
+        @Override
+        protected ObjectFactory initialValue() {
+            return new ObjectFactory();
+        }
+    };
+
+    private HttpClient taxiiClient;
+    private URL endpoint;
+    private Extractor extractor;
+    private String hbaseTable;
+    private String columnFamily;
+    private Map<String, HTableInterface> connectionCache = new HashMap<>();
+    private HttpClientContext context;
+    private String collection;
+    private String subscriptionId;
+    private EnrichmentConverter converter = new EnrichmentConverter();
+    private Date beginTime;
+    private Configuration config;
+    private boolean inProgress = false;
+    private Set<String> allowedIndicatorTypes;
+    public TaxiiHandler( TaxiiConnectionConfig connectionConfig
+                       , Extractor extractor
+                       , Configuration config
+                       ) throws Exception
+    {
+        LOG.info("Loading configuration: " + connectionConfig);
+        this.allowedIndicatorTypes = connectionConfig.getAllowedIndicatorTypes();
+        this.extractor = extractor;
+        this.collection = connectionConfig.getCollection();
+        this.subscriptionId = connectionConfig.getSubscriptionId();
+        hbaseTable = connectionConfig.getTable();
+        columnFamily = connectionConfig.getColumnFamily();
+        this.beginTime = connectionConfig.getBeginTime();
+        this.config = config;
+        initializeClient(connectionConfig);
+        LOG.info("Configured, starting polling " + endpoint + " for " + collection);
+    }
+
+    protected synchronized HTableInterface getTable(String table) throws IOException {
+        HTableInterface ret = connectionCache.get(table);
+        if(ret == null) {
+            ret = createHTable(table);
+            connectionCache.put(table, ret);
+        }
+        return ret;
+    }
+
+    protected synchronized HTableInterface createHTable(String tableInfo) throws IOException {
+        return new HTable(config, tableInfo);
+    }
+    /**
+     * The action to be performed by this timer task.
+     */
+    @Override
+    public void run() {
+        if(inProgress) {
+            return;
+        }
+        Date ts = new Date();
+        LOG.info("Polling..." + new SimpleDateFormat().format(ts));
+        try {
+            inProgress = true;
+            // Prepare the message to send.
+            String sessionID = MessageHelper.generateMessageId();
+            PollRequest request = messageFactory.get().createPollRequest()
+                    .withMessageId(sessionID)
+                    .withCollectionName(collection);
+            if (subscriptionId != null) {
+                request = request.withSubscriptionID(subscriptionId);
+            } else {
+                request = request.withPollParameters(messageFactory.get().createPollParametersType());
+            }
+            if (beginTime != null) {
+                Calendar gc = GregorianCalendar.getInstance();
+                gc.setTime(beginTime);
+                XMLGregorianCalendar gTime = null;
+                try {
+                    gTime = DatatypeFactory.newInstance().newXMLGregorianCalendar((GregorianCalendar) gc).normalize();
+                } catch (DatatypeConfigurationException e) {
+                    LOG.error("Unable to set the begin time", e);
+                }
+                gTime.setFractionalSecond(null);
+                LOG.info("Begin Time: " + gTime);
+                request.setExclusiveBeginTimestamp(gTime);
+            }
+
+            try {
+                PollResponse response = call(request, PollResponse.class);
+                LOG.info("Got Poll Response with " + response.getContentBlocks().size() + " blocks");
+                int numProcessed = 0;
+                long avgTimeMS = 0;
+                long timeStartedBlock = System.currentTimeMillis();
+                for (ContentBlock block : response.getContentBlocks()) {
+                    AnyMixedContentType content = block.getContent();
+                    for (Object o : content.getContent()) {
+                        numProcessed++;
+                        long timeS = System.currentTimeMillis();
+                        String xml = null;
+                        if (o instanceof Element) {
+                            Element element = (Element) o;
+                            xml = getStringFromDocument(element.getOwnerDocument());
+                            if(LOG.isDebugEnabled() && Math.random() < 0.01) {
+                                LOG.debug("Random Stix doc: " + xml);
+                            }
+                            for (LookupKV<EnrichmentKey, EnrichmentValue> kv : extractor.extract(xml)) {
+                                if(allowedIndicatorTypes.isEmpty()
+                                || allowedIndicatorTypes.contains(kv.getKey().type)
+                                  )
+                                {
+                                    kv.getValue().getMetadata().put("source_type", "taxii");
+                                    kv.getValue().getMetadata().put("taxii_url", endpoint.toString());
+                                    kv.getValue().getMetadata().put("taxii_collection", collection);
+                                    Put p = converter.toPut(columnFamily, kv.getKey(), kv.getValue());
+                                    HTableInterface table = getTable(hbaseTable);
+                                    table.put(p);
+                                    LOG.info("Found Threat Intel: " + kv.getKey() + " => " + kv.getValue());
+                                }
+                            }
+                        }
+                        avgTimeMS += System.currentTimeMillis() - timeS;
+                    }
+                    if( (numProcessed + 1) % 100 == 0) {
+                        LOG.info("Processed " + numProcessed + " in " + (System.currentTimeMillis() - timeStartedBlock) + " ms, avg time: " + avgTimeMS / content.getContent().size());
+                        timeStartedBlock = System.currentTimeMillis();
+                        avgTimeMS = 0;
+                        numProcessed = 0;
+                    }
+                }
+            } catch (Exception e) {
+                LOG.error(e.getMessage(), e);
+                throw new RuntimeException("Unable to make request", e);
+            }
+        }
+        finally {
+            inProgress = false;
+            beginTime = ts;
+        }
+    }
+    public String getStringFromDocument(Document doc)
+    {
+        try
+        {
+            DOMSource domSource = new DOMSource(doc);
+            StringWriter writer = new StringWriter();
+            StreamResult result = new StreamResult(writer);
+            TransformerFactory tf = TransformerFactory.newInstance();
+            Transformer transformer = tf.newTransformer();
+            transformer.transform(domSource, result);
+            return writer.toString();
+        }
+        catch(TransformerException ex)
+        {
+            ex.printStackTrace();
+            return null;
+        }
+    }
+    private <RESPONSE_T> RESPONSE_T call( Object request, Class<RESPONSE_T> responseClazz) throws URISyntaxException, JAXBException, IOException {
+        return call(taxiiClient, endpoint.toURI(), request, context, responseClazz);
+    }
+
+    private void initializeClient(TaxiiConnectionConfig config) throws Exception {
+        LOG.info("Initializing client..");
+        if(context == null) {
+            context = createContext(config.getEndpoint(), config.getUsername(), config.getPassword(), config.getPort());
+        }
+        URL endpoint = config.getEndpoint();
+        if(config.getType() == ConnectionType.DISCOVER) {
+            LOG.info("Discovering endpoint");
+            endpoint = discoverPollingClient(config.getProxy(), endpoint, config.getUsername(), config.getPassword(), context, collection).pollEndpoint;
+            this.endpoint = endpoint;
+            LOG.info("Discovered endpoint as " + endpoint);
+        }
+        taxiiClient = buildClient(config.getProxy(), config.getUsername(), config.getPassword());
+    }
+
+    private static class DiscoveryResults {
+        URL pollEndpoint;
+        URL collectionManagementEndpoint;
+        List<String> collections = new ArrayList<>();
+    }
+    private static DiscoveryResults discoverPollingClient(URL proxy, URL endpoint, String username, String password, HttpClientContext context, String defaultCollection) throws Exception {
+
+        DiscoveryResults results = new DiscoveryResults();
+        {
+            HttpClient discoverClient = buildClient(proxy, username, password);
+            String sessionID = MessageHelper.generateMessageId();
+            // Prepare the message to send.
+            DiscoveryRequest request = messageFactory.get().createDiscoveryRequest()
+                    .withMessageId(sessionID);
+            DiscoveryResponse response = call(discoverClient, endpoint.toURI(), request, context, DiscoveryResponse.class);
+            for (ServiceInstanceType serviceInstance : response.getServiceInstances()) {
+                if (serviceInstance.isAvailable() && serviceInstance.getServiceType() == ServiceTypeEnum.POLL) {
+                    results.pollEndpoint = new URL(serviceInstance.getAddress());
+                }
+                else if(serviceInstance.isAvailable() && serviceInstance.getServiceType() == ServiceTypeEnum.COLLECTION_MANAGEMENT) {
+                    results.collectionManagementEndpoint= new URL(serviceInstance.getAddress());
+                }
+            }
+            if (results.pollEndpoint == null) {
+                throw new RuntimeException("Unable to discover a poll TAXII feed");
+            }
+        }
+        if(defaultCollection == null)
+        //get collections
+        {
+            HttpClient discoverClient = buildClient(proxy, username, password);
+            String sessionID = MessageHelper.generateMessageId();
+            CollectionInformationRequest request = messageFactory.get().createCollectionInformationRequest()
+                                                                 .withMessageId(sessionID);
+            CollectionInformationResponse response = call(discoverClient, results.collectionManagementEndpoint.toURI(), request, context, CollectionInformationResponse.class);
+            LOG.info("Unable to find the default collection; available collections are:");
+            for(CollectionRecordType c : response.getCollections()) {
+                LOG.info(c.getCollectionName());
+                results.collections.add(c.getCollectionName());
+            }
+            System.exit(0);
+        }
+        return results;
+    }
+
+    private static HttpClientContext createContext(URL endpoint, String username, String password, int port) {
+        HttpClientContext context = null;
+        HttpHost target = new HttpHost(endpoint.getHost(), port, endpoint.getProtocol());
+        if (username != null && password != null) {
+
+            CredentialsProvider credsProvider = new BasicCredentialsProvider();
+            credsProvider.setCredentials(
+                    new AuthScope(target.getHostName(), target.getPort()),
+                    new UsernamePasswordCredentials(username, password));
+
+            // http://hc.apache.org/httpcomponents-client-ga/tutorial/html/authentication.html
+            AuthCache authCache = new BasicAuthCache();
+            authCache.put(target, new BasicScheme());
+
+            // Add AuthCache to the execution context
+            context = HttpClientContext.create();
+            context.setCredentialsProvider(credsProvider);
+            context.setAuthCache(authCache);
+        } else {
+            context = null;
+        }
+        return context;
+    }
+
+
+    public static <RESPONSE_T, REQUEST_T> RESPONSE_T call( HttpClient taxiiClient
+            , URI endpoint
+            , REQUEST_T request
+            , HttpClientContext context
+            , Class<RESPONSE_T> responseClazz
+    ) throws JAXBException, IOException {
+        //TaxiiXml taxiiXml = xmlFactory.get().createTaxiiXml();
+        //String req = taxiiXml.marshalToString(request, true);
+        // Call the service
+        Object responseObj =  taxiiClient.callTaxiiService(endpoint, request, context);
+        LOG.info("Request made : " + request.getClass().getCanonicalName() + " => " + responseObj.getClass().getCanonicalName() + " (expected " + responseClazz.getCanonicalName() + ")");
+        //String resp = taxiiXml.marshalToString(responseObj, true);
+        try {
+            return responseClazz.cast(responseObj);
+        }
+        catch(ClassCastException cce) {
+            TaxiiXml taxiiXml = xmlFactory.get().createTaxiiXml();
+            String resp = taxiiXml.marshalToString(responseObj, true);
+            String msg = "Didn't return the response we expected: " + responseObj.getClass() + " \n" + resp;
+            LOG.error(msg, cce);
+            throw new RuntimeException(msg, cce);
+        }
+    }
+    private static HttpClient buildClient(URL proxy, String username, String password) throws Exception
+    {
+        HttpClient client = new HttpClient(); // Start with a default TAXII HTTP client.
+
+        // Create an Apache HttpClientBuilder to be customized by the command line arguments.
+        HttpClientBuilder builder = HttpClientBuilder.create().useSystemProperties();
+
+        // Proxy
+        if (proxy != null) {
+            HttpHost proxyHost = new HttpHost(proxy.getHost(), proxy.getPort(), proxy.getProtocol());
+            builder.setProxy(proxyHost);
+        }
+
+        // Basic authentication. User & Password
+        if (username != null ^ password != null) {
+            throw new Exception("'username' and 'password' arguments are required to appear together.");
+        }
+
+
+        // from:  http://stackoverflow.com/questions/19517538/ignoring-ssl-certificate-in-apache-httpclient-4-3
+        SSLContextBuilder ssbldr = new SSLContextBuilder();
+        ssbldr.loadTrustMaterial(null, new TrustSelfSignedStrategy());
+        SSLConnectionSocketFactory sslsf = new SSLConnectionSocketFactory(ssbldr.build(),SSLConnectionSocketFactory.BROWSER_COMPATIBLE_HOSTNAME_VERIFIER);
+
+
+        Registry<ConnectionSocketFactory> registry = RegistryBuilder.<ConnectionSocketFactory>create()
+                .register("http", new PlainConnectionSocketFactory())
+                .register("https", sslsf)
+                .build();
+
+
+        PoolingHttpClientConnectionManager cm = new PoolingHttpClientConnectionManager(registry);
+        cm.setMaxTotal(20);//max connection
+
+        System.setProperty("jsse.enableSNIExtension", "false"); //""
+        CloseableHttpClient httpClient = builder
+                .setSSLSocketFactory(sslsf)
+                .setConnectionManager(cm)
+                .build();
+
+        client.setHttpclient(httpClient);
+        return client;
+    }
+    public static void main(String... argv) throws Exception {
+        URL endpoint = new URL("http://hailataxii.com/taxii-discovery-service");
+        String username = "guest";
+        String password = "guest";
+        TaxiiConnectionConfig config = new TaxiiConnectionConfig();
+        config = config.withConnectionType(ConnectionType.DISCOVER)
+                       .withEndpoint(endpoint)
+                       .withUsername(username)
+                       .withCollection("guest.Abuse_ch")
+                       .withPassword(password);
+        //TaxiiHandler handler = new TaxiiHandler(config, null);
+        //handler.run();
+        //discoverPollingClient(null, endpoint, username, password, context);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/9621c55e/metron-streaming/Metron-DataLoads/src/main/java/org/apache/metron/dataloads/nonbulk/taxii/TaxiiLoader.java
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-DataLoads/src/main/java/org/apache/metron/dataloads/nonbulk/taxii/TaxiiLoader.java b/metron-streaming/Metron-DataLoads/src/main/java/org/apache/metron/dataloads/nonbulk/taxii/TaxiiLoader.java
new file mode 100644
index 0000000..712fcf3
--- /dev/null
+++ b/metron-streaming/Metron-DataLoads/src/main/java/org/apache/metron/dataloads/nonbulk/taxii/TaxiiLoader.java
@@ -0,0 +1,208 @@
+/**
+ * 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.dataloads.nonbulk.taxii;
+
+import com.google.common.base.Function;
+import com.google.common.base.Joiner;
+import org.apache.commons.cli.*;
+import org.apache.commons.cli.ParseException;
+import org.apache.commons.io.FileUtils;
+import org.apache.curator.framework.CuratorFramework;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.util.GenericOptionsParser;
+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.enrichment.EnrichmentConfig;
+import org.apache.metron.utils.JSONUtils;
+
+import javax.annotation.Nullable;
+import java.io.File;
+import java.text.*;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Timer;
+
+public class TaxiiLoader {
+  private static abstract class OptionHandler implements Function<String, Option> {}
+  private enum TaxiiOptions {
+    HELP("h", new OptionHandler() {
+
+      @Nullable
+      @Override
+      public Option apply(@Nullable String s) {
+        return new Option(s, "help", false, "Generate Help screen");
+      }
+    })
+    ,EXTRACTOR_CONFIG("e", new OptionHandler() {
+      @Nullable
+      @Override
+      public Option apply(@Nullable String s) {
+        Option o = new Option(s, "extractor_config", true, "JSON Document describing the extractor for this input data source");
+        o.setArgName("JSON_FILE");
+        o.setRequired(true);
+        return o;
+      }
+    })
+    ,CONNECTION_CONFIG("c", new OptionHandler() {
+      @Nullable
+      @Override
+      public Option apply(@Nullable String s) {
+        Option o = new Option(s, "taxii_connection_config", true, "The JSON config file to configure the connection");
+        o.setArgName("config_file");
+        o.setRequired(true);
+        return o;
+      }
+    })
+    ,TIME_BETWEEN_POLLS("p", new OptionHandler() {
+      @Nullable
+      @Override
+      public Option apply(@Nullable String s) {
+        Option o = new Option(s, "time_between_polls", true, "The time between polls (in ms)");
+        o.setArgName("MS");
+        o.setRequired(false);
+        return o;
+      }
+    })
+    ,BEGIN_TIME("b", new OptionHandler() {
+      @Nullable
+      @Override
+      public Option apply(@Nullable String s) {
+        Option o = new Option(s, "begin_time", true, "Start time to poll the Taxii server (all data from that point will be gathered in the first pull).");
+        o.setArgName(DATE_FORMAT.toPattern());
+        o.setRequired(false);
+        return o;
+      }
+    })
+    ,LOG4J_PROPERTIES("l", new OptionHandler() {
+      @Nullable
+      @Override
+      public Option apply(@Nullable String s) {
+        Option o = new Option(s, "log4j", true, "The log4j properties file to load");
+        o.setArgName("FILE");
+        o.setRequired(false);
+        return o;
+      }
+    })
+    ,ENRICHMENT_CONFIG("n", new OptionHandler() {
+      @Nullable
+      @Override
+      public Option apply(@Nullable String s) {
+        Option o = new Option(s, "enrichment_config", true
+                , "JSON Document describing the enrichment configuration details." +
+                "  This is used to associate an enrichment type with a field type in zookeeper."
+        );
+        o.setArgName("JSON_FILE");
+        o.setRequired(false);
+        return o;
+      }
+    })
+    ;
+    Option option;
+    String shortCode;
+    TaxiiOptions(String shortCode, OptionHandler 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(TaxiiOptions.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( "TaxiiLoader", getOptions());
+    }
+
+    public static Options getOptions() {
+      Options ret = new Options();
+      for(TaxiiOptions o : TaxiiOptions.values()) {
+        ret.addOption(o.option);
+      }
+      return ret;
+    }
+  }
+  public static final SimpleDateFormat DATE_FORMAT = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss");
+  public static final long ONE_HR_IN_MS = 60*60*1000;
+  public static final long DEFAULT_TIME_BETWEEN_POLLS = ONE_HR_IN_MS;
+
+
+  public static void main(String... argv) throws Exception {
+    Configuration conf = HBaseConfiguration.create();
+    String zkQuorum = conf.get(HConstants.ZOOKEEPER_QUORUM);
+    String[] otherArgs = new GenericOptionsParser(conf, argv).getRemainingArgs();
+
+    CommandLine cli = TaxiiOptions.parse(new PosixParser(), otherArgs);
+    if(TaxiiOptions.LOG4J_PROPERTIES.has(cli)) {
+      PropertyConfigurator.configure(TaxiiOptions.LOG4J_PROPERTIES.get(cli));
+    }
+    ExtractorHandler handler = ExtractorHandler.load(FileUtils.readFileToString(new File(TaxiiOptions.EXTRACTOR_CONFIG.get(cli))));
+    Extractor e = handler.getExtractor();
+    EnrichmentConfig enrichmentConfig = null;
+    if(TaxiiOptions.ENRICHMENT_CONFIG.has(cli)) {
+      enrichmentConfig = JSONUtils.INSTANCE.load( new File(TaxiiOptions.ENRICHMENT_CONFIG.get(cli))
+              , EnrichmentConfig.class
+      );
+      enrichmentConfig.updateSensorConfigs();
+    }
+
+    Timer timer = new Timer();
+    if(e instanceof StixExtractor) {
+      StixExtractor extractor = (StixExtractor)e;
+      TaxiiConnectionConfig connectionConfig = TaxiiConnectionConfig.load(FileUtils.readFileToString(new File(TaxiiOptions.CONNECTION_CONFIG.get(cli))));
+      if(TaxiiOptions.BEGIN_TIME.has(cli)) {
+        Date d = DATE_FORMAT.parse(TaxiiOptions.BEGIN_TIME.get(cli));
+        connectionConfig.withBeginTime(d);
+      }
+      long timeBetween = DEFAULT_TIME_BETWEEN_POLLS;
+      if(TaxiiOptions.TIME_BETWEEN_POLLS.has(cli)) {
+        timeBetween = Long.parseLong(TaxiiOptions.TIME_BETWEEN_POLLS.get(cli));
+      }
+      timer.scheduleAtFixedRate(new TaxiiHandler(connectionConfig, extractor, conf), 0, timeBetween);
+    }
+    else {
+      throw new IllegalStateException("Extractor must be a STIX Extractor");
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/9621c55e/metron-streaming/Metron-DataLoads/src/main/java/org/apache/metron/dataloads/taxii/ConnectionType.java
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-DataLoads/src/main/java/org/apache/metron/dataloads/taxii/ConnectionType.java b/metron-streaming/Metron-DataLoads/src/main/java/org/apache/metron/dataloads/taxii/ConnectionType.java
deleted file mode 100644
index 1e95507..0000000
--- a/metron-streaming/Metron-DataLoads/src/main/java/org/apache/metron/dataloads/taxii/ConnectionType.java
+++ /dev/null
@@ -1,23 +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.dataloads.taxii;
-
-public enum ConnectionType {
-   POLL, DISCOVER;
-}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/9621c55e/metron-streaming/Metron-DataLoads/src/main/java/org/apache/metron/dataloads/taxii/TableInfo.java
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-DataLoads/src/main/java/org/apache/metron/dataloads/taxii/TableInfo.java b/metron-streaming/Metron-DataLoads/src/main/java/org/apache/metron/dataloads/taxii/TableInfo.java
deleted file mode 100644
index ddf542e..0000000
--- a/metron-streaming/Metron-DataLoads/src/main/java/org/apache/metron/dataloads/taxii/TableInfo.java
+++ /dev/null
@@ -1,72 +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.dataloads.taxii;
-
-import com.google.common.base.Splitter;
-import com.google.common.collect.Iterables;
-import org.apache.hadoop.hbase.client.HTableInterface;
-
-public class TableInfo {
-    private String tableName;
-    private String columnFamily;
-    public TableInfo(String s) {
-        Iterable<String> i = Splitter.on(":").split(s);
-        if(Iterables.size(i) != 2) {
-            throw new IllegalStateException("Malformed table:cf => " + s);
-        }
-        tableName = Iterables.getFirst(i, null);
-        columnFamily = Iterables.getLast(i);
-    }
-
-    public String getTableName() {
-        return tableName;
-    }
-
-    public String getColumnFamily() {
-        return columnFamily;
-    }
-
-    @Override
-    public boolean equals(Object o) {
-        if (this == o) return true;
-        if (o == null || getClass() != o.getClass()) return false;
-
-        TableInfo tableInfo = (TableInfo) o;
-
-        if (getTableName() != null ? !getTableName().equals(tableInfo.getTableName()) : tableInfo.getTableName() != null)
-            return false;
-        return getColumnFamily() != null ? getColumnFamily().equals(tableInfo.getColumnFamily()) : tableInfo.getColumnFamily() == null;
-
-    }
-
-    @Override
-    public int hashCode() {
-        int result = getTableName() != null ? getTableName().hashCode() : 0;
-        result = 31 * result + (getColumnFamily() != null ? getColumnFamily().hashCode() : 0);
-        return result;
-    }
-
-    @Override
-    public String toString() {
-        return "TableInfo{" +
-                "tableName='" + tableName + '\'' +
-                ", columnFamily='" + columnFamily + '\'' +
-                '}';
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/9621c55e/metron-streaming/Metron-DataLoads/src/main/java/org/apache/metron/dataloads/taxii/TaxiiConnectionConfig.java
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-DataLoads/src/main/java/org/apache/metron/dataloads/taxii/TaxiiConnectionConfig.java b/metron-streaming/Metron-DataLoads/src/main/java/org/apache/metron/dataloads/taxii/TaxiiConnectionConfig.java
deleted file mode 100644
index dab8f0c..0000000
--- a/metron-streaming/Metron-DataLoads/src/main/java/org/apache/metron/dataloads/taxii/TaxiiConnectionConfig.java
+++ /dev/null
@@ -1,196 +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.dataloads.taxii;
-
-import org.codehaus.jackson.map.ObjectMapper;
-
-import java.io.ByteArrayInputStream;
-import java.io.IOException;
-import java.io.InputStream;
-import java.net.MalformedURLException;
-import java.net.URL;
-import java.nio.charset.Charset;
-import java.text.DateFormat;
-import java.text.ParseException;
-import java.text.SimpleDateFormat;
-import java.util.Date;
-import java.util.HashMap;
-import java.util.Map;
-
-public class TaxiiConnectionConfig {
-    final static ObjectMapper _mapper = new ObjectMapper();
-    private URL endpoint;
-    private int port = 443;
-    private URL proxy;
-    private String username;
-    private String password;
-    private ConnectionType type;
-    private String collection = "default";
-    private String subscriptionId = null;
-    private Date beginTime;
-    private Map<String, TableInfo> tableMap;
-    public TaxiiConnectionConfig withTableMap(Map<String, TableInfo> tableMap) {
-        this.tableMap = tableMap;
-        return this;
-    }
-    public TaxiiConnectionConfig withBeginTime(Date time) {
-        this.beginTime = time;
-        return this;
-    }
-    public TaxiiConnectionConfig withSubscriptionId(String subId) {
-        this.subscriptionId = subId;
-        return this;
-    }
-    public TaxiiConnectionConfig withCollection(String collection) {
-        this.collection = collection;
-        return this;
-    }
-
-    public TaxiiConnectionConfig withPort(int port) {
-        this.port = port;
-        return this;
-    }
-    public TaxiiConnectionConfig withEndpoint(URL endpoint) {
-        this.endpoint = endpoint;
-        return this;
-    }
-    public TaxiiConnectionConfig withProxy(URL proxy) {
-        this.proxy = proxy;
-        return this;
-    }
-    public TaxiiConnectionConfig withUsername(String username) {
-        this.username = username;
-        return this;
-    }
-    public TaxiiConnectionConfig withPassword(String password) {
-        this.password = password;
-        return this;
-    }
-    public TaxiiConnectionConfig withConnectionType(ConnectionType type) {
-        this.type= type;
-        return this;
-    }
-
-    public void setEndpoint(String endpoint) throws MalformedURLException {
-        this.endpoint = new URL(endpoint);
-    }
-
-    public void setPort(int port) {
-        this.port = port;
-    }
-
-    public void setProxy(String proxy) throws MalformedURLException {
-        this.proxy = new URL(proxy);
-    }
-
-    public void setUsername(String username) {
-        this.username = username;
-    }
-
-    public void setPassword(String password) {
-        this.password = password;
-    }
-
-    public void setType(ConnectionType type) {
-        this.type = type;
-    }
-
-    public void setCollection(String collection) {
-        this.collection = collection;
-    }
-
-    public void setSubscriptionId(String subscriptionId) {
-        this.subscriptionId = subscriptionId;
-    }
-
-    public void setBeginTime(String beginTime) throws ParseException {
-        SimpleDateFormat sdf = (SimpleDateFormat)DateFormat.getDateInstance(DateFormat.MEDIUM);
-        this.beginTime = sdf.parse(beginTime);
-    }
-
-    public void setTableMap(Map<String, String> tableMap) {
-        this.tableMap = new HashMap<>();
-        for(Map.Entry<String, String> kv : tableMap.entrySet()) {
-            this.tableMap.put(kv.getKey(), new TableInfo(kv.getValue()));
-        }
-    }
-
-    public Map<String, TableInfo> getTableMap() {
-        return tableMap;
-    }
-
-    public Date getBeginTime() {
-        return beginTime;
-    }
-    public int getPort() {
-        return port;
-    }
-    public URL getEndpoint() {
-        return endpoint;
-    }
-
-    public URL getProxy() {
-        return proxy;
-    }
-
-    public String getUsername() {
-        return username;
-    }
-
-    public String getPassword() {
-        return password;
-    }
-
-    public ConnectionType getType() {
-        return type;
-    }
-
-    public String getCollection() {
-        return collection;
-    }
-    public String getSubscriptionId() {
-        return subscriptionId;
-    }
-    public static synchronized TaxiiConnectionConfig load(InputStream is) throws IOException {
-        TaxiiConnectionConfig ret = _mapper.readValue(is, TaxiiConnectionConfig.class);
-        return ret;
-    }
-    public static synchronized TaxiiConnectionConfig load(String s, Charset c) throws IOException {
-        return load( new ByteArrayInputStream(s.getBytes(c)));
-    }
-    public static synchronized TaxiiConnectionConfig load(String s) throws IOException {
-        return load( s, Charset.defaultCharset());
-    }
-
-    @Override
-    public String toString() {
-        return "TaxiiConnectionConfig{" +
-                "endpoint=" + endpoint +
-                ", port=" + port +
-                ", proxy=" + proxy +
-                ", username='" + username + '\'' +
-                ", password=" + (password == null?"null" : "'******'") +
-                ", type=" + type +
-                ", collection='" + collection + '\'' +
-                ", subscriptionId='" + subscriptionId + '\'' +
-                ", beginTime=" + beginTime +
-                ", tableMap=" + tableMap +
-                '}';
-    }
-}


[5/6] incubator-metron git commit: METRON-93: Generalize the HBase threat intel infrastructure to support enrichments closes apache/incubator-metron#64

Posted by ce...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/9621c55e/metron-streaming/Metron-Common/src/main/java/org/apache/metron/utils/ConfigurationsUtils.java
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-Common/src/main/java/org/apache/metron/utils/ConfigurationsUtils.java b/metron-streaming/Metron-Common/src/main/java/org/apache/metron/utils/ConfigurationsUtils.java
index 20026b2..45e93a1 100644
--- a/metron-streaming/Metron-Common/src/main/java/org/apache/metron/utils/ConfigurationsUtils.java
+++ b/metron-streaming/Metron-Common/src/main/java/org/apache/metron/utils/ConfigurationsUtils.java
@@ -31,6 +31,7 @@ import org.apache.curator.framework.CuratorFrameworkFactory;
 import org.apache.curator.retry.ExponentialBackoffRetry;
 import org.apache.metron.Constants;
 import org.apache.metron.domain.Configurations;
+import org.apache.metron.domain.SensorEnrichmentConfig;
 import org.apache.zookeeper.KeeperException;
 
 import java.io.File;
@@ -52,23 +53,40 @@ public class ConfigurationsUtils {
     writeToZookeeper(path, Files.readAllBytes(Paths.get(filePath)), zookeeperUrl);
   }
 
+  public static void writeToZookeeperFromFile(String path, String filePath, CuratorFramework client) throws Exception {
+    writeToZookeeper(path, Files.readAllBytes(Paths.get(filePath)), client);
+  }
   public static void writerGlobalConfigToZookeeper(byte[] configData, String zookeeperUrl) throws Exception {
     writeToZookeeper(Constants.ZOOKEEPER_GLOBAL_ROOT, configData, zookeeperUrl);
   }
 
+  public static void writerGlobalConfigToZookeeper(byte[] configData, CuratorFramework client) throws Exception {
+    writeToZookeeper(Constants.ZOOKEEPER_GLOBAL_ROOT, configData, client);
+  }
   public static void writeSensorEnrichmentConfigToZookeeper(String sensorType, byte[] configData, String zookeeperUrl) throws Exception {
     writeToZookeeper(Constants.ZOOKEEPER_SENSOR_ROOT + "/" + sensorType, configData, zookeeperUrl);
   }
 
+  public static void writeSensorEnrichmentConfigToZookeeper(String sensorType, byte[] configData, CuratorFramework client) throws Exception {
+    writeToZookeeper(Constants.ZOOKEEPER_SENSOR_ROOT + "/" + sensorType, configData, client);
+  }
   public static void writeToZookeeper(String path, byte[] configData, String zookeeperUrl) throws Exception {
     CuratorFramework client = getClient(zookeeperUrl);
     client.start();
     try {
+      writeToZookeeper(path, configData, 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);
     }
-    client.close();
   }
 
   public static void updateConfigsFromZookeeper(Configurations configurations, CuratorFramework client) throws Exception {
@@ -114,12 +132,21 @@ public class ConfigurationsUtils {
   public static void dumpConfigs(String zookeeperUrl) throws Exception {
     CuratorFramework client = getClient(zookeeperUrl);
     client.start();
-    List<String> children = client.getChildren().forPath(Constants.ZOOKEEPER_TOPOLOGY_ROOT);
-    for (String child : children) {
-      byte[] data = client.getData().forPath(Constants.ZOOKEEPER_TOPOLOGY_ROOT + "/" + child);
-      System.out.println("Config for source " + child);
-      System.out.println(new String(data));
-      System.out.println();
+    //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();
   }

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/9621c55e/metron-streaming/Metron-Common/src/main/java/org/apache/metron/utils/JSONUtils.java
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-Common/src/main/java/org/apache/metron/utils/JSONUtils.java b/metron-streaming/Metron-Common/src/main/java/org/apache/metron/utils/JSONUtils.java
index cffcd68..34f98bb 100644
--- a/metron-streaming/Metron-Common/src/main/java/org/apache/metron/utils/JSONUtils.java
+++ b/metron-streaming/Metron-Common/src/main/java/org/apache/metron/utils/JSONUtils.java
@@ -22,8 +22,7 @@ import com.fasterxml.jackson.core.JsonProcessingException;
 import com.fasterxml.jackson.core.type.TypeReference;
 import com.fasterxml.jackson.databind.ObjectMapper;
 
-import java.io.IOException;
-import java.io.InputStream;
+import java.io.*;
 
 public enum JSONUtils {
   INSTANCE;
@@ -58,11 +57,16 @@ public enum JSONUtils {
   public <T> T load(String is, TypeReference<T> ref) throws IOException {
     return _mapper.get().readValue(is, ref);
   }
-
+  public <T> T load(File f, TypeReference<T> ref) throws IOException {
+    return _mapper.get().readValue(new BufferedInputStream(new FileInputStream(f)), ref);
+  }
   public <T> T load(InputStream is, Class<T> clazz) throws IOException {
     return _mapper.get().readValue(is, clazz);
   }
 
+  public <T> T load(File f, Class<T> clazz) throws IOException {
+    return _mapper.get().readValue(new BufferedInputStream(new FileInputStream(f)), clazz);
+  }
   public <T> T load(String is, Class<T> clazz) throws IOException {
     return _mapper.get().readValue(is, clazz);
   }

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/9621c55e/metron-streaming/Metron-Common/src/test/java/org/apache/metron/enrichment/EnrichmentConfigTest.java
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-Common/src/test/java/org/apache/metron/enrichment/EnrichmentConfigTest.java b/metron-streaming/Metron-Common/src/test/java/org/apache/metron/enrichment/EnrichmentConfigTest.java
new file mode 100644
index 0000000..1f29403
--- /dev/null
+++ b/metron-streaming/Metron-Common/src/test/java/org/apache/metron/enrichment/EnrichmentConfigTest.java
@@ -0,0 +1,197 @@
+package org.apache.metron.enrichment;
+
+import org.adrianwalker.multilinestring.Multiline;
+import org.apache.metron.Constants;
+import org.apache.metron.domain.SensorEnrichmentConfig;
+import org.apache.metron.utils.JSONUtils;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.List;
+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/9621c55e/metron-streaming/Metron-Common/src/test/java/org/apache/metron/hbase/converters/enrichment/EnrichmentConverterTest.java
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-Common/src/test/java/org/apache/metron/hbase/converters/enrichment/EnrichmentConverterTest.java b/metron-streaming/Metron-Common/src/test/java/org/apache/metron/hbase/converters/enrichment/EnrichmentConverterTest.java
new file mode 100644
index 0000000..9327b68
--- /dev/null
+++ b/metron-streaming/Metron-Common/src/test/java/org/apache/metron/hbase/converters/enrichment/EnrichmentConverterTest.java
@@ -0,0 +1,34 @@
+package org.apache.metron.hbase.converters.enrichment;
+
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.metron.reference.lookup.LookupKV;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.util.HashMap;
+
+public class EnrichmentConverterTest {
+  @Test
+  public void testKeyConversion() {
+    EnrichmentKey k1 = new EnrichmentKey("type", "indicator1");
+    byte[] serialized = k1.toBytes();
+    EnrichmentKey k2 = new EnrichmentKey();
+    k2.fromBytes(serialized);
+    Assert.assertEquals(k1, k2);
+  }
+
+  @Test
+  public void testValueConversion() throws IOException {
+    EnrichmentConverter converter = new EnrichmentConverter();
+    EnrichmentKey k1 = new EnrichmentKey("type", "indicator");
+    EnrichmentValue v1 = new EnrichmentValue(new HashMap<String, String>() {{
+      put("k1", "v1");
+      put("k2", "v2");
+    }});
+    Put serialized = converter.toPut("cf", k1, v1);
+    LookupKV<EnrichmentKey, EnrichmentValue> kv = converter.fromPut(serialized,"cf");
+    Assert.assertEquals(k1, kv.getKey());
+    Assert.assertEquals(v1, kv.getValue());
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/9621c55e/metron-streaming/Metron-DataLoads/README.md
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-DataLoads/README.md b/metron-streaming/Metron-DataLoads/README.md
index b6fe229..4ad5d5f 100644
--- a/metron-streaming/Metron-DataLoads/README.md
+++ b/metron-streaming/Metron-DataLoads/README.md
@@ -1,50 +1,252 @@
 # Metron-DataLoads
 
-This project is a collection of classes to assist with loading of various enrichment sources into Metron.
+This project is a collection of classes to assist with loading of
+various enrichment and threat intelligence sources into Metron.
+
+## Simple HBase Enrichments/Threat Intelligence
+
+The vast majority of enrichments and threat intelligence processing tend
+toward the following pattern:
+* Take a field
+* Look up the field in a key/value store
+* If the key exists, then either it's a threat to be alerted or it should be enriched with the value associated with the key.
+
+As such, we have created this capability as a default threat intel and enrichment adapter.  The basic primitive for simple enrichments and threat intelligence sources
+is a complex key containing the following:
+* Type : The type of threat intel or enrichment (e.g. malicious_ip)
+* Indicator : The indicator in question
+* Value : The value to associate with the type, indicator pair.  This is a JSON map.
+
+At present, all of the dataloads utilities function by converting raw data
+sources to this primitive key (type, indicator) and value to be placed in HBase.
+
+In the case of threat intel, a hit on the threat intel table will result
+in:
+* The `is_alert` field being set to `true` in the index
+* A field named `threatintels.hbaseThreatIntel.$field.$threatintel_type` is set to `alert` 
+   * `$field` is the field in the original document that was a match (e.g. `src_ip_addr`) 
+   * `$threatintel_type` is the type of threat intel imported (defined in the Extractor configuration below).
+
+In the case of simple hbase enrichment, a hit on the enrichments table
+will result in the following new field for each key in the value:`enrichments.hbaseEnrichment.$field.$enrichment_type.$key` 
+* `$field` is the field in the original document that was a match (e.g.  `src_ip_addr`)
+* `$enrichment_type` is the type of enrichment imported (defined in the Extractor configuration below).
+* `$key` is a key in the JSON map associated with the row in HBase.
+
+For instance, in the situation where we had the following very silly key/value in
+HBase in the enrichment table:
+* indicator: `127.0.0.1`
+* type : `important_addresses`
+* value: `{ "name" : "localhost", "location" : "home" }`
+
+If we had a document whose `ip_src_addr` came through with a value of
+`127.0.0.1`, we would have the following fields added to the indexed
+document:
+* `enrichments.hbaseEnrichment.ip_src_addr.important_addresses.name` : `localhost`
+* `enrichments.hbaseEnrichment.ip_src_addr.important_addresses.location` : `home`
+
+## Extractor Framework
+
+For the purpose of ingesting data of a variety of formats, we have
+created an Extractor framework which allows for common data formats to
+be interpreted as enrichment or threat intelligence sources.  The
+formats supported at present are:
+* CSV (both threat intel and enrichment)
+* STIX (threat intel only)
+* Custom (pass your own class)
+
+All of the current utilities take a JSON file to configure how to
+interpret input data.  This JSON describes the type of data and the
+schema if necessary for the data if it is not fixed (as in STIX, e.g.).
+
+### CSV Extractor
+
+Consider the following example configuration file which
+describes how to process a CSV file.
 
-## Threat Intel Enrichment
+````
+{
+  "config" : {
+    "columns" : {
+         "ip" : 0
+        ,"source" : 2
+    }
+    ,"indicator_column" : "ip"
+    ,"type" : "malicious_ip"
+    ,"separator" : ","
+  }
+  ,"extractor" : "CSV"
+}
+````
 
-Threat Intel enrichment data sources can be loaded into Metron using the ThreatIntelLoader class and an implementation of a ThreatIntelSource interface. Both are described below.
+In this example, we have instructed the extractor of the schema (i.e. the columns field), 
+two columns at the first and third position.  We have indicated that the `ip` column is the indicator type
+and that the enrichment type is named `malicious_ip`.  We have also indicated that the extractor to use is the CSV Extractor.
+The other option is the STIX extractor or a fully qualified classname for your own extractor.
 
-### ThreatIntelSource Interface
+The meta column values will show up in the value in HBase because it is called out as a non-indicator column.  The key
+for the value will be 'meta'.  For instance, given an input string of `123.45.123.12,something,the grapevine`, the following key, value
+would be extracted:
+* Indicator : `123.45.123.12`
+* Type : `malicious_ip`
+* Value : `{ "source" : "the grapevine" }`
 
-This inteface extends the Iterator interface and must implement the following methods:
+### STIX Extractor
 
-`void initializeSource(Configuration config);`
+Consider the following config for importing STIX documents.  This is a threat intelligence interchange
+format, so it is particularly relevant and attractive data to import for our purposes.  Because STIX is
+a standard format, there is no need to specify the schema or how to interpret the documents.
 
-Put any setup that needs to be done here. This will be called by ThreatIntelLoader before attempting to fetch any data from the source. The paramter config is a Configuration object created from the configuration file passed to ThreatIntelLoader. See the ThreatIntelLoader section below for more details
+We support a subset of STIX messages for importation:
 
-`void cleanupSource();`
+| STIX Type | Specific Type | Enrichment Type Name |
+|-----------|---------------|----------------------|
+| Address   | IPV_4_ADDR    | address:IPV_4_ADDR   |
+| Address   | IPV_6_ADDR    | address:IPV_6_ADDR   |
+| Address   | E_MAIL        | address:E_MAIL       |
+| Address   | MAC           | address:MAC          |
+| Domain    | FQDN          | domain:FQDN          |
+| Hostname  |               | hostname             |
 
-This is called after all data is retrieved, just before ThreatIntelLoader exists. Perform any clean up here if needed.
 
-`JSONObject next()`
+NOTE: The enrichment type will be used as the type above.
 
-This method should return the next piece of intel to be stored in Metron. The returned JSONObject must have the following fields:
+Consider the following configuration for an Extractor
 
-* indicator - The indicator that will be checked against during enrichment. For example, and IP Address or a Hostname.
-* source - The source of the data, which can be any unique string to identify the origin of the intel. This will be the column qualifer in HBase and be used to group matches on in Storm
-* data - A JSONArray of JSONObjects that detail the intel for the indicator. The JSONObjects have no required format
+````
+{
+  "config" : {
+    "stix_address_categories" : "IPV_4_ADDR"
+  }
+  ,"extractor" : "STIX"
+}
+````
 
+In here, we're configuring the STIX extractor to load from a series of STIX files, however we only want to bring in IPv4
+addresses from the set of all possible addresses.  Note that if no categories are specified for import, all are assumed.
+Also, only address and domain types allow filtering via `stix_address_categories` and `stix_domain_categories` config
+parameters.
 
-`boolean hasNext()`
+## Enrichment Config
 
-Returns true if there are more sources to read. Otherwise, false.
+In order to automatically add new enrichment and threat intel types to existing, running enrichment topologies, you will
+need to add new fields and new types to the zookeeper configuration.  A convenience parameter has been made to assist in this
+when doing an import.  Namely, you can specify the enrichment configs and how they associate with the fields of the 
+documents flowing through the enrichment topology.
 
+Consider the following Enrichment Configuration JSON.  This one is for a threat intelligence type:
+
+````
+{
+  "zkQuorum" : "localhost:2181"
+ ,"sensorToFieldList" : {
+    "bro" : {
+           "type" : "THREAT_INTEL"
+          ,"fieldToEnrichmentTypes" : {
+             "ip_src_addr" : [ "malicious_ip" ]
+            ,"ip_dst_addr" : [ "malicious_ip" ]
+                                      }
+           }
+                        }
+}
+````
 
-### ThreatIntelLoader
+We have to specify the following:
+* The zookeeper quorum which holds the cluster configuration
+* The mapping between the fields in the enriched documents and the enrichment types.
 
-This class is intenteded to be called from the commandline on the Metron cluster and is responsible for taking intel from a ThreatIntelSource implementation and putting them into HBase.
+This configuration allows the ingestion tools to update zookeeper post-ingestion so that the enrichment topology can take advantage
+immediately of the new type.
 
-#### Usage
+
+## Loading Utilities
+
+The two configurations above are used in the three separate ingestion tools:
+* Taxii Loader
+* Bulk load from HDFS via MapReduce
+* Flat File ingestion
+
+### Taxii Loader
+
+The shell script `$METRON_HOME/bin/threatintel_taxii_load.sh` can be used to poll a Taxii server for STIX documents and ingest them into HBase.  
+It is quite common for this Taxii server to be an aggregation server such as Soltra Edge.
+
+In addition to the Enrichment and Extractor configs described above, this loader requires a configuration file describing the connection information
+to the Taxii server.  An illustrative example of such a configuration file is:
 
 ````
-usage: ThreatIntelLoader [--configFile <c>] --source <s> --table <t>
-    --configFile <c>   Configuration file for source class
-    --source <s>       Source class to use
-    --table <t>        HBase table to load into
+{
+   "endpoint" : "http://localhost:8282/taxii-discovery-service"
+  ,"type" : "DISCOVER"
+  ,"collection" : "guest.Abuse_ch"
+  ,"table" : "threat_intel"
+  ,"columnFamily" : "cf"
+  ,"allowedIndicatorTypes" : [ "domainname:FQDN", "address:IPV_4_ADDR" ]
+}
 ````
 
-* configFile - the file passed in by this class is used to provide configuration options to the ThreatIntelSource implementation being used.
-* source - the implementation of ThreatIntelSource to use
-* table - the hbase table to store the threat intel in for enrichment later. This should match what the corresponding enrichment bolt is using in Storm
+As you can see, we are specifying the following information:
+* endpoint : The URL of the endpoint
+* type : `POLL` or `DISCOVER` depending on the endpoint.
+* collection : The Taxii collection to ingest
+* table : The HBase table to import into
+* columnFamily : The column family to import into
+* allowedIndicatorTypes : an array of acceptable threat intel types (see the "Enrichment Type Name" column of the Stix table above for the possibilities).
+
+The parameters for the utility are as follows:
+
+| Short Code | Long Code                 | Is Required? | Description                                                                                                                                        |
+|------------|---------------------------|--------------|----------------------------------------------------------------------------------------------------------------------------------------------------|
+| -h         |                           | No           | Generate the help screen/set of options                                                                                                            |
+| -e         | --extractor_config        | Yes          | JSON Document describing the extractor for this input data source                                                                                  |
+| -c         | --taxii_connection_config | Yes          | The JSON config file to configure the connection                                                                                                   |
+| -p         | --time_between_polls      | No           | The time between polling the Taxii server in milliseconds. (default: 1 hour)                                                                       |
+| -b         | --begin_time              | No           | Start time to poll the Taxii server (all data from that point will be gathered in the first pull).  The format for the date is yyyy-MM-dd HH:mm:ss |
+| -l         | --log4j                   | No           | The Log4j Properties to load                                                                                                                       |
+| -n         | --enrichment_config       | No           | The JSON document describing the enrichments to configure.  Unlike other loaders, this is run first if specified.                                  |
+
+
+### Bulk Load from HDFS
+
+The shell script `$METRON_HOME/bin/threatintel_bulk_load.sh` will kick off a MR job to load data staged in HDFS into an HBase table.  Note: despite what
+the naming may suggest, this utility works for enrichment as well as threat intel due to the underlying infrastructure being the same.
+
+The parameters for the utility are as follows:
+
+| Short Code | Long Code           | Is Required? | Description                                                                                                       |
+|------------|---------------------|--------------|-------------------------------------------------------------------------------------------------------------------|
+| -h         |                     | No           | Generate the help screen/set of options                                                                           |
+| -e         | --extractor_config  | Yes          | JSON Document describing the extractor for this input data source                                                 |
+| -t         | --table             | Yes          | The HBase table to import into                                                                                    |
+| -f         | --column_family     | Yes          | The HBase table column family to import into                                                                      |
+| -i         | --input             | Yes          | The input data location on HDFS                                                                                   |
+| -n         | --enrichment_config | No           | The JSON document describing the enrichments to configure.  Unlike other loaders, this is run first if specified. |
+or threat intel.
+
+### Flatfile Loader
+
+The shell script `$METRON_HOME/bin/flatfile_loader.sh` will read data from local disk and load the enrichment or threat intel data into an HBase table.  
+Note: This utility works for enrichment as well as threat intel due to the underlying infrastructure being the same.
+
+One special thing to note here is that there is a special configuration
+parameter to the Extractor config that is only considered during this
+loader:
+* inputFormatHandler : This specifies how to consider the data.  The two implementations are `BY_LINE` and `org.apache.metron.dataloads.extractor.inputformat.WholeFileFormat`.
+
+The default is `BY_LINE`, which makes sense for a list of CSVs where
+each line indicates a unit of information which can be imported.
+However, if you are importing a set of STIX documents, then you want
+each document to be considered as input to the Extractor.
+
+The parameters for the utility are as follows:
+
+| Short Code | Long Code           | Is Required? | Description                                                                                                                                                                          |
+|------------|---------------------|--------------|--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|
+| -h         |                     | No           | Generate the help screen/set of options                                                                                                                                              |
+| -e         | --extractor_config  | Yes          | JSON Document describing the extractor for this input data source                                                                                                                    |
+| -t         | --hbase_table       | Yes          | The HBase table to import into                                                                                                                                                       |
+| -c         | --hbase_cf          | Yes          | The HBase table column family to import into                                                                                                                                         |
+| -i         | --input             | Yes          | The input data location on local disk.  If this is a file, then that file will be loaded.  If this is a directory, then the files will be loaded recursively under that directory. |
+| -l         | --log4j             | No           | The log4j properties file to load                                                                                                                                                    |
+| -n         | --enrichment_config | No           | The JSON document describing the enrichments to configure.  Unlike other loaders, this is run first if specified.                                                                    |
+

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/9621c55e/metron-streaming/Metron-DataLoads/src/main/bash/flatfile_loader.sh
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-DataLoads/src/main/bash/flatfile_loader.sh b/metron-streaming/Metron-DataLoads/src/main/bash/flatfile_loader.sh
new file mode 100755
index 0000000..a9db639
--- /dev/null
+++ b/metron-streaming/Metron-DataLoads/src/main/bash/flatfile_loader.sh
@@ -0,0 +1,39 @@
+#!/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-DataLoads-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
+hadoop jar /usr/metron/0.1BETA/lib/Metron-DataLoads-0.1BETA.jar org.apache.metron.dataloads.nonbulk.flatfile.SimpleEnrichmentFlatFileLoader "$@"

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/9621c55e/metron-streaming/Metron-DataLoads/src/main/bash/threatintel_taxii_load.sh
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-DataLoads/src/main/bash/threatintel_taxii_load.sh b/metron-streaming/Metron-DataLoads/src/main/bash/threatintel_taxii_load.sh
index 77011fb..59e70d8 100755
--- a/metron-streaming/Metron-DataLoads/src/main/bash/threatintel_taxii_load.sh
+++ b/metron-streaming/Metron-DataLoads/src/main/bash/threatintel_taxii_load.sh
@@ -36,4 +36,4 @@ for jar in $(echo $HADOOP_CLASSPATH | sed 's/:/ /g');do
   fi
 done
 export HADOOP_CLASSPATH
-hadoop jar /usr/metron/0.1BETA/lib/Metron-DataLoads-0.1BETA.jar org.apache.metron.dataloads.taxii.TaxiiLoader "$@"
+hadoop jar /usr/metron/0.1BETA/lib/Metron-DataLoads-0.1BETA.jar org.apache.metron.dataloads.nonbulk.taxii.TaxiiLoader "$@"

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/9621c55e/metron-streaming/Metron-DataLoads/src/main/java/org/apache/metron/dataloads/ThreatIntelLoader.java
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-DataLoads/src/main/java/org/apache/metron/dataloads/ThreatIntelLoader.java b/metron-streaming/Metron-DataLoads/src/main/java/org/apache/metron/dataloads/ThreatIntelLoader.java
deleted file mode 100644
index 8ed2e23..0000000
--- a/metron-streaming/Metron-DataLoads/src/main/java/org/apache/metron/dataloads/ThreatIntelLoader.java
+++ /dev/null
@@ -1,195 +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.dataloads;
-
-
-import java.io.File;
-import java.io.IOException;
-import java.io.InterruptedIOException;
-import java.util.ArrayList;
-
-import org.apache.commons.cli.CommandLineParser;
-import org.apache.commons.cli.CommandLine;
-import org.apache.commons.cli.BasicParser;
-import org.apache.commons.cli.HelpFormatter;
-import org.apache.commons.cli.Options;
-import org.apache.commons.cli.OptionBuilder;
-import org.apache.commons.configuration.PropertiesConfiguration;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.HBaseConfiguration;
-import org.apache.hadoop.hbase.client.HTable;
-import org.apache.hadoop.hbase.client.Put;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.log4j.Logger;
-import org.json.simple.JSONObject;
-import org.json.simple.JSONArray;
-
-import org.apache.metron.dataloads.interfaces.ThreatIntelSource;
-
-public class ThreatIntelLoader {
-
-	
-	private static final Logger LOG = Logger.getLogger(ThreatIntelLoader.class);
-	
-	private static int BULK_SIZE = 50; 
-	
-	public static void main(String[] args) {
-		
-		PropertiesConfiguration sourceConfig = null;
-		ThreatIntelSource threatIntelSource = null;
-		ArrayList<Put> putList = null;
-		HTable table = null;
-		Configuration hConf = null;
-		
-		CommandLine commandLine = parseCommandLine(args);
-		File configFile = new File(commandLine.getOptionValue("configFile"));
-		
-		try {
-			sourceConfig = new PropertiesConfiguration(configFile);
-		} catch (org.apache.commons.configuration.ConfigurationException e) {
-			LOG.error("Error in configuration file " + configFile);
-			LOG.error(e);
-			System.exit(-1);
-		}
-		
-		try {
-			threatIntelSource = (ThreatIntelSource) Class.forName(commandLine.getOptionValue("source")).newInstance();
-			threatIntelSource.initializeSource(sourceConfig);
-		} catch (ClassNotFoundException|InstantiationException|IllegalAccessException e) {
-			LOG.error("Error while trying to load class " + commandLine.getOptionValue("source"));
-			LOG.error(e);
-			System.exit(-1);
-		}
-		
-		hConf = HBaseConfiguration.create();
-		try {
-			table = new HTable(hConf, commandLine.getOptionValue("table"));
-		} catch (IOException e) {
-			LOG.error("Exception when processing HBase config");
-			LOG.error(e);
-			System.exit(-1);
-		}
-		
-		
-		putList = new ArrayList<Put>();
-		
-		while (threatIntelSource.hasNext()) {
-			
-			JSONObject intel = threatIntelSource.next();
-			
-			/*
-			 * If any of the required fields from threatIntelSource are
-			 * missing, or contain invalid data, don't put it in HBase.
-			 */
-			try {				
-
-				putList.add(putRequestFromIntel(intel));		
-				
-				if (putList.size() == BULK_SIZE) {
-					table.put(putList);
-					putList.clear();
-				}
-				
-			} catch (NullPointerException|ClassCastException e) {
-				LOG.error("Exception while processing intel object");
-				LOG.error(intel.toString());
-				LOG.error(e);
-			} catch (InterruptedIOException|org.apache.hadoop.hbase.client.RetriesExhaustedWithDetailsException e) {
-				LOG.error("Problem communicationg with HBase");
-				LOG.error(e);
-				System.exit(-1);
-			} catch (IOException e) {
-				LOG.error("Problem communicationg with HBase");
-				LOG.error(e);
-				System.exit(-1);
-			}
-		}
-		
-	}
-	/*
-	 * Takes a JSONObject from a ThreatIntelSource implementation, ensures
-	 * that the format of the returned JSONObect is correct, and returns
-	 * a Put request for HBase.
-	 * 
-	 * @param	intel	The JSONObject from a ThreatIntelSource
-	 * @return			A put request for the intel data 
-	 * @throws	NullPointerException If a required field is missing
-	 * @throws	ClassCastException If a field has an invalid type
-	 * 
-	 */
-	private static Put putRequestFromIntel(JSONObject intel) {
-		
-		Put tempPut = new Put(Bytes.toBytes((String) intel.get("indicator")));
-		
-		JSONArray intelArray = (JSONArray) intel.get("data");
-		
-		tempPut.add(Bytes.toBytes("source"),
-					Bytes.toBytes((String) intel.get("source")),
-					Bytes.toBytes(intelArray.toString()));
-		
-		return tempPut;
-	}
-	/*
-	 * Handles parsing of command line options and validates the options are used
-	 * correctly. This will not validate the value of the options, it will just
-	 * ensure that the required options are used. If the options are used 
-	 * incorrectly, the help is printed, and the program exits.
-	 * 
-	 * @param  args The arguments from the CLI
-	 * @return 		A CommandLine with the CLI arguments
-	 * 
-	 */
-	private static CommandLine parseCommandLine(String[] args) {
-		
-		CommandLineParser parser = new BasicParser();
-		CommandLine cli = null;
-		
-		Options options = new Options();
-		
-		options.addOption(OptionBuilder.withArgName("s").
-				withLongOpt("source").
-				isRequired(true).
-				hasArg(true).
-				withDescription("Source class to use").
-				create()
-				);
-		options.addOption(OptionBuilder.withArgName("t").
-				withLongOpt("table").
-				isRequired(true).
-				hasArg(true).
-				withDescription("HBase table to load into").
-				create()
-				);
-		options.addOption(OptionBuilder.withArgName("c").
-				withLongOpt("configFile").
-				hasArg(true).
-				withDescription("Configuration file for source class").
-				create()
-				);
-		
-		try {
-			cli = parser.parse(options, args);
-		} catch(org.apache.commons.cli.ParseException e) {
-			HelpFormatter formatter = new HelpFormatter();
-			formatter.printHelp("ThreatIntelLoader", options, true);
-			System.exit(-1);
-		}
-		
-		return cli;
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/9621c55e/metron-streaming/Metron-DataLoads/src/main/java/org/apache/metron/dataloads/bulk/LeastRecentlyUsedPruner.java
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-DataLoads/src/main/java/org/apache/metron/dataloads/bulk/LeastRecentlyUsedPruner.java b/metron-streaming/Metron-DataLoads/src/main/java/org/apache/metron/dataloads/bulk/LeastRecentlyUsedPruner.java
index 8f1f205..7acc96c 100644
--- a/metron-streaming/Metron-DataLoads/src/main/java/org/apache/metron/dataloads/bulk/LeastRecentlyUsedPruner.java
+++ b/metron-streaming/Metron-DataLoads/src/main/java/org/apache/metron/dataloads/bulk/LeastRecentlyUsedPruner.java
@@ -150,7 +150,7 @@ public class LeastRecentlyUsedPruner {
 
         public static void printHelp() {
             HelpFormatter formatter = new HelpFormatter();
-            formatter.printHelp( "ThreatIntelBulkLoader", getOptions());
+            formatter.printHelp( "LeastRecentlyUsedPruner", getOptions());
         }
 
         public static Options getOptions() {

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/9621c55e/metron-streaming/Metron-DataLoads/src/main/java/org/apache/metron/dataloads/bulk/ThreatIntelBulkLoader.java
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-DataLoads/src/main/java/org/apache/metron/dataloads/bulk/ThreatIntelBulkLoader.java b/metron-streaming/Metron-DataLoads/src/main/java/org/apache/metron/dataloads/bulk/ThreatIntelBulkLoader.java
index 02f9f9b..8c0a7e4 100644
--- a/metron-streaming/Metron-DataLoads/src/main/java/org/apache/metron/dataloads/bulk/ThreatIntelBulkLoader.java
+++ b/metron-streaming/Metron-DataLoads/src/main/java/org/apache/metron/dataloads/bulk/ThreatIntelBulkLoader.java
@@ -32,8 +32,10 @@ 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.enrichment.EnrichmentConfig;
 import org.apache.metron.hbase.converters.HbaseConverter;
-import org.apache.metron.hbase.converters.threatintel.ThreatIntelConverter;
+import org.apache.metron.hbase.converters.enrichment.EnrichmentConverter;
+import org.apache.metron.utils.JSONUtils;
 
 import javax.annotation.Nullable;
 import java.io.File;
@@ -43,188 +45,215 @@ import java.text.*;
 import java.util.Date;
 
 public class ThreatIntelBulkLoader  {
-    private static abstract class OptionHandler implements Function<String, Option> {}
-    private enum BulkLoadOptions {
-        HELP("h", new OptionHandler() {
+  private static abstract class OptionHandler implements Function<String, Option> {}
+  private enum BulkLoadOptions {
+    HELP("h", new OptionHandler() {
 
-            @Nullable
-            @Override
-            public Option apply(@Nullable String s) {
-                return new Option(s, "help", false, "Generate Help screen");
-            }
-        })
-        ,TABLE("t", new OptionHandler() {
-            @Nullable
-            @Override
-            public Option apply(@Nullable String s) {
-                Option o = new Option(s, "table", true, "HBase table to import data into");
-                o.setRequired(true);
-                o.setArgName("HBASE_TABLE");
-                return o;
-            }
-        })
-        ,COLUMN_FAMILY("f", new OptionHandler() {
-            @Nullable
-            @Override
-            public Option apply(@Nullable String s) {
-                Option o = new Option(s, "column_family", true, "Column family of the HBase table to import into");
-                o.setRequired(true);
-                o.setArgName("CF_NAME");
-                return o;
-            }
-        })
-        ,EXTRACTOR_CONFIG("e", new OptionHandler() {
-            @Nullable
-            @Override
-            public Option apply(@Nullable String s) {
-                Option o = new Option(s, "extractor_config", true, "JSON Document describing the extractor for this input data source");
-                o.setArgName("JSON_FILE");
-                o.setRequired(true);
-                return o;
-            }
-        })
-        ,INPUT_DATA("i", new OptionHandler() {
-            @Nullable
-            @Override
-            public Option apply(@Nullable String s) {
-                Option o = new Option(s, "input", true, "Input directory in HDFS for the data to import into HBase");
-                o.setArgName("DIR");
-                o.setRequired(true);
-                return o;
-            }
-        })
-        ,AS_OF_TIME("a", new OptionHandler() {
-            @Nullable
-            @Override
-            public Option apply(@Nullable String s) {
-                Option o = new Option(s, "as_of", true, "The last read timestamp to mark the records with (omit for time of execution)");
-                o.setArgName("datetime");
-                o.setRequired(false);
-                return o;
-            }
-        })
-        ,AS_OF_TIME_FORMAT("z", new OptionHandler() {
-            @Nullable
-            @Override
-            public Option apply(@Nullable String s) {
-                Option o = new Option(s, "as_of_format", true, "The format of the as_of time (only used in conjunction with the as_of option)");
-                o.setArgName("format");
-                o.setRequired(false);
-                return o;
-            }
-        })
-        ,CONVERTER("c", new OptionHandler() {
-            @Nullable
-            @Override
-            public Option apply(@Nullable String s) {
-                Option o = new Option(s, "converter", true, "The HBase converter class to use (Default is threat intel)");
-                o.setArgName("class");
-                o.setRequired(false);
-                return o;
-            }
-        })
-        ;
-        Option option;
-        String shortCode;
-        BulkLoadOptions(String shortCode, OptionHandler optionHandler) {
-            this.shortCode = shortCode;
-            this.option = optionHandler.apply(shortCode);
-        }
+      @Nullable
+      @Override
+      public Option apply(@Nullable String s) {
+        return new Option(s, "help", false, "Generate Help screen");
+      }
+    })
+    ,TABLE("t", new OptionHandler() {
+      @Nullable
+      @Override
+      public Option apply(@Nullable String s) {
+        Option o = new Option(s, "table", true, "HBase table to import data into");
+        o.setRequired(true);
+        o.setArgName("HBASE_TABLE");
+        return o;
+      }
+    })
+    ,COLUMN_FAMILY("f", new OptionHandler() {
+      @Nullable
+      @Override
+      public Option apply(@Nullable String s) {
+        Option o = new Option(s, "column_family", true, "Column family of the HBase table to import into");
+        o.setRequired(true);
+        o.setArgName("CF_NAME");
+        return o;
+      }
+    })
+    ,EXTRACTOR_CONFIG("e", new OptionHandler() {
+      @Nullable
+      @Override
+      public Option apply(@Nullable String s) {
+        Option o = new Option(s, "extractor_config", true, "JSON Document describing the extractor for this input data source");
+        o.setArgName("JSON_FILE");
+        o.setRequired(true);
+        return o;
+      }
+    })
+    ,INPUT_DATA("i", new OptionHandler() {
+      @Nullable
+      @Override
+      public Option apply(@Nullable String s) {
+        Option o = new Option(s, "input", true, "Input directory in HDFS for the data to import into HBase");
+        o.setArgName("DIR");
+        o.setRequired(true);
+        return o;
+      }
+    })
+    ,AS_OF_TIME("a", new OptionHandler() {
+      @Nullable
+      @Override
+      public Option apply(@Nullable String s) {
+        Option o = new Option(s, "as_of", true, "The last read timestamp to mark the records with (omit for time of execution)");
+        o.setArgName("datetime");
+        o.setRequired(false);
+        return o;
+      }
+    })
+    ,AS_OF_TIME_FORMAT("z", new OptionHandler() {
+      @Nullable
+      @Override
+      public Option apply(@Nullable String s) {
+        Option o = new Option(s, "as_of_format", true, "The format of the as_of time (only used in conjunction with the as_of option)");
+        o.setArgName("format");
+        o.setRequired(false);
+        return o;
+      }
+    })
+    ,CONVERTER("c", new OptionHandler() {
+      @Nullable
+      @Override
+      public Option apply(@Nullable String s) {
+        Option o = new Option(s, "converter", true, "The HBase converter class to use (Default is threat intel)");
+        o.setArgName("class");
+        o.setRequired(false);
+        return o;
+      }
+    })
+    ,ENRICHMENT_CONFIG("n", new OptionHandler() {
+      @Nullable
+      @Override
+      public Option apply(@Nullable String s) {
+        Option o = new Option(s, "enrichment_config", true
+                , "JSON Document describing the enrichment configuration details." +
+                "  This is used to associate an enrichment type with a field type in zookeeper."
+        );
+        o.setArgName("JSON_FILE");
+        o.setRequired(false);
+        return o;
+      }
+    })
+    ;
+    Option option;
+    String shortCode;
+    BulkLoadOptions(String shortCode, OptionHandler optionHandler) {
+      this.shortCode = shortCode;
+      this.option = optionHandler.apply(shortCode);
+    }
 
-        public boolean has(CommandLine cli) {
-            return cli.hasOption(shortCode);
-        }
+    public boolean has(CommandLine cli) {
+      return cli.hasOption(shortCode);
+    }
 
-        public String get(CommandLine cli) {
-            return cli.getOptionValue(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(ThreatIntelBulkLoader.BulkLoadOptions.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 CommandLine parse(CommandLineParser parser, String[] args) {
+      try {
+        CommandLine cli = parser.parse(getOptions(), args);
+        if(ThreatIntelBulkLoader.BulkLoadOptions.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( "ThreatIntelBulkLoader", getOptions());
-        }
+    public static void printHelp() {
+      HelpFormatter formatter = new HelpFormatter();
+      formatter.printHelp( "ThreatIntelBulkLoader", getOptions());
+    }
 
-        public static Options getOptions() {
-            Options ret = new Options();
-            for(BulkLoadOptions o : BulkLoadOptions.values()) {
-               ret.addOption(o.option);
-            }
-            return ret;
-        }
+    public static Options getOptions() {
+      Options ret = new Options();
+      for(BulkLoadOptions o : BulkLoadOptions.values()) {
+        ret.addOption(o.option);
+      }
+      return ret;
     }
-    private static long getTimestamp(CommandLine cli) throws java.text.ParseException {
-        if(BulkLoadOptions.AS_OF_TIME.has(cli)) {
-            if(!BulkLoadOptions.AS_OF_TIME_FORMAT.has(cli)) {
-                throw new IllegalStateException("Unable to proceed: Specified as_of_time without an associated format.");
-            }
-            else {
-                DateFormat format = new SimpleDateFormat(BulkLoadOptions.AS_OF_TIME_FORMAT.get(cli));
-                Date d = format.parse(BulkLoadOptions.AS_OF_TIME.get(cli));
-                return d.getTime();
-            }
-        }
-        else {
-            return System.currentTimeMillis();
-        }
+  }
+  private static long getTimestamp(CommandLine cli) throws java.text.ParseException {
+    if(BulkLoadOptions.AS_OF_TIME.has(cli)) {
+      if(!BulkLoadOptions.AS_OF_TIME_FORMAT.has(cli)) {
+        throw new IllegalStateException("Unable to proceed: Specified as_of_time without an associated format.");
+      }
+      else {
+        DateFormat format = new SimpleDateFormat(BulkLoadOptions.AS_OF_TIME_FORMAT.get(cli));
+        Date d = format.parse(BulkLoadOptions.AS_OF_TIME.get(cli));
+        return d.getTime();
+      }
     }
-    private static String readExtractorConfig(File configFile) throws IOException {
-        return Joiner.on("\n").join(Files.readLines(configFile, Charset.defaultCharset()));
+    else {
+      return System.currentTimeMillis();
     }
+  }
+  private static String readExtractorConfig(File configFile) throws IOException {
+    return Joiner.on("\n").join(Files.readLines(configFile, Charset.defaultCharset()));
+  }
 
-    public static Job createJob(Configuration conf, String input, String table, String cf, String extractorConfigContents, long ts, HbaseConverter converter) throws IOException {
-        Job job = new Job(conf);
-        job.setJobName("ThreatIntelBulkLoader: " + input + " => " +  table + ":" + cf);
-        System.out.println("Configuring " + job.getJobName());
-        job.setJarByClass(ThreatIntelBulkLoader.class);
-        job.setMapperClass(org.apache.metron.dataloads.hbase.mr.BulkLoadMapper.class);
-        job.setOutputFormatClass(TableOutputFormat.class);
-        job.getConfiguration().set(TableOutputFormat.OUTPUT_TABLE, table);
-        job.getConfiguration().set(BulkLoadMapper.COLUMN_FAMILY_KEY, cf);
-        job.getConfiguration().set(BulkLoadMapper.CONFIG_KEY, extractorConfigContents);
-        job.getConfiguration().set(BulkLoadMapper.LAST_SEEN_KEY, "" + ts);
-        job.getConfiguration().set(BulkLoadMapper.CONVERTER_KEY, converter.getClass().getName());
-        job.setOutputKeyClass(ImmutableBytesWritable.class);
-        job.setOutputValueClass(Put.class);
-        job.setNumReduceTasks(0);
-        ExtractorHandler handler = ExtractorHandler.load(extractorConfigContents);
-        handler.getInputFormatHandler().set(job, new Path(input), handler.getConfig());
-        return job;
-    }
+  public static Job createJob(Configuration conf, String input, String table, String cf, String extractorConfigContents, long ts, HbaseConverter converter) throws IOException {
+    Job job = new Job(conf);
+    job.setJobName("ThreatIntelBulkLoader: " + input + " => " +  table + ":" + cf);
+    System.out.println("Configuring " + job.getJobName());
+    job.setJarByClass(ThreatIntelBulkLoader.class);
+    job.setMapperClass(org.apache.metron.dataloads.hbase.mr.BulkLoadMapper.class);
+    job.setOutputFormatClass(TableOutputFormat.class);
+    job.getConfiguration().set(TableOutputFormat.OUTPUT_TABLE, table);
+    job.getConfiguration().set(BulkLoadMapper.COLUMN_FAMILY_KEY, cf);
+    job.getConfiguration().set(BulkLoadMapper.CONFIG_KEY, extractorConfigContents);
+    job.getConfiguration().set(BulkLoadMapper.LAST_SEEN_KEY, "" + ts);
+    job.getConfiguration().set(BulkLoadMapper.CONVERTER_KEY, converter.getClass().getName());
+    job.setOutputKeyClass(ImmutableBytesWritable.class);
+    job.setOutputValueClass(Put.class);
+    job.setNumReduceTasks(0);
+    ExtractorHandler handler = ExtractorHandler.load(extractorConfigContents);
+    handler.getInputFormatHandler().set(job, new Path(input), handler.getConfig());
+    return job;
+  }
 
-    public static void main(String... argv) throws IOException, java.text.ParseException, ClassNotFoundException, InterruptedException, IllegalAccessException, InstantiationException {
-        Configuration conf = HBaseConfiguration.create();
-        String[] otherArgs = new GenericOptionsParser(conf, argv).getRemainingArgs();
+  public static void main(String... argv) throws Exception {
+    Configuration conf = HBaseConfiguration.create();
+    String[] otherArgs = new GenericOptionsParser(conf, argv).getRemainingArgs();
 
-        CommandLine cli = BulkLoadOptions.parse(new PosixParser(), otherArgs);
-        Long ts = getTimestamp(cli);
-        String input = BulkLoadOptions.INPUT_DATA.get(cli);
-        String table = BulkLoadOptions.TABLE.get(cli);
-        String cf = BulkLoadOptions.COLUMN_FAMILY.get(cli);
-        String extractorConfigContents = readExtractorConfig(new File(BulkLoadOptions.EXTRACTOR_CONFIG.get(cli)));
-        String converterClass = ThreatIntelConverter.class.getName();
-        if(BulkLoadOptions.CONVERTER.has(cli)) {
-            converterClass = BulkLoadOptions.CONVERTER.get(cli);
-        }
-        HbaseConverter converter = (HbaseConverter) Class.forName(converterClass).newInstance();
-        Job job = createJob(conf, input, table, cf, extractorConfigContents, ts, converter);
-        System.out.println(conf);
-        System.exit(job.waitForCompletion(true) ? 0 : 1);
+    CommandLine cli = BulkLoadOptions.parse(new PosixParser(), otherArgs);
+    Long ts = getTimestamp(cli);
+    String input = BulkLoadOptions.INPUT_DATA.get(cli);
+    String table = BulkLoadOptions.TABLE.get(cli);
+    String cf = BulkLoadOptions.COLUMN_FAMILY.get(cli);
+    String extractorConfigContents = readExtractorConfig(new File(BulkLoadOptions.EXTRACTOR_CONFIG.get(cli)));
+    String converterClass = EnrichmentConverter.class.getName();
+    if(BulkLoadOptions.CONVERTER.has(cli)) {
+      converterClass = BulkLoadOptions.CONVERTER.get(cli);
+    }
+    EnrichmentConfig enrichmentConfig = null;
+    if(BulkLoadOptions.ENRICHMENT_CONFIG.has(cli)) {
+      enrichmentConfig = JSONUtils.INSTANCE.load( new File(BulkLoadOptions.ENRICHMENT_CONFIG.get(cli))
+              , EnrichmentConfig.class
+      );
+    }
+
+    HbaseConverter converter = (HbaseConverter) Class.forName(converterClass).newInstance();
+    Job job = createJob(conf, input, table, cf, extractorConfigContents, ts, converter);
+    System.out.println(conf);
+    boolean jobRet = job.waitForCompletion(true);
+    if(!jobRet) {
+      System.exit(1);
+    }
+    if(enrichmentConfig != null) {
+        enrichmentConfig.updateSensorConfigs();
     }
+    System.exit(0);
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/9621c55e/metron-streaming/Metron-DataLoads/src/main/java/org/apache/metron/dataloads/extractor/Extractor.java
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-DataLoads/src/main/java/org/apache/metron/dataloads/extractor/Extractor.java b/metron-streaming/Metron-DataLoads/src/main/java/org/apache/metron/dataloads/extractor/Extractor.java
index 30e56d8..119a68f 100644
--- a/metron-streaming/Metron-DataLoads/src/main/java/org/apache/metron/dataloads/extractor/Extractor.java
+++ b/metron-streaming/Metron-DataLoads/src/main/java/org/apache/metron/dataloads/extractor/Extractor.java
@@ -17,9 +17,9 @@
  */
 package org.apache.metron.dataloads.extractor;
 
-import org.apache.hadoop.hbase.client.Put;
 import org.apache.metron.reference.lookup.LookupKV;
-import org.apache.metron.threatintel.ThreatIntelResults;
+import org.apache.metron.reference.lookup.LookupKey;
+import org.apache.metron.reference.lookup.LookupValue;
 
 import java.io.IOException;
 import java.util.Map;

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/9621c55e/metron-streaming/Metron-DataLoads/src/main/java/org/apache/metron/dataloads/extractor/ExtractorHandler.java
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-DataLoads/src/main/java/org/apache/metron/dataloads/extractor/ExtractorHandler.java b/metron-streaming/Metron-DataLoads/src/main/java/org/apache/metron/dataloads/extractor/ExtractorHandler.java
index b9d270a..5d17cbe 100644
--- a/metron-streaming/Metron-DataLoads/src/main/java/org/apache/metron/dataloads/extractor/ExtractorHandler.java
+++ b/metron-streaming/Metron-DataLoads/src/main/java/org/apache/metron/dataloads/extractor/ExtractorHandler.java
@@ -49,11 +49,7 @@ public class ExtractorHandler {
     public void setInputFormatHandler(String handler) {
         try {
             this.inputFormatHandler= Formats.create(handler);
-        } catch (ClassNotFoundException e) {
-            throw new IllegalStateException("Unable to create an inputformathandler", e);
-        } catch (IllegalAccessException e) {
-            throw new IllegalStateException("Unable to create an inputformathandler", e);
-        } catch (InstantiationException e) {
+        } catch (ClassNotFoundException | InstantiationException | IllegalAccessException e) {
             throw new IllegalStateException("Unable to create an inputformathandler", e);
         }
     }
@@ -64,11 +60,7 @@ public class ExtractorHandler {
     public void setExtractor(String extractor) {
         try {
             this.extractor = Extractors.create(extractor);
-        } catch (ClassNotFoundException e) {
-            throw new IllegalStateException("Unable to create an extractor", e);
-        } catch (IllegalAccessException e) {
-            throw new IllegalStateException("Unable to create an extractor", e);
-        } catch (InstantiationException e) {
+        } catch (ClassNotFoundException | IllegalAccessException | InstantiationException e) {
             throw new IllegalStateException("Unable to create an extractor", e);
         }
     }

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/9621c55e/metron-streaming/Metron-DataLoads/src/main/java/org/apache/metron/dataloads/extractor/csv/CSVExtractor.java
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-DataLoads/src/main/java/org/apache/metron/dataloads/extractor/csv/CSVExtractor.java b/metron-streaming/Metron-DataLoads/src/main/java/org/apache/metron/dataloads/extractor/csv/CSVExtractor.java
index 94e9ccb..92b7242 100644
--- a/metron-streaming/Metron-DataLoads/src/main/java/org/apache/metron/dataloads/extractor/csv/CSVExtractor.java
+++ b/metron-streaming/Metron-DataLoads/src/main/java/org/apache/metron/dataloads/extractor/csv/CSVExtractor.java
@@ -21,103 +21,119 @@ import com.google.common.base.Splitter;
 import com.google.common.collect.Iterables;
 import com.opencsv.CSVParser;
 import com.opencsv.CSVParserBuilder;
-import org.apache.hadoop.hbase.client.Put;
 import org.apache.metron.dataloads.extractor.Extractor;
-import org.apache.metron.hbase.converters.HbaseConverter;
 import org.apache.metron.reference.lookup.LookupKV;
 import org.apache.metron.reference.lookup.LookupKey;
-import org.apache.metron.reference.lookup.LookupValue;
-import org.apache.metron.threatintel.ThreatIntelResults;
 
 import java.io.IOException;
 import java.util.*;
 
 public class CSVExtractor implements Extractor {
-    public static final String COLUMNS_KEY="columns";
-    public static final String INDICATOR_COLUMN_KEY="indicator_column";
-    public static final String SEPARATOR_KEY="separator";
-    public static final String LOOKUP_CONVERTER = "lookupConverter";
+  public static final String COLUMNS_KEY="columns";
+  public static final String INDICATOR_COLUMN_KEY="indicator_column";
+  public static final String TYPE_COLUMN_KEY="type_column";
+  public static final String TYPE_KEY="type";
+  public static final String SEPARATOR_KEY="separator";
+  public static final String LOOKUP_CONVERTER = "lookup_converter";
 
-    private int indicatorColumn;
-    private Map<String, Integer> columnMap = new HashMap<>();
-    private CSVParser parser;
-    private LookupConverter converter = LookupConverters.THREAT_INTEL.getConverter();
+  private int typeColumn;
+  private String type;
+  private int indicatorColumn;
+  private Map<String, Integer> columnMap = new HashMap<>();
+  private CSVParser parser;
+  private LookupConverter converter = LookupConverters.ENRICHMENT.getConverter();
 
-    @Override
-    public Iterable<LookupKV> extract(String line) throws IOException {
-        if(line.trim().startsWith("#")) {
-            //comment
-            return Collections.emptyList();
-        }
-        String[] tokens = parser.parseLine(line);
-        LookupKey key = converter.toKey(tokens[indicatorColumn]);
-        Map<String, String> values = new HashMap<>();
-        for(Map.Entry<String, Integer> kv : columnMap.entrySet()) {
-            values.put(kv.getKey(), tokens[kv.getValue()]);
-        }
-        return Arrays.asList(new LookupKV(key, converter.toValue(values)));
+  @Override
+  public Iterable<LookupKV> extract(String line) throws IOException {
+    if(line.trim().startsWith("#")) {
+      //comment
+      return Collections.emptyList();
     }
+    String[] tokens = parser.parseLine(line);
 
-    private static Map.Entry<String, Integer> getColumnMapEntry(String column, int i) {
-        if(column.contains(":")) {
-            Iterable<String> tokens = Splitter.on(':').split(column);
-            String col = Iterables.getFirst(tokens, null);
-            Integer pos = Integer.parseInt(Iterables.getLast(tokens));
-            return new AbstractMap.SimpleEntry<>(col, pos);
-        }
-        else {
-            return new AbstractMap.SimpleEntry<>(column, i);
-        }
+    LookupKey key = converter.toKey(getType(tokens), tokens[indicatorColumn]);
+    Map<String, String> values = new HashMap<>();
+    for(Map.Entry<String, Integer> kv : columnMap.entrySet()) {
+      values.put(kv.getKey(), tokens[kv.getValue()]);
+    }
+    return Arrays.asList(new LookupKV(key, converter.toValue(values)));
+  }
 
+  private String getType(String[] tokens) {
+    if(type == null) {
+      return tokens[typeColumn];
     }
-    private static Map<String, Integer> getColumnMap(Map<String, Object> config) {
-        Map<String, Integer> columnMap = new HashMap<>();
-        if(config.containsKey(COLUMNS_KEY)) {
-            Object columnsObj = config.get(COLUMNS_KEY);
-            if(columnsObj instanceof String) {
-                String columns = (String)columnsObj;
-                int i = 0;
-                for (String column : Splitter.on(',').split(columns)) {
-                    Map.Entry<String, Integer> e = getColumnMapEntry(column, i++);
-                    columnMap.put(e.getKey(), e.getValue());
-                }
-            }
-            else if(columnsObj instanceof List) {
-                List columns = (List)columnsObj;
-                int i = 0;
-                for(Object column : columns) {
-                    Map.Entry<String, Integer> e = getColumnMapEntry(column.toString(), i++);
-                    columnMap.put(e.getKey(), e.getValue());
-                }
-            }
-            else if(columnsObj instanceof Map) {
-                Map<Object, Object> map = (Map<Object, Object>)columnsObj;
-                for(Map.Entry<Object, Object> e : map.entrySet()) {
-                    columnMap.put(e.getKey().toString(), Integer.parseInt(e.getValue().toString()));
-                }
-            }
-        }
-        return columnMap;
+    else {
+      return type;
     }
+  }
 
-    @Override
-    public void initialize(Map<String, Object> config) {
-        if(config.containsKey(COLUMNS_KEY)) {
-            columnMap = getColumnMap(config);
-        }
-        else {
-            throw new IllegalStateException("CSVExtractor requires " + COLUMNS_KEY + " configuration");
-        }
-        if(config.containsKey(INDICATOR_COLUMN_KEY)) {
-            indicatorColumn = columnMap.get(config.get(INDICATOR_COLUMN_KEY).toString());
+  private static Map.Entry<String, Integer> getColumnMapEntry(String column, int i) {
+    if(column.contains(":")) {
+      Iterable<String> tokens = Splitter.on(':').split(column);
+      String col = Iterables.getFirst(tokens, null);
+      Integer pos = Integer.parseInt(Iterables.getLast(tokens));
+      return new AbstractMap.SimpleEntry<>(col, pos);
+    }
+    else {
+      return new AbstractMap.SimpleEntry<>(column, i);
+    }
+
+  }
+  private static Map<String, Integer> getColumnMap(Map<String, Object> config) {
+    Map<String, Integer> columnMap = new HashMap<>();
+    if(config.containsKey(COLUMNS_KEY)) {
+      Object columnsObj = config.get(COLUMNS_KEY);
+      if(columnsObj instanceof String) {
+        String columns = (String)columnsObj;
+        int i = 0;
+        for (String column : Splitter.on(',').split(columns)) {
+          Map.Entry<String, Integer> e = getColumnMapEntry(column, i++);
+          columnMap.put(e.getKey(), e.getValue());
         }
-        if(config.containsKey(SEPARATOR_KEY)) {
-            char separator = config.get(SEPARATOR_KEY).toString().charAt(0);
-            parser = new CSVParserBuilder().withSeparator(separator)
-                                           .build();
+      }
+      else if(columnsObj instanceof List) {
+        List columns = (List)columnsObj;
+        int i = 0;
+        for(Object column : columns) {
+          Map.Entry<String, Integer> e = getColumnMapEntry(column.toString(), i++);
+          columnMap.put(e.getKey(), e.getValue());
         }
-        if(config.containsKey(LOOKUP_CONVERTER)) {
-           converter = LookupConverters.getConverter((String) config.get(LOOKUP_CONVERTER));
+      }
+      else if(columnsObj instanceof Map) {
+        Map<Object, Object> map = (Map<Object, Object>)columnsObj;
+        for(Map.Entry<Object, Object> e : map.entrySet()) {
+          columnMap.put(e.getKey().toString(), Integer.parseInt(e.getValue().toString()));
         }
+      }
+    }
+    return columnMap;
+  }
+
+  @Override
+  public void initialize(Map<String, Object> config) {
+    if(config.containsKey(COLUMNS_KEY)) {
+      columnMap = getColumnMap(config);
+    }
+    else {
+      throw new IllegalStateException("CSVExtractor requires " + COLUMNS_KEY + " configuration");
+    }
+    if(config.containsKey(INDICATOR_COLUMN_KEY)) {
+      indicatorColumn = columnMap.get(config.get(INDICATOR_COLUMN_KEY).toString());
+    }
+    if(config.containsKey(TYPE_KEY)) {
+      type = config.get(TYPE_KEY).toString();
+    }
+    else if(config.containsKey(TYPE_COLUMN_KEY)) {
+      typeColumn = columnMap.get(config.get(TYPE_COLUMN_KEY).toString());
+    }
+    if(config.containsKey(SEPARATOR_KEY)) {
+      char separator = config.get(SEPARATOR_KEY).toString().charAt(0);
+      parser = new CSVParserBuilder().withSeparator(separator)
+              .build();
+    }
+    if(config.containsKey(LOOKUP_CONVERTER)) {
+      converter = LookupConverters.getConverter((String) config.get(LOOKUP_CONVERTER));
     }
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/9621c55e/metron-streaming/Metron-DataLoads/src/main/java/org/apache/metron/dataloads/extractor/csv/LookupConverter.java
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-DataLoads/src/main/java/org/apache/metron/dataloads/extractor/csv/LookupConverter.java b/metron-streaming/Metron-DataLoads/src/main/java/org/apache/metron/dataloads/extractor/csv/LookupConverter.java
index 9e9b79f..1fa9378 100644
--- a/metron-streaming/Metron-DataLoads/src/main/java/org/apache/metron/dataloads/extractor/csv/LookupConverter.java
+++ b/metron-streaming/Metron-DataLoads/src/main/java/org/apache/metron/dataloads/extractor/csv/LookupConverter.java
@@ -25,6 +25,6 @@ import org.apache.metron.reference.lookup.LookupValue;
 import java.util.Map;
 
 public interface LookupConverter {
-    LookupKey toKey(String indicator);
+    LookupKey toKey(String type, String indicator);
     LookupValue toValue(Map<String, String> metadata);
 }

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/9621c55e/metron-streaming/Metron-DataLoads/src/main/java/org/apache/metron/dataloads/extractor/csv/LookupConverters.java
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-DataLoads/src/main/java/org/apache/metron/dataloads/extractor/csv/LookupConverters.java b/metron-streaming/Metron-DataLoads/src/main/java/org/apache/metron/dataloads/extractor/csv/LookupConverters.java
index 7f9218a..aa23851 100644
--- a/metron-streaming/Metron-DataLoads/src/main/java/org/apache/metron/dataloads/extractor/csv/LookupConverters.java
+++ b/metron-streaming/Metron-DataLoads/src/main/java/org/apache/metron/dataloads/extractor/csv/LookupConverters.java
@@ -18,8 +18,8 @@
 
 package org.apache.metron.dataloads.extractor.csv;
 
-import org.apache.metron.hbase.converters.threatintel.ThreatIntelKey;
-import org.apache.metron.hbase.converters.threatintel.ThreatIntelValue;
+import org.apache.metron.hbase.converters.enrichment.EnrichmentKey;
+import org.apache.metron.hbase.converters.enrichment.EnrichmentValue;
 import org.apache.metron.reference.lookup.LookupKey;
 import org.apache.metron.reference.lookup.LookupValue;
 
@@ -27,15 +27,16 @@ import java.util.Map;
 
 public enum LookupConverters {
 
-    THREAT_INTEL(new LookupConverter() {
+    ENRICHMENT(new LookupConverter() {
         @Override
-        public LookupKey toKey(String indicator) {
-            return new ThreatIntelKey(indicator);
+        public LookupKey toKey(String type, String indicator) {
+            return new EnrichmentKey(type, indicator);
+
         }
 
         @Override
         public LookupValue toValue(Map<String, String> metadata) {
-            return new ThreatIntelValue(metadata);
+            return new EnrichmentValue(metadata);
         }
     })
     ;

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/9621c55e/metron-streaming/Metron-DataLoads/src/main/java/org/apache/metron/dataloads/extractor/stix/StixExtractor.java
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-DataLoads/src/main/java/org/apache/metron/dataloads/extractor/stix/StixExtractor.java b/metron-streaming/Metron-DataLoads/src/main/java/org/apache/metron/dataloads/extractor/stix/StixExtractor.java
index b3829b4..9d7e4f5 100644
--- a/metron-streaming/Metron-DataLoads/src/main/java/org/apache/metron/dataloads/extractor/stix/StixExtractor.java
+++ b/metron-streaming/Metron-DataLoads/src/main/java/org/apache/metron/dataloads/extractor/stix/StixExtractor.java
@@ -18,13 +18,11 @@
 package org.apache.metron.dataloads.extractor.stix;
 
 import com.google.common.base.Splitter;
-import com.google.common.collect.Iterables;
 import org.apache.commons.io.FileUtils;
 import org.apache.metron.dataloads.extractor.Extractor;
 import org.apache.metron.dataloads.extractor.stix.types.ObjectTypeHandler;
 import org.apache.metron.dataloads.extractor.stix.types.ObjectTypeHandlers;
 import org.apache.metron.reference.lookup.LookupKV;
-import org.apache.metron.threatintel.ThreatIntelResults;
 import org.mitre.cybox.common_2.*;
 import org.mitre.cybox.cybox_2.ObjectType;
 import org.mitre.cybox.cybox_2.Observable;

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/9621c55e/metron-streaming/Metron-DataLoads/src/main/java/org/apache/metron/dataloads/extractor/stix/types/AbstractObjectTypeHandler.java
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-DataLoads/src/main/java/org/apache/metron/dataloads/extractor/stix/types/AbstractObjectTypeHandler.java b/metron-streaming/Metron-DataLoads/src/main/java/org/apache/metron/dataloads/extractor/stix/types/AbstractObjectTypeHandler.java
index c84dba9..b637c6e 100644
--- a/metron-streaming/Metron-DataLoads/src/main/java/org/apache/metron/dataloads/extractor/stix/types/AbstractObjectTypeHandler.java
+++ b/metron-streaming/Metron-DataLoads/src/main/java/org/apache/metron/dataloads/extractor/stix/types/AbstractObjectTypeHandler.java
@@ -29,6 +29,8 @@ public abstract class AbstractObjectTypeHandler<T extends ObjectPropertiesType>
     public Class<T> getTypeClass() {
         return objectPropertiesType;
     }
-
+    public String getType() {
+        return getTypeClass().getSimpleName().toLowerCase();
+    }
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/9621c55e/metron-streaming/Metron-DataLoads/src/main/java/org/apache/metron/dataloads/extractor/stix/types/AddressHandler.java
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-DataLoads/src/main/java/org/apache/metron/dataloads/extractor/stix/types/AddressHandler.java b/metron-streaming/Metron-DataLoads/src/main/java/org/apache/metron/dataloads/extractor/stix/types/AddressHandler.java
index 638a9ce..81ea957 100644
--- a/metron-streaming/Metron-DataLoads/src/main/java/org/apache/metron/dataloads/extractor/stix/types/AddressHandler.java
+++ b/metron-streaming/Metron-DataLoads/src/main/java/org/apache/metron/dataloads/extractor/stix/types/AddressHandler.java
@@ -19,10 +19,9 @@ package org.apache.metron.dataloads.extractor.stix.types;
 
 import com.google.common.base.Splitter;
 import org.apache.metron.dataloads.extractor.stix.StixExtractor;
-import org.apache.metron.hbase.converters.threatintel.ThreatIntelKey;
-import org.apache.metron.hbase.converters.threatintel.ThreatIntelValue;
+import org.apache.metron.hbase.converters.enrichment.EnrichmentKey;
+import org.apache.metron.hbase.converters.enrichment.EnrichmentValue;
 import org.apache.metron.reference.lookup.LookupKV;
-import org.apache.metron.threatintel.ThreatIntelResults;
 import org.mitre.cybox.common_2.StringObjectPropertyType;
 import org.mitre.cybox.objects.Address;
 import org.mitre.cybox.objects.CategoryTypeEnum;
@@ -31,47 +30,65 @@ import java.io.IOException;
 import java.util.*;
 
 public class AddressHandler extends AbstractObjectTypeHandler<Address> {
-    public static final String SPECIFIC_CATEGORY_CONFIG = "stix_address_categories";
-    public static final EnumSet<CategoryTypeEnum> SUPPORTED_CATEGORIES = EnumSet.of(CategoryTypeEnum.E_MAIL
-                                                                                   ,CategoryTypeEnum.IPV_4_ADDR
-                                                                                   ,CategoryTypeEnum.IPV_6_ADDR
-                                                                                   ,CategoryTypeEnum.MAC
-                                                               ) ;
-    public AddressHandler() {
-        super(Address.class);
-    }
-
-    @Override
-    public Iterable<LookupKV> extract(final Address type, Map<String, Object> config) throws IOException {
-        List<LookupKV> ret = new ArrayList<>();
-        final CategoryTypeEnum category= type.getCategory();
-        if(!SUPPORTED_CATEGORIES.contains(category)) {
-           return ret;
-        }
-        if(config != null && config.containsKey(SPECIFIC_CATEGORY_CONFIG)) {
-            List<CategoryTypeEnum> categories = new ArrayList<>();
-            for(String c : Splitter.on(",").split(config.get(SPECIFIC_CATEGORY_CONFIG).toString())) {
-                categories.add(CategoryTypeEnum.valueOf(c));
-            }
-            EnumSet<CategoryTypeEnum> specificCategories = EnumSet.copyOf(categories);
-            if(!specificCategories.contains(category)) {
-                return ret;
-            }
+  public static final String SPECIFIC_CATEGORY_CONFIG = "stix_address_categories";
+  public static final String TYPE_CONFIG = "stix_address_type";
+  public static final EnumSet<CategoryTypeEnum> SUPPORTED_CATEGORIES = EnumSet.of(CategoryTypeEnum.E_MAIL
+          ,CategoryTypeEnum.IPV_4_ADDR
+          ,CategoryTypeEnum.IPV_6_ADDR
+          ,CategoryTypeEnum.MAC
+  ) ;
+  public AddressHandler() {
+    super(Address.class);
+  }
 
+  @Override
+  public Iterable<LookupKV> extract(final Address type, Map<String, Object> config) throws IOException {
+    List<LookupKV> ret = new ArrayList<>();
+    final CategoryTypeEnum category= type.getCategory();
+    if(!SUPPORTED_CATEGORIES.contains(category)) {
+      return ret;
+    }
+    String typeStr = getType();
+    if(config != null) {
+      if(config.containsKey(SPECIFIC_CATEGORY_CONFIG)) {
+        List<CategoryTypeEnum> categories = new ArrayList<>();
+        for (String c : Splitter.on(",").split(config.get(SPECIFIC_CATEGORY_CONFIG).toString())) {
+          categories.add(CategoryTypeEnum.valueOf(c));
         }
-        StringObjectPropertyType value = type.getAddressValue();
-        for(String token : StixExtractor.split(value)) {
-            LookupKV results = new LookupKV(new ThreatIntelKey(token)
-                                           , new ThreatIntelValue(
-                                                                    new HashMap<String, String>() {{
-                                                                        put("source-type", "STIX");
-                                                                        put("indicator-type", type.getClass().getSimpleName() + ":" + category);
-                                                                        put("source", type.toXMLString());
-                                                                    }}
-                                                                 )
-                                           );
-                ret.add(results);
+        EnumSet<CategoryTypeEnum> specificCategories = EnumSet.copyOf(categories);
+        if (!specificCategories.contains(category)) {
+          return ret;
         }
-        return ret;
+      }
+      if(config.containsKey(TYPE_CONFIG)) {
+        typeStr = config.get(TYPE_CONFIG).toString();
+      }
+    }
+    StringObjectPropertyType value = type.getAddressValue();
+    for(String token : StixExtractor.split(value)) {
+      final String indicatorType = typeStr + ":" + category;
+      LookupKV results = new LookupKV(new EnrichmentKey(indicatorType, token)
+              , new EnrichmentValue(
+              new HashMap<String, String>() {{
+                put("source-type", "STIX");
+                put("indicator-type", indicatorType);
+                put("source", type.toXMLString());
+              }}
+      )
+      );
+      ret.add(results);
+    }
+    return ret;
+  }
+
+  @Override
+  public List<String> getPossibleTypes() {
+    String typeStr = getType();
+    List<String> ret = new ArrayList<>();
+    for(CategoryTypeEnum e : SUPPORTED_CATEGORIES)
+    {
+       ret.add(typeStr + ":" + e);
     }
+    return ret;
+  }
 }


[2/6] incubator-metron git commit: METRON-93: Generalize the HBase threat intel infrastructure to support enrichments closes apache/incubator-metron#64

Posted by ce...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/9621c55e/metron-streaming/Metron-DataLoads/src/test/java/org/apache/metron/dataloads/taxii/TaxiiIntegrationTest.java
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-DataLoads/src/test/java/org/apache/metron/dataloads/taxii/TaxiiIntegrationTest.java b/metron-streaming/Metron-DataLoads/src/test/java/org/apache/metron/dataloads/taxii/TaxiiIntegrationTest.java
deleted file mode 100644
index f61d729..0000000
--- a/metron-streaming/Metron-DataLoads/src/test/java/org/apache/metron/dataloads/taxii/TaxiiIntegrationTest.java
+++ /dev/null
@@ -1,121 +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.dataloads.taxii;
-
-import com.google.common.base.Splitter;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.HBaseConfiguration;
-import org.apache.hadoop.hbase.client.HTableInterface;
-import org.apache.hadoop.hbase.client.Put;
-import org.apache.metron.dataloads.extractor.stix.StixExtractor;
-import org.apache.metron.hbase.converters.threatintel.ThreatIntelConverter;
-import org.apache.metron.integration.util.UnitTestHelper;
-import org.apache.metron.integration.util.mock.MockHTable;
-import org.junit.After;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Test;
-
-import java.io.IOException;
-import java.util.HashSet;
-import java.util.Set;
-
-public class TaxiiIntegrationTest {
-
-    @Before
-    public void setup() throws IOException {
-        MockTaxiiService.start(8282);
-    }
-
-    @After
-    public void teardown() {
-        MockTaxiiService.shutdown();
-        MockHTable.Provider.clear();
-    }
-
-
-
-    @Test
-    public void testTaxii() throws Exception {
-        /**
-         {
-            "endpoint" : "http://localhost:8282/taxii-discovery-service"
-           ,"type" : "DISCOVER"
-           ,"collection" : "guest.Abuse_ch"
-           ,"tableMap" : {
-                    "DomainName:FQDN" : "malicious_domain:cf"
-                   ,"Address:IPV_4_ADDR" : "malicious_address:cf"
-                         }
-         }
-         */
-        String taxiiConnectionConfig = "{\n" +
-                "            \"endpoint\" : \"http://localhost:8282/taxii-discovery-service\"\n" +
-                "           ,\"type\" : \"DISCOVER\"\n" +
-                "           ,\"collection\" : \"guest.Abuse_ch\"\n" +
-                "           ,\"tableMap\" : {\n" +
-                "                    \"DomainName:FQDN\" : \"malicious_domain:cf\"\n" +
-                "                   ,\"Address:IPV_4_ADDR\" : \"malicious_address:cf\"\n" +
-                "                         }\n" +
-                "         }";
-        final MockHTable.Provider provider = new MockHTable.Provider();
-        final Configuration config = HBaseConfiguration.create();
-        TaxiiHandler handler = new TaxiiHandler(TaxiiConnectionConfig.load(taxiiConnectionConfig), new StixExtractor(), config ) {
-            @Override
-            protected synchronized HTableInterface createHTable(TableInfo tableInfo) throws IOException {
-                return provider.addToCache(tableInfo.getTableName(), tableInfo.getColumnFamily());
-            }
-        };
-        //UnitTestHelper.verboseLogging();
-        handler.run();
-        Set<String> maliciousDomains;
-        {
-            MockHTable table = (MockHTable) provider.getTable(config, "malicious_domain");
-            maliciousDomains = getIndicators(table.getPutLog(), "cf");
-        }
-        Assert.assertTrue(maliciousDomains.contains("www.office-112.com"));
-        Assert.assertEquals(numStringsMatch(MockTaxiiService.pollMsg, "DomainNameObj:Value condition=\"Equals\""), maliciousDomains.size());
-        Set<String> maliciousAddresses;
-        {
-            MockHTable table = (MockHTable) provider.getTable(config, "malicious_address");
-            maliciousAddresses= getIndicators(table.getPutLog(), "cf");
-        }
-        Assert.assertTrue(maliciousAddresses.contains("94.102.53.142"));
-        Assert.assertEquals(numStringsMatch(MockTaxiiService.pollMsg, "AddressObj:Address_Value condition=\"Equal\""), maliciousAddresses.size());
-        MockHTable.Provider.clear();
-    }
-
-    private static int numStringsMatch(String xmlBundle, String text) {
-        int cnt = 0;
-        for(String line : Splitter.on("\n").split(xmlBundle)) {
-            if(line.contains(text)) {
-                cnt++;
-            }
-        }
-        return cnt;
-    }
-
-    private static Set<String> getIndicators(Iterable<Put> puts, String cf) throws IOException {
-        ThreatIntelConverter converter = new ThreatIntelConverter();
-        Set<String> ret = new HashSet<>();
-        for(Put p : puts) {
-            ret.add(converter.fromPut(p, cf).getKey().indicator);
-        }
-        return ret;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/9621c55e/metron-streaming/Metron-Elasticsearch/pom.xml
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-Elasticsearch/pom.xml b/metron-streaming/Metron-Elasticsearch/pom.xml
index ab9242a..d2eade3 100644
--- a/metron-streaming/Metron-Elasticsearch/pom.xml
+++ b/metron-streaming/Metron-Elasticsearch/pom.xml
@@ -27,6 +27,11 @@
     </properties>
     <dependencies>
         <dependency>
+            <groupId>com.google.guava</groupId>
+            <artifactId>guava</artifactId>
+            <version>${global_hbase_guava_version}</version>
+        </dependency>
+        <dependency>
             <groupId>org.apache.metron</groupId>
             <artifactId>Metron-Common</artifactId>
             <version>${project.parent.version}</version>

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/9621c55e/metron-streaming/Metron-Elasticsearch/src/main/java/org/apache/metron/writer/ElasticsearchWriter.java
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-Elasticsearch/src/main/java/org/apache/metron/writer/ElasticsearchWriter.java b/metron-streaming/Metron-Elasticsearch/src/main/java/org/apache/metron/writer/ElasticsearchWriter.java
index 45631f2..81aeec2 100644
--- a/metron-streaming/Metron-Elasticsearch/src/main/java/org/apache/metron/writer/ElasticsearchWriter.java
+++ b/metron-streaming/Metron-Elasticsearch/src/main/java/org/apache/metron/writer/ElasticsearchWriter.java
@@ -76,7 +76,7 @@ public class ElasticsearchWriter implements BulkMessageWriter<JSONObject>, Seria
         indexName = sensorEnrichmentConfig.getIndex();
       }
       IndexRequestBuilder indexRequestBuilder = client.prepareIndex(indexName + "_index_" + indexPostfix,
-              sensorType);
+              sensorType + "_doc");
 
       indexRequestBuilder.setSource(message.toJSONString());
       bulkRequest.add(indexRequestBuilder);

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/9621c55e/metron-streaming/Metron-Elasticsearch/src/test/java/org/apache/metron/integration/ElasticsearchEnrichmentIntegrationTest.java
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-Elasticsearch/src/test/java/org/apache/metron/integration/ElasticsearchEnrichmentIntegrationTest.java b/metron-streaming/Metron-Elasticsearch/src/test/java/org/apache/metron/integration/ElasticsearchEnrichmentIntegrationTest.java
index 2765c25..4f26365 100644
--- a/metron-streaming/Metron-Elasticsearch/src/test/java/org/apache/metron/integration/ElasticsearchEnrichmentIntegrationTest.java
+++ b/metron-streaming/Metron-Elasticsearch/src/test/java/org/apache/metron/integration/ElasticsearchEnrichmentIntegrationTest.java
@@ -54,7 +54,7 @@ public class ElasticsearchEnrichmentIntegrationTest extends EnrichmentIntegratio
         if (elasticSearchComponent.hasIndex(index)) {
           List<Map<String, Object>> docsFromDisk;
           try {
-            docs = elasticSearchComponent.getAllIndexedDocs(index, "yaf");
+            docs = elasticSearchComponent.getAllIndexedDocs(index, "yaf_doc");
             docsFromDisk = readDocsFromDisk(hdfsDir);
             System.out.println(docs.size() + " vs " + inputMessages.size() + " vs " + docsFromDisk.size());
           } catch (IOException e) {

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/9621c55e/metron-streaming/Metron-EnrichmentAdapters/pom.xml
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-EnrichmentAdapters/pom.xml b/metron-streaming/Metron-EnrichmentAdapters/pom.xml
index 83e1aed..7399ade 100644
--- a/metron-streaming/Metron-EnrichmentAdapters/pom.xml
+++ b/metron-streaming/Metron-EnrichmentAdapters/pom.xml
@@ -28,7 +28,7 @@
         <mysql.version>5.1.31</mysql.version>
         <slf4j.version>1.7.7</slf4j.version>
         <storm.hdfs.version>0.1.2</storm.hdfs.version>
-        <guava.version>${global_guava_version}</guava.version>
+        <guava.version>${global_hbase_guava_version}</guava.version>
     </properties>
     <dependencies>
         <dependency>
@@ -99,6 +99,7 @@
             <groupId>org.apache.hadoop</groupId>
             <artifactId>hadoop-common</artifactId>
             <version>${global_hadoop_version}</version>
+            <scope>provided</scope>
             <exclusions>
                 <exclusion>
                     <artifactId>servlet-api</artifactId>

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/9621c55e/metron-streaming/Metron-EnrichmentAdapters/src/main/java/org/apache/metron/enrichment/adapters/cif/AbstractCIFAdapter.java
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-EnrichmentAdapters/src/main/java/org/apache/metron/enrichment/adapters/cif/AbstractCIFAdapter.java b/metron-streaming/Metron-EnrichmentAdapters/src/main/java/org/apache/metron/enrichment/adapters/cif/AbstractCIFAdapter.java
index 1d0b5c1..73a7ad5 100644
--- a/metron-streaming/Metron-EnrichmentAdapters/src/main/java/org/apache/metron/enrichment/adapters/cif/AbstractCIFAdapter.java
+++ b/metron-streaming/Metron-EnrichmentAdapters/src/main/java/org/apache/metron/enrichment/adapters/cif/AbstractCIFAdapter.java
@@ -20,12 +20,13 @@ package org.apache.metron.enrichment.adapters.cif;
 
 import java.io.Serializable;
 
+import org.apache.metron.enrichment.bolt.CacheKey;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import org.apache.metron.enrichment.interfaces.EnrichmentAdapter;
 
-public abstract class AbstractCIFAdapter implements EnrichmentAdapter,Serializable{
+public abstract class AbstractCIFAdapter implements EnrichmentAdapter<CacheKey>,Serializable{
 
 	/**
 	 * 

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/9621c55e/metron-streaming/Metron-EnrichmentAdapters/src/main/java/org/apache/metron/enrichment/adapters/cif/CIFHbaseAdapter.java
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-EnrichmentAdapters/src/main/java/org/apache/metron/enrichment/adapters/cif/CIFHbaseAdapter.java b/metron-streaming/Metron-EnrichmentAdapters/src/main/java/org/apache/metron/enrichment/adapters/cif/CIFHbaseAdapter.java
index 1ab3b83..63d6c0b 100644
--- a/metron-streaming/Metron-EnrichmentAdapters/src/main/java/org/apache/metron/enrichment/adapters/cif/CIFHbaseAdapter.java
+++ b/metron-streaming/Metron-EnrichmentAdapters/src/main/java/org/apache/metron/enrichment/adapters/cif/CIFHbaseAdapter.java
@@ -23,6 +23,7 @@ import java.io.Serializable;
 import java.util.HashMap;
 import java.util.Map;
 
+import org.apache.metron.enrichment.bolt.CacheKey;
 import org.apache.metron.enrichment.interfaces.EnrichmentAdapter;
 import org.json.simple.JSONObject;
 import org.apache.hadoop.conf.Configuration;
@@ -36,7 +37,7 @@ import org.apache.hadoop.hbase.KeyValue;
 import org.apache.log4j.Logger;
 
 @SuppressWarnings("unchecked")
-public class CIFHbaseAdapter implements EnrichmentAdapter<String>,Serializable {
+public class CIFHbaseAdapter implements EnrichmentAdapter<CacheKey>,Serializable {
 
 	private static final long serialVersionUID = 1L;
 	private String _tableName;
@@ -55,12 +56,12 @@ public class CIFHbaseAdapter implements EnrichmentAdapter<String>,Serializable {
 			.getLogger(CIFHbaseAdapter.class);
 
 	@Override
-	public void logAccess(String value) {
+	public void logAccess(CacheKey value) {
 
 	}
 
-	public JSONObject enrich(String metadata) {
-
+	public JSONObject enrich(CacheKey k) {
+		String metadata = k.getValue();
 		JSONObject output = new JSONObject();
 		LOGGER.debug("=======Looking Up For:" + metadata);
 		output.putAll(getCIFObject(metadata));

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/9621c55e/metron-streaming/Metron-EnrichmentAdapters/src/main/java/org/apache/metron/enrichment/adapters/geo/GeoAdapter.java
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-EnrichmentAdapters/src/main/java/org/apache/metron/enrichment/adapters/geo/GeoAdapter.java b/metron-streaming/Metron-EnrichmentAdapters/src/main/java/org/apache/metron/enrichment/adapters/geo/GeoAdapter.java
index 4e48756..5d12a29 100644
--- a/metron-streaming/Metron-EnrichmentAdapters/src/main/java/org/apache/metron/enrichment/adapters/geo/GeoAdapter.java
+++ b/metron-streaming/Metron-EnrichmentAdapters/src/main/java/org/apache/metron/enrichment/adapters/geo/GeoAdapter.java
@@ -19,6 +19,7 @@ package org.apache.metron.enrichment.adapters.geo;
 
 import org.apache.commons.validator.routines.InetAddressValidator;
 import org.apache.metron.enrichment.adapters.jdbc.JdbcAdapter;
+import org.apache.metron.enrichment.bolt.CacheKey;
 import org.json.simple.JSONObject;
 
 import java.net.InetAddress;
@@ -29,19 +30,19 @@ public class GeoAdapter extends JdbcAdapter {
   private InetAddressValidator ipvalidator = new InetAddressValidator();
 
   @Override
-  public void logAccess(String value) {
+  public void logAccess(CacheKey value) {
 
   }
 
   @SuppressWarnings("unchecked")
   @Override
-  public JSONObject enrich(String value) {
+  public JSONObject enrich(CacheKey value) {
     JSONObject enriched = new JSONObject();
     try {
-      InetAddress addr = InetAddress.getByName(value);
+      InetAddress addr = InetAddress.getByName(value.getValue());
       if (addr.isAnyLocalAddress() || addr.isLoopbackAddress()
               || addr.isSiteLocalAddress() || addr.isMulticastAddress()
-              || !ipvalidator.isValidInet4Address(value)) {
+              || !ipvalidator.isValidInet4Address(value.getValue())) {
         return new JSONObject();
       }
       String locidQuery = "select IPTOLOCID(\"" + value
@@ -67,8 +68,7 @@ public class GeoAdapter extends JdbcAdapter {
       }
       resultSet.close();
     } catch (Exception e) {
-      e.printStackTrace();
-      _LOG.error("Enrichment failure: " + e);
+      _LOG.error("Enrichment failure: " + e.getMessage(), e);
       return new JSONObject();
     }
     return enriched;

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/9621c55e/metron-streaming/Metron-EnrichmentAdapters/src/main/java/org/apache/metron/enrichment/adapters/host/AbstractHostAdapter.java
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-EnrichmentAdapters/src/main/java/org/apache/metron/enrichment/adapters/host/AbstractHostAdapter.java b/metron-streaming/Metron-EnrichmentAdapters/src/main/java/org/apache/metron/enrichment/adapters/host/AbstractHostAdapter.java
index 5118e5f..329456f 100644
--- a/metron-streaming/Metron-EnrichmentAdapters/src/main/java/org/apache/metron/enrichment/adapters/host/AbstractHostAdapter.java
+++ b/metron-streaming/Metron-EnrichmentAdapters/src/main/java/org/apache/metron/enrichment/adapters/host/AbstractHostAdapter.java
@@ -20,13 +20,14 @@ package org.apache.metron.enrichment.adapters.host;
 
 import java.io.Serializable;
 
+import org.apache.metron.enrichment.bolt.CacheKey;
 import org.json.simple.JSONObject;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import org.apache.metron.enrichment.interfaces.EnrichmentAdapter;
 
-public abstract class AbstractHostAdapter implements EnrichmentAdapter<String>,
+public abstract class AbstractHostAdapter implements EnrichmentAdapter<CacheKey>,
 				Serializable{
 
 	/**
@@ -37,7 +38,7 @@ public abstract class AbstractHostAdapter implements EnrichmentAdapter<String>,
 			.getLogger(AbstractHostAdapter.class);
 	
 	abstract public boolean initializeAdapter();
-	abstract public JSONObject enrich(String metadata);
+	abstract public JSONObject enrich(CacheKey metadata);
 
 	@Override
 	public void cleanup() {

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/9621c55e/metron-streaming/Metron-EnrichmentAdapters/src/main/java/org/apache/metron/enrichment/adapters/host/HostFromJSONListAdapter.java
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-EnrichmentAdapters/src/main/java/org/apache/metron/enrichment/adapters/host/HostFromJSONListAdapter.java b/metron-streaming/Metron-EnrichmentAdapters/src/main/java/org/apache/metron/enrichment/adapters/host/HostFromJSONListAdapter.java
index c55b918..640e548 100644
--- a/metron-streaming/Metron-EnrichmentAdapters/src/main/java/org/apache/metron/enrichment/adapters/host/HostFromJSONListAdapter.java
+++ b/metron-streaming/Metron-EnrichmentAdapters/src/main/java/org/apache/metron/enrichment/adapters/host/HostFromJSONListAdapter.java
@@ -17,6 +17,7 @@
  */
 package org.apache.metron.enrichment.adapters.host;
 
+import org.apache.metron.enrichment.bolt.CacheKey;
 import org.json.simple.JSONArray;
 import org.json.simple.JSONObject;
 import org.json.simple.JSONValue;
@@ -53,14 +54,14 @@ public class HostFromJSONListAdapter extends AbstractHostAdapter {
   }
 
   @Override
-  public void logAccess(String value) {
+  public void logAccess(CacheKey value) {
 
   }
 
   @SuppressWarnings("unchecked")
   @Override
-  public JSONObject enrich(String metadata) {
-
+  public JSONObject enrich(CacheKey k) {
+    String metadata = k.getValue();
 
     if(!_known_hosts.containsKey(metadata))
       return new JSONObject();

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/9621c55e/metron-streaming/Metron-EnrichmentAdapters/src/main/java/org/apache/metron/enrichment/adapters/host/HostFromPropertiesFileAdapter.java
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-EnrichmentAdapters/src/main/java/org/apache/metron/enrichment/adapters/host/HostFromPropertiesFileAdapter.java b/metron-streaming/Metron-EnrichmentAdapters/src/main/java/org/apache/metron/enrichment/adapters/host/HostFromPropertiesFileAdapter.java
index d7fcfbd..f92bd3f 100644
--- a/metron-streaming/Metron-EnrichmentAdapters/src/main/java/org/apache/metron/enrichment/adapters/host/HostFromPropertiesFileAdapter.java
+++ b/metron-streaming/Metron-EnrichmentAdapters/src/main/java/org/apache/metron/enrichment/adapters/host/HostFromPropertiesFileAdapter.java
@@ -20,6 +20,7 @@ package org.apache.metron.enrichment.adapters.host;
 
 import java.util.Map;
 
+import org.apache.metron.enrichment.bolt.CacheKey;
 import org.json.simple.JSONObject;
 
 @SuppressWarnings("serial")
@@ -43,20 +44,20 @@ public class HostFromPropertiesFileAdapter extends AbstractHostAdapter {
 	}
 
 	@Override
-	public void logAccess(String value) {
+	public void logAccess(CacheKey value) {
 
 	}
 
 	@SuppressWarnings("unchecked")
     @Override
-	public JSONObject enrich(String metadata) {
+	public JSONObject enrich(CacheKey metadata) {
 		
 		
-		if(!_known_hosts.containsKey(metadata))
+		if(!_known_hosts.containsKey(metadata.getValue()))
 			return new JSONObject();
 		
 		JSONObject enrichment = new JSONObject();
-		enrichment.put("known_info", (JSONObject) _known_hosts.get(metadata));
+		enrichment.put("known_info", (JSONObject) _known_hosts.get(metadata.getValue()));
 		return enrichment;
 	}
 	

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/9621c55e/metron-streaming/Metron-EnrichmentAdapters/src/main/java/org/apache/metron/enrichment/adapters/jdbc/JdbcAdapter.java
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-EnrichmentAdapters/src/main/java/org/apache/metron/enrichment/adapters/jdbc/JdbcAdapter.java b/metron-streaming/Metron-EnrichmentAdapters/src/main/java/org/apache/metron/enrichment/adapters/jdbc/JdbcAdapter.java
index 5eabdd2..b21044f 100644
--- a/metron-streaming/Metron-EnrichmentAdapters/src/main/java/org/apache/metron/enrichment/adapters/jdbc/JdbcAdapter.java
+++ b/metron-streaming/Metron-EnrichmentAdapters/src/main/java/org/apache/metron/enrichment/adapters/jdbc/JdbcAdapter.java
@@ -17,6 +17,7 @@
  */
 package org.apache.metron.enrichment.adapters.jdbc;
 
+import org.apache.metron.enrichment.bolt.CacheKey;
 import org.apache.metron.enrichment.interfaces.EnrichmentAdapter;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -25,7 +26,7 @@ import java.io.Serializable;
 import java.net.InetAddress;
 import java.sql.*;
 
-public abstract class JdbcAdapter implements EnrichmentAdapter<String>,
+public abstract class JdbcAdapter implements EnrichmentAdapter<CacheKey>,
         Serializable {
 
   protected static final Logger _LOG = LoggerFactory

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/9621c55e/metron-streaming/Metron-EnrichmentAdapters/src/main/java/org/apache/metron/enrichment/adapters/simplehbase/SimpleHBaseAdapter.java
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-EnrichmentAdapters/src/main/java/org/apache/metron/enrichment/adapters/simplehbase/SimpleHBaseAdapter.java b/metron-streaming/Metron-EnrichmentAdapters/src/main/java/org/apache/metron/enrichment/adapters/simplehbase/SimpleHBaseAdapter.java
new file mode 100644
index 0000000..2b7d1a0
--- /dev/null
+++ b/metron-streaming/Metron-EnrichmentAdapters/src/main/java/org/apache/metron/enrichment/adapters/simplehbase/SimpleHBaseAdapter.java
@@ -0,0 +1,119 @@
+/**
+ * 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.enrichment.adapters.simplehbase;
+
+
+import com.google.common.collect.Iterables;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.metron.enrichment.bolt.CacheKey;
+import org.apache.metron.enrichment.interfaces.EnrichmentAdapter;
+import org.apache.metron.enrichment.utils.EnrichmentUtils;
+import org.apache.metron.hbase.converters.enrichment.EnrichmentKey;
+import org.apache.metron.hbase.converters.enrichment.EnrichmentValue;
+import org.apache.metron.hbase.lookup.EnrichmentLookup;
+import org.apache.metron.reference.lookup.LookupKV;
+import org.apache.metron.reference.lookup.accesstracker.NoopAccessTracker;
+import org.json.simple.JSONObject;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.List;
+import java.util.Map;
+
+public class SimpleHBaseAdapter implements EnrichmentAdapter<CacheKey>,Serializable {
+  protected static final Logger _LOG = LoggerFactory.getLogger(SimpleHBaseAdapter.class);
+  protected SimpleHBaseConfig config;
+  protected EnrichmentLookup lookup;
+
+  public SimpleHBaseAdapter() {
+  }
+  public SimpleHBaseAdapter(SimpleHBaseConfig config) {
+    withConfig(config);
+  }
+
+  public SimpleHBaseAdapter withConfig(SimpleHBaseConfig config) {
+    this.config = config;
+    return this;
+  }
+
+  @Override
+  public void logAccess(CacheKey value) {
+  }
+
+
+  @Override
+  public JSONObject enrich(CacheKey value) {
+    JSONObject enriched = new JSONObject();
+    List<String> enrichmentTypes = value.getConfig()
+                                        .getFieldToEnrichmentTypeMap()
+                                        .get(EnrichmentUtils.toTopLevelField(value.getField()));
+    if(enrichmentTypes != null && value.getValue() != null) {
+      try {
+        for (LookupKV<EnrichmentKey, EnrichmentValue> kv :
+                lookup.get(Iterables.transform(enrichmentTypes
+                                              , new EnrichmentUtils.TypeToKey(value.getValue())
+                                              )
+                          , lookup.getTable()
+                          , false
+                          )
+            )
+        {
+          if (kv != null && kv.getValue() != null && kv.getValue().getMetadata() != null) {
+            for (Map.Entry<String, String> values : kv.getValue().getMetadata().entrySet()) {
+              enriched.put(kv.getKey().type + "." + values.getKey(), values.getValue());
+            }
+            _LOG.trace("Enriched type " + kv.getKey().type + " => " + enriched);
+          }
+        }
+      }
+      catch (IOException e) {
+        _LOG.error("Unable to retrieve value: " + e.getMessage(), e);
+        throw new RuntimeException("Unable to retrieve value: " + e.getMessage(), e);
+      }
+    }
+    return enriched;
+  }
+
+  @Override
+  public boolean initializeAdapter() {
+    String hbaseTable = config.getHBaseTable();
+    Configuration hbaseConfig = HBaseConfiguration.create();
+    try {
+      lookup = new EnrichmentLookup( config.getProvider().getTable(hbaseConfig, hbaseTable)
+                                   , config.getHBaseCF()
+                                   , new NoopAccessTracker()
+                                   );
+    } catch (IOException e) {
+      throw new RuntimeException("Unable to initialize adapter: " + e.getMessage(), e);
+    }
+    return true;
+  }
+
+  @Override
+  public void cleanup() {
+    try {
+      lookup.close();
+    } catch (Exception e) {
+      throw new RuntimeException("Unable to cleanup access tracker", e);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/9621c55e/metron-streaming/Metron-EnrichmentAdapters/src/main/java/org/apache/metron/enrichment/adapters/simplehbase/SimpleHBaseConfig.java
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-EnrichmentAdapters/src/main/java/org/apache/metron/enrichment/adapters/simplehbase/SimpleHBaseConfig.java b/metron-streaming/Metron-EnrichmentAdapters/src/main/java/org/apache/metron/enrichment/adapters/simplehbase/SimpleHBaseConfig.java
new file mode 100644
index 0000000..fefe008
--- /dev/null
+++ b/metron-streaming/Metron-EnrichmentAdapters/src/main/java/org/apache/metron/enrichment/adapters/simplehbase/SimpleHBaseConfig.java
@@ -0,0 +1,55 @@
+/**
+ * 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.enrichment.adapters.simplehbase;
+
+import org.apache.metron.enrichment.utils.EnrichmentUtils;
+import org.apache.metron.hbase.HTableProvider;
+import org.apache.metron.hbase.TableProvider;
+
+import java.io.Serializable;
+
+
+public class SimpleHBaseConfig implements Serializable {
+  private String hBaseTable;
+  private String hBaseCF;
+  private TableProvider provider = new HTableProvider();
+  public String getHBaseTable() {
+    return hBaseTable;
+  }
+  public String getHBaseCF() {
+    return hBaseCF;
+  }
+
+  public TableProvider getProvider() {
+    return provider;
+  }
+
+  public SimpleHBaseConfig withProviderImpl(String connectorImpl) {
+    provider = EnrichmentUtils.getTableProvider(connectorImpl, new HTableProvider());
+    return this;
+  }
+  public SimpleHBaseConfig withHBaseTable(String hBaseTable) {
+    this.hBaseTable = hBaseTable;
+    return this;
+  }
+
+  public SimpleHBaseConfig withHBaseCF(String cf) {
+    this.hBaseCF= cf;
+    return this;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/9621c55e/metron-streaming/Metron-EnrichmentAdapters/src/main/java/org/apache/metron/enrichment/adapters/threat/AbstractThreatAdapter.java
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-EnrichmentAdapters/src/main/java/org/apache/metron/enrichment/adapters/threat/AbstractThreatAdapter.java b/metron-streaming/Metron-EnrichmentAdapters/src/main/java/org/apache/metron/enrichment/adapters/threat/AbstractThreatAdapter.java
deleted file mode 100644
index 9c14cdf..0000000
--- a/metron-streaming/Metron-EnrichmentAdapters/src/main/java/org/apache/metron/enrichment/adapters/threat/AbstractThreatAdapter.java
+++ /dev/null
@@ -1,41 +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.enrichment.adapters.threat;
-
-import java.io.Serializable;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import org.apache.metron.enrichment.interfaces.EnrichmentAdapter;
-
-public abstract class AbstractThreatAdapter implements EnrichmentAdapter,Serializable{
-
-	
-	private static final long serialVersionUID = 1524030932856141771L;
-	protected static final Logger LOG = LoggerFactory
-			.getLogger(AbstractThreatAdapter.class);
-	
-	abstract public boolean initializeAdapter();
-
-	@Override
-	public void cleanup() {
-
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/9621c55e/metron-streaming/Metron-EnrichmentAdapters/src/main/java/org/apache/metron/enrichment/adapters/threat/ThreatHbaseAdapter.java
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-EnrichmentAdapters/src/main/java/org/apache/metron/enrichment/adapters/threat/ThreatHbaseAdapter.java b/metron-streaming/Metron-EnrichmentAdapters/src/main/java/org/apache/metron/enrichment/adapters/threat/ThreatHbaseAdapter.java
deleted file mode 100644
index 1ce99cb..0000000
--- a/metron-streaming/Metron-EnrichmentAdapters/src/main/java/org/apache/metron/enrichment/adapters/threat/ThreatHbaseAdapter.java
+++ /dev/null
@@ -1,140 +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.enrichment.adapters.threat;
-
-import java.io.IOException;
-import java.io.Serializable;
-import java.util.HashMap;
-import java.util.Map;
-
-import org.apache.metron.enrichment.interfaces.EnrichmentAdapter;
-import org.json.simple.JSONObject;
-import org.json.simple.parser.JSONParser;
-import org.json.simple.parser.ParseException;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.HBaseConfiguration;
-import org.apache.hadoop.hbase.client.Get;
-import org.apache.hadoop.hbase.client.HConnection;
-import org.apache.hadoop.hbase.client.HConnectionManager;
-import org.apache.hadoop.hbase.client.HTableInterface;
-import org.apache.hadoop.hbase.client.Result;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.log4j.Logger;
-import org.slf4j.LoggerFactory;
-
-@SuppressWarnings("unchecked")
-public class ThreatHbaseAdapter implements EnrichmentAdapter<String>,
-				Serializable {
-
-	protected static final org.slf4j.Logger LOG = LoggerFactory
-					.getLogger(ThreatHbaseAdapter.class);
-	private static final long serialVersionUID = 1L;
-	private String _tableName;
-	private HTableInterface table;
-	private String _quorum;
-	private String _port;
-
-	public ThreatHbaseAdapter(String quorum, String port, String tableName) {
-		_quorum = quorum;
-		_port = port;
-		_tableName = tableName;
-	}
-
-	/** The LOGGER. */
-	private static final Logger LOGGER = Logger
-			.getLogger(ThreatHbaseAdapter.class);
-
-	@Override
-	public void logAccess(String value) {
-
-	}
-
-	public JSONObject enrich(String metadata) {
-
-		JSONObject output = new JSONObject();
-		LOGGER.debug("=======Looking Up For:" + metadata);
-		output.putAll(getThreatObject(metadata));
-
-		return output;
-	}
-
-	@SuppressWarnings({ "rawtypes", "deprecation" })
-	protected Map getThreatObject(String key) {
-
-		LOGGER.debug("=======Pinging HBase For:" + key);
-		
-		Get get = new Get(Bytes.toBytes(key));
-		Result rs;
-		Map output = new HashMap();
-
-		try {
-			rs = table.get(get);
-
-			if (!rs.isEmpty()) {
-				byte[] source_family = Bytes.toBytes("source");
-				JSONParser parser = new JSONParser();
-				
-				Map<byte[], byte[]> sourceFamilyMap = rs.getFamilyMap(source_family);
-				
-				for (Map.Entry<byte[], byte[]> entry  : sourceFamilyMap.entrySet()) {
-					String k = Bytes.toString(entry.getKey());
-					LOGGER.debug("=======Found intel from source: " + k);
-					output.put(k,parser.parse(Bytes.toString(entry.getValue())));
-	            }
-			}
-		} catch (IOException e) {
-			// TODO Auto-generated catch block
-			e.printStackTrace();
-		} catch (ParseException e) {
-			// TODO Auto-generated catch block
-			e.printStackTrace();
-		}
-		return output;
-	}
-
-	@Override
-	public boolean initializeAdapter() {
-
-		// Initialize HBase Table
-		Configuration conf = null;
-		conf = HBaseConfiguration.create();
-		conf.set("hbase.zookeeper.quorum", _quorum);
-		conf.set("hbase.zookeeper.property.clientPort", _port);
-
-		try {
-			LOGGER.debug("=======Connecting to HBASE===========");
-			LOGGER.debug("=======ZOOKEEPER = "
-					+ conf.get("hbase.zookeeper.quorum"));
-			HConnection connection = HConnectionManager.createConnection(conf);
-			table = connection.getTable(_tableName);
-			return true;
-		} catch (IOException e) {
-			// TODO Auto-generated catch block
-			LOGGER.debug("=======Unable to Connect to HBASE===========");
-			e.printStackTrace();
-		}
-
-		return false;
-	}
-
-	@Override
-	public void cleanup() {
-
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/9621c55e/metron-streaming/Metron-EnrichmentAdapters/src/main/java/org/apache/metron/enrichment/adapters/threatintel/ThreatIntelAdapter.java
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-EnrichmentAdapters/src/main/java/org/apache/metron/enrichment/adapters/threatintel/ThreatIntelAdapter.java b/metron-streaming/Metron-EnrichmentAdapters/src/main/java/org/apache/metron/enrichment/adapters/threatintel/ThreatIntelAdapter.java
new file mode 100644
index 0000000..c80b57a
--- /dev/null
+++ b/metron-streaming/Metron-EnrichmentAdapters/src/main/java/org/apache/metron/enrichment/adapters/threatintel/ThreatIntelAdapter.java
@@ -0,0 +1,135 @@
+/**
+ * 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.enrichment.adapters.threatintel;
+
+import com.google.common.base.Splitter;
+import com.google.common.collect.Iterables;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.metron.enrichment.bolt.CacheKey;
+import org.apache.metron.enrichment.interfaces.EnrichmentAdapter;
+import org.apache.metron.enrichment.utils.EnrichmentUtils;
+import org.apache.metron.hbase.converters.enrichment.EnrichmentKey;
+import org.apache.metron.hbase.lookup.EnrichmentLookup;
+import org.apache.metron.reference.lookup.accesstracker.BloomAccessTracker;
+import org.apache.metron.reference.lookup.accesstracker.PersistentAccessTracker;
+import org.json.simple.JSONObject;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.List;
+import java.util.UUID;
+
+public class ThreatIntelAdapter implements EnrichmentAdapter<CacheKey>,Serializable {
+  protected static final Logger _LOG = LoggerFactory.getLogger(ThreatIntelAdapter.class);
+  protected ThreatIntelConfig config;
+  protected EnrichmentLookup lookup;
+
+  public ThreatIntelAdapter() {
+  }
+  public ThreatIntelAdapter(ThreatIntelConfig config) {
+    withConfig(config);
+  }
+
+  public ThreatIntelAdapter withConfig(ThreatIntelConfig config) {
+    this.config = config;
+    return this;
+  }
+
+  @Override
+  public void logAccess(CacheKey value) {
+    List<String> enrichmentTypes = value.getConfig().getFieldToThreatIntelTypeMap().get(value.getField());
+    if(enrichmentTypes != null) {
+      for(String enrichmentType : enrichmentTypes) {
+        lookup.getAccessTracker().logAccess(new EnrichmentKey(enrichmentType, value.getValue()));
+      }
+    }
+  }
+
+
+  @Override
+  public JSONObject enrich(CacheKey value) {
+    JSONObject enriched = new JSONObject();
+    List<String> enrichmentTypes = value.getConfig()
+                                        .getFieldToThreatIntelTypeMap()
+                                        .get(EnrichmentUtils.toTopLevelField(value.getField()));
+    if(enrichmentTypes != null) {
+      int i = 0;
+      try {
+        for (Boolean isThreat :
+                lookup.exists(Iterables.transform(enrichmentTypes
+                                                 , new EnrichmentUtils.TypeToKey(value.getValue())
+                                                 )
+                             , lookup.getTable()
+                             , false
+                             )
+            )
+        {
+          String enrichmentType = enrichmentTypes.get(i++);
+          if (isThreat) {
+            enriched.put(enrichmentType, "alert");
+            _LOG.trace("Enriched value => " + enriched);
+          }
+        }
+      }
+      catch(IOException e) {
+        throw new RuntimeException("Unable to retrieve value", e);
+      }
+    }
+    //throw new RuntimeException("Unable to retrieve value " + value);
+    return enriched;
+  }
+
+  @Override
+  public boolean initializeAdapter() {
+    PersistentAccessTracker accessTracker;
+    String hbaseTable = config.getHBaseTable();
+    int expectedInsertions = config.getExpectedInsertions();
+    double falsePositives = config.getFalsePositiveRate();
+    String trackerHBaseTable = config.getTrackerHBaseTable();
+    String trackerHBaseCF = config.getTrackerHBaseCF();
+    long millisecondsBetweenPersist = config.getMillisecondsBetweenPersists();
+    BloomAccessTracker bat = new BloomAccessTracker(hbaseTable, expectedInsertions, falsePositives);
+    Configuration hbaseConfig = HBaseConfiguration.create();
+    try {
+      accessTracker = new PersistentAccessTracker( hbaseTable
+              , UUID.randomUUID().toString()
+              , config.getProvider().getTable(hbaseConfig, trackerHBaseTable)
+              , trackerHBaseCF
+              , bat
+              , millisecondsBetweenPersist
+      );
+      lookup = new EnrichmentLookup(config.getProvider().getTable(hbaseConfig, hbaseTable), config.getHBaseCF(), accessTracker);
+    } catch (IOException e) {
+      throw new IllegalStateException("Unable to initialize ThreatIntelAdapter", e);
+    }
+
+    return true;
+  }
+
+  @Override
+  public void cleanup() {
+    try {
+      lookup.close();
+    } catch (Exception e) {
+      throw new RuntimeException("Unable to cleanup access tracker", e);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/9621c55e/metron-streaming/Metron-EnrichmentAdapters/src/main/java/org/apache/metron/enrichment/adapters/threatintel/ThreatIntelConfig.java
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-EnrichmentAdapters/src/main/java/org/apache/metron/enrichment/adapters/threatintel/ThreatIntelConfig.java b/metron-streaming/Metron-EnrichmentAdapters/src/main/java/org/apache/metron/enrichment/adapters/threatintel/ThreatIntelConfig.java
new file mode 100644
index 0000000..ff29aea
--- /dev/null
+++ b/metron-streaming/Metron-EnrichmentAdapters/src/main/java/org/apache/metron/enrichment/adapters/threatintel/ThreatIntelConfig.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.enrichment.adapters.threatintel;
+
+import org.apache.metron.enrichment.utils.EnrichmentUtils;
+import org.apache.metron.hbase.HTableProvider;
+import org.apache.metron.hbase.TableProvider;
+
+import java.io.Serializable;
+import java.lang.reflect.InvocationTargetException;
+
+public class ThreatIntelConfig implements Serializable {
+  public static final long MS_IN_HOUR = 10000*60*60;
+  private String hBaseTable;
+  private String hBaseCF;
+  private double falsePositiveRate = 0.03;
+  private int expectedInsertions = 100000;
+  private String trackerHBaseTable;
+  private String trackerHBaseCF;
+  private long millisecondsBetweenPersists = 2*MS_IN_HOUR;
+  private TableProvider provider = new HTableProvider();
+
+  public String getHBaseTable() {
+    return hBaseTable;
+  }
+
+  public int getExpectedInsertions() {
+    return expectedInsertions;
+  }
+
+  public double getFalsePositiveRate() {
+    return falsePositiveRate;
+  }
+
+  public String getTrackerHBaseTable() {
+    return trackerHBaseTable;
+  }
+
+  public String getTrackerHBaseCF() {
+    return trackerHBaseCF;
+  }
+
+  public long getMillisecondsBetweenPersists() {
+    return millisecondsBetweenPersists;
+  }
+
+  public String getHBaseCF() {
+    return hBaseCF;
+  }
+
+  public TableProvider getProvider() {
+    return provider;
+  }
+
+  public ThreatIntelConfig withProviderImpl(String connectorImpl) {
+    provider = EnrichmentUtils.getTableProvider(connectorImpl, new HTableProvider());
+    return this;
+  }
+
+  public ThreatIntelConfig withTrackerHBaseTable(String hBaseTable) {
+    this.trackerHBaseTable = hBaseTable;
+    return this;
+  }
+
+  public ThreatIntelConfig withTrackerHBaseCF(String cf) {
+    this.trackerHBaseCF = cf;
+    return this;
+  }
+  public ThreatIntelConfig withHBaseTable(String hBaseTable) {
+    this.hBaseTable = hBaseTable;
+    return this;
+  }
+
+  public ThreatIntelConfig withHBaseCF(String cf) {
+    this.hBaseCF= cf;
+    return this;
+  }
+
+  public ThreatIntelConfig withFalsePositiveRate(double falsePositiveRate) {
+    this.falsePositiveRate = falsePositiveRate;
+    return this;
+  }
+
+  public ThreatIntelConfig withExpectedInsertions(int expectedInsertions) {
+    this.expectedInsertions = expectedInsertions;
+    return this;
+  }
+
+  public ThreatIntelConfig withMillisecondsBetweenPersists(long millisecondsBetweenPersists) {
+    this.millisecondsBetweenPersists = millisecondsBetweenPersists;
+    return this;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/9621c55e/metron-streaming/Metron-EnrichmentAdapters/src/main/java/org/apache/metron/enrichment/adapters/whois/WhoisHBaseAdapter.java
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-EnrichmentAdapters/src/main/java/org/apache/metron/enrichment/adapters/whois/WhoisHBaseAdapter.java b/metron-streaming/Metron-EnrichmentAdapters/src/main/java/org/apache/metron/enrichment/adapters/whois/WhoisHBaseAdapter.java
index 442e609..8f0f589 100644
--- a/metron-streaming/Metron-EnrichmentAdapters/src/main/java/org/apache/metron/enrichment/adapters/whois/WhoisHBaseAdapter.java
+++ b/metron-streaming/Metron-EnrichmentAdapters/src/main/java/org/apache/metron/enrichment/adapters/whois/WhoisHBaseAdapter.java
@@ -29,6 +29,7 @@ import org.apache.hadoop.hbase.client.HConnection;
 import org.apache.hadoop.hbase.client.HConnectionManager;
 import org.apache.hadoop.hbase.client.HTableInterface;
 import org.apache.hadoop.hbase.client.Result;
+import org.apache.metron.enrichment.bolt.CacheKey;
 import org.apache.metron.enrichment.interfaces.EnrichmentAdapter;
 import org.json.simple.JSONObject;
 
@@ -36,7 +37,7 @@ import org.apache.metron.tldextractor.BasicTldExtractor;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-public class WhoisHBaseAdapter implements EnrichmentAdapter<String>,
+public class WhoisHBaseAdapter implements EnrichmentAdapter<CacheKey>,
 				Serializable {
 
 	protected static final Logger LOG = LoggerFactory
@@ -80,7 +81,7 @@ public class WhoisHBaseAdapter implements EnrichmentAdapter<String>,
 
 			LOG.trace("--------CONNECTED TO TABLE: " + table);
 
-			JSONObject tester = enrich("cisco.com");
+			JSONObject tester = enrich(new CacheKey("whois", "cisco.com", null));
 
 			if (tester.keySet().size() == 0)
 				throw new IOException(
@@ -96,13 +97,13 @@ public class WhoisHBaseAdapter implements EnrichmentAdapter<String>,
 	}
 
 	@Override
-	public void logAccess(String value) {
+	public void logAccess(CacheKey value) {
 
 	}
 
 	@SuppressWarnings({ "unchecked", "deprecation" })
-	public JSONObject enrich(String metadataIn) {
-		
+	public JSONObject enrich(CacheKey k) {
+		String metadataIn = k.getValue();
 		String metadata = tldex.extract2LD(metadataIn);
 
 		LOG.trace("[Metron] Pinging HBase For:" + metadata);

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/9621c55e/metron-streaming/Metron-EnrichmentAdapters/src/main/java/org/apache/metron/enrichment/bolt/CacheKey.java
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-EnrichmentAdapters/src/main/java/org/apache/metron/enrichment/bolt/CacheKey.java b/metron-streaming/Metron-EnrichmentAdapters/src/main/java/org/apache/metron/enrichment/bolt/CacheKey.java
new file mode 100644
index 0000000..5d5b1e1
--- /dev/null
+++ b/metron-streaming/Metron-EnrichmentAdapters/src/main/java/org/apache/metron/enrichment/bolt/CacheKey.java
@@ -0,0 +1,73 @@
+/**
+ * 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.enrichment.bolt;
+
+import org.apache.metron.domain.SensorEnrichmentConfig;
+
+public class CacheKey {
+  private String field;
+  private String value;
+  private SensorEnrichmentConfig config;
+
+  public CacheKey(String field, String value, SensorEnrichmentConfig config) {
+    this.field = field;
+    this.value = value;
+    this.config = config;
+  }
+
+  public String getField() {
+    return field;
+  }
+
+  public String getValue() {
+    return value;
+  }
+
+  public SensorEnrichmentConfig getConfig() {
+    return config;
+  }
+
+  @Override
+  public String toString() {
+    return "CacheKey{" +
+            "field='" + field + '\'' +
+            ", value='" + value + '\'' +
+            '}';
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) return true;
+    if (o == null || getClass() != o.getClass()) return false;
+
+    CacheKey cacheKey = (CacheKey) o;
+
+    if (getField() != null ? !getField().equals(cacheKey.getField()) : cacheKey.getField() != null) return false;
+    if (getValue() != null ? !getValue().equals(cacheKey.getValue()) : cacheKey.getValue() != null) return false;
+    return config != null ? config.equals(cacheKey.config) : cacheKey.config == null;
+
+  }
+
+  @Override
+  public int hashCode() {
+    int result = getField() != null ? getField().hashCode() : 0;
+    result = 31 * result + (getValue() != null ? getValue().hashCode() : 0);
+    result = 31 * result + (config != null ? config.hashCode() : 0);
+    return result;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/9621c55e/metron-streaming/Metron-EnrichmentAdapters/src/main/java/org/apache/metron/enrichment/bolt/EnrichmentJoinBolt.java
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-EnrichmentAdapters/src/main/java/org/apache/metron/enrichment/bolt/EnrichmentJoinBolt.java b/metron-streaming/Metron-EnrichmentAdapters/src/main/java/org/apache/metron/enrichment/bolt/EnrichmentJoinBolt.java
index bfb4d91..9b47d71 100644
--- a/metron-streaming/Metron-EnrichmentAdapters/src/main/java/org/apache/metron/enrichment/bolt/EnrichmentJoinBolt.java
+++ b/metron-streaming/Metron-EnrichmentAdapters/src/main/java/org/apache/metron/enrichment/bolt/EnrichmentJoinBolt.java
@@ -20,6 +20,7 @@ package org.apache.metron.enrichment.bolt;
 import backtype.storm.task.TopologyContext;
 import org.apache.metron.bolt.JoinBolt;
 import org.apache.metron.domain.Enrichment;
+import org.apache.metron.domain.SensorEnrichmentConfig;
 import org.apache.metron.topology.TopologyUtils;
 import org.json.simple.JSONObject;
 import org.slf4j.Logger;
@@ -56,8 +57,11 @@ public class EnrichmentJoinBolt extends JoinBolt<JSONObject> {
   public Set<String> getStreamIds(JSONObject message) {
     Set<String> streamIds = new HashSet<>();
     String sourceType = TopologyUtils.getSensorType(message);
-    for (String enrichmentType : getFieldMap(sourceType).keySet()) {
-      streamIds.add(enrichmentType);
+    Map<String, List<String>>  fieldMap = getFieldMap(sourceType);
+    if(fieldMap != null) {
+      for (String enrichmentType : getFieldMap(sourceType).keySet()) {
+        streamIds.add(enrichmentType);
+      }
     }
     streamIds.add("message");
     return streamIds;
@@ -85,7 +89,19 @@ public class EnrichmentJoinBolt extends JoinBolt<JSONObject> {
     return message;
   }
 
-  protected Map<String, List<String>> getFieldMap(String sensorType) {
-    return configurations.getSensorEnrichmentConfig(sensorType).getEnrichmentFieldMap();
+  public Map<String, List<String>> getFieldMap(String sourceType) {
+    if(sourceType != null) {
+      SensorEnrichmentConfig config = configurations.getSensorEnrichmentConfig(sourceType);
+      if (config != null) {
+        return config.getEnrichmentFieldMap();
+      }
+      else {
+        LOG.error("Unable to retrieve a sensor enrichment config of " + sourceType);
+      }
+    }
+    else {
+      LOG.error("Trying to retrieve a field map with source type of null");
+    }
+    return null;
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/9621c55e/metron-streaming/Metron-EnrichmentAdapters/src/main/java/org/apache/metron/enrichment/bolt/GenericEnrichmentBolt.java
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-EnrichmentAdapters/src/main/java/org/apache/metron/enrichment/bolt/GenericEnrichmentBolt.java b/metron-streaming/Metron-EnrichmentAdapters/src/main/java/org/apache/metron/enrichment/bolt/GenericEnrichmentBolt.java
index 08b223c..7b76c57 100644
--- a/metron-streaming/Metron-EnrichmentAdapters/src/main/java/org/apache/metron/enrichment/bolt/GenericEnrichmentBolt.java
+++ b/metron-streaming/Metron-EnrichmentAdapters/src/main/java/org/apache/metron/enrichment/bolt/GenericEnrichmentBolt.java
@@ -30,6 +30,7 @@ import com.google.common.cache.LoadingCache;
 import org.apache.metron.Constants;
 import org.apache.metron.bolt.ConfiguredBolt;
 import org.apache.metron.domain.Enrichment;
+import org.apache.metron.domain.SensorEnrichmentConfig;
 import org.apache.metron.enrichment.interfaces.EnrichmentAdapter;
 import org.apache.metron.helpers.topology.ErrorUtils;
 import org.json.simple.JSONObject;
@@ -65,13 +66,13 @@ public class GenericEnrichmentBolt extends ConfiguredBolt {
           .getLogger(GenericEnrichmentBolt.class);
   private OutputCollector collector;
 
-
   protected String enrichmentType;
-  protected EnrichmentAdapter adapter;
-  protected transient CacheLoader<String, JSONObject> loader;
-  protected transient LoadingCache<String, JSONObject> cache;
+  protected EnrichmentAdapter<CacheKey> adapter;
+  protected transient CacheLoader<CacheKey, JSONObject> loader;
+  protected transient LoadingCache<CacheKey, JSONObject> cache;
   protected Long maxCacheSize;
   protected Long maxTimeRetain;
+  protected boolean invalidateCacheOnReload = false;
 
   public GenericEnrichmentBolt(String zookeeperUrl) {
     super(zookeeperUrl);
@@ -108,9 +109,23 @@ public class GenericEnrichmentBolt extends ConfiguredBolt {
     return this;
   }
 
+  public GenericEnrichmentBolt withCacheInvalidationOnReload(boolean cacheInvalidationOnReload) {
+    this.invalidateCacheOnReload= cacheInvalidationOnReload;
+    return this;
+  }
+  @Override
+  protected void reloadCallback() {
+    if(invalidateCacheOnReload) {
+      if (cache != null) {
+        cache.invalidateAll();
+      }
+    }
+  }
+
   @Override
   public void prepare(Map conf, TopologyContext topologyContext,
                       OutputCollector collector) {
+    super.prepare(conf, topologyContext, collector);
     this.collector = collector;
     if (this.maxCacheSize == null)
       throw new IllegalStateException("MAX_CACHE_SIZE_OBJECTS_NUM must be specified");
@@ -118,8 +133,8 @@ public class GenericEnrichmentBolt extends ConfiguredBolt {
       throw new IllegalStateException("MAX_TIME_RETAIN_MINUTES must be specified");
     if (this.adapter == null)
       throw new IllegalStateException("Adapter must be specified");
-    loader = new CacheLoader<String, JSONObject>() {
-      public JSONObject load(String key) throws Exception {
+    loader = new CacheLoader<CacheKey, JSONObject>() {
+      public JSONObject load(CacheKey key) throws Exception {
         return adapter.enrich(key);
       }
     };
@@ -144,6 +159,7 @@ public class GenericEnrichmentBolt extends ConfiguredBolt {
   public void execute(Tuple tuple) {
     String key = tuple.getStringByField("key");
     JSONObject rawMessage = (JSONObject) tuple.getValueByField("message");
+
     JSONObject enrichedMessage = new JSONObject();
     enrichedMessage.put("adapter." + adapter.getClass().getSimpleName().toLowerCase() + ".begin.ts", "" + System.currentTimeMillis());
     try {
@@ -151,6 +167,13 @@ public class GenericEnrichmentBolt extends ConfiguredBolt {
         throw new Exception("Could not parse binary stream to JSON");
       if (key == null)
         throw new Exception("Key is not valid");
+      String sourceType = null;
+      if(rawMessage.containsKey(Constants.SENSOR_TYPE)) {
+        sourceType = rawMessage.get(Constants.SENSOR_TYPE).toString();
+      }
+      else {
+        throw new RuntimeException("Source type is missing from enrichment fragment: " + rawMessage.toJSONString());
+      }
       for (Object o : rawMessage.keySet()) {
         String field = (String) o;
         String value = (String) rawMessage.get(field);
@@ -159,8 +182,13 @@ public class GenericEnrichmentBolt extends ConfiguredBolt {
         } else {
           JSONObject enrichedField = new JSONObject();
           if (value != null && value.length() != 0) {
-            adapter.logAccess(value);
-            enrichedField = cache.getUnchecked(value);
+            SensorEnrichmentConfig config = configurations.getSensorEnrichmentConfig(sourceType);
+            if(config == null) {
+              throw new RuntimeException("Unable to find " + config);
+            }
+            CacheKey cacheKey= new CacheKey(field, value, config);
+            adapter.logAccess(cacheKey);
+            enrichedField = cache.getUnchecked(cacheKey);
             if (enrichedField == null)
               throw new Exception("[Metron] Could not enrich string: "
                       + value);

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/9621c55e/metron-streaming/Metron-EnrichmentAdapters/src/main/java/org/apache/metron/enrichment/bolt/ThreatIntelJoinBolt.java
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-EnrichmentAdapters/src/main/java/org/apache/metron/enrichment/bolt/ThreatIntelJoinBolt.java b/metron-streaming/Metron-EnrichmentAdapters/src/main/java/org/apache/metron/enrichment/bolt/ThreatIntelJoinBolt.java
index 014e0a9..d0bc833 100644
--- a/metron-streaming/Metron-EnrichmentAdapters/src/main/java/org/apache/metron/enrichment/bolt/ThreatIntelJoinBolt.java
+++ b/metron-streaming/Metron-EnrichmentAdapters/src/main/java/org/apache/metron/enrichment/bolt/ThreatIntelJoinBolt.java
@@ -17,6 +17,7 @@
  */
 package org.apache.metron.enrichment.bolt;
 
+import org.apache.metron.domain.SensorEnrichmentConfig;
 import org.json.simple.JSONObject;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -34,8 +35,15 @@ public class ThreatIntelJoinBolt extends EnrichmentJoinBolt {
   }
 
   @Override
-  public Map<String, List<String>> getFieldMap(String sensorType) {
-    return configurations.getSensorEnrichmentConfig(sensorType).getThreatIntelFieldMap();
+  public Map<String, List<String>> getFieldMap(String sourceType) {
+    SensorEnrichmentConfig config = configurations.getSensorEnrichmentConfig(sourceType);
+    if(config != null) {
+      return config.getThreatIntelFieldMap();
+    }
+    else {
+      LOG.error("Unable to retrieve sensor config: " + sourceType);
+      return null;
+    }
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/9621c55e/metron-streaming/Metron-EnrichmentAdapters/src/main/java/org/apache/metron/enrichment/utils/EnrichmentUtils.java
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-EnrichmentAdapters/src/main/java/org/apache/metron/enrichment/utils/EnrichmentUtils.java b/metron-streaming/Metron-EnrichmentAdapters/src/main/java/org/apache/metron/enrichment/utils/EnrichmentUtils.java
index 228f844..7a58673 100644
--- a/metron-streaming/Metron-EnrichmentAdapters/src/main/java/org/apache/metron/enrichment/utils/EnrichmentUtils.java
+++ b/metron-streaming/Metron-EnrichmentAdapters/src/main/java/org/apache/metron/enrichment/utils/EnrichmentUtils.java
@@ -17,7 +17,16 @@
  */
 package org.apache.metron.enrichment.utils;
 
+import com.google.common.base.Function;
 import com.google.common.base.Joiner;
+import com.google.common.base.Splitter;
+import com.google.common.collect.Iterables;
+import org.apache.metron.hbase.HTableProvider;
+import org.apache.metron.hbase.TableProvider;
+import org.apache.metron.hbase.converters.enrichment.EnrichmentKey;
+
+import javax.annotation.Nullable;
+import java.lang.reflect.InvocationTargetException;
 
 public class EnrichmentUtils {
 
@@ -27,6 +36,46 @@ public class EnrichmentUtils {
     return Joiner.on(".").join(new String[]{KEY_PREFIX, enrichmentName, field});
   }
 
+  public static class TypeToKey implements Function<String, EnrichmentKey> {
+    private final String indicator;
+
+    public TypeToKey(String indicator) {
+      this.indicator = indicator;
+
+    }
+    @Nullable
+    @Override
+    public EnrichmentKey apply(@Nullable String enrichmentType) {
+      return new EnrichmentKey(enrichmentType, indicator);
+    }
+  }
+  public static String toTopLevelField(String field) {
+    if(field == null) {
+      return null;
+    }
+    return Iterables.getLast(Splitter.on('.').split(field));
+  }
 
+  public static TableProvider getTableProvider(String connectorImpl, TableProvider defaultImpl) {
+    if(connectorImpl == null || connectorImpl.length() == 0 || connectorImpl.charAt(0) == '$') {
+      return defaultImpl;
+    }
+    else {
+      try {
+        Class<? extends TableProvider> clazz = (Class<? extends TableProvider>) Class.forName(connectorImpl);
+        return clazz.getConstructor().newInstance();
+      } catch (InstantiationException e) {
+        throw new IllegalStateException("Unable to instantiate connector.", e);
+      } catch (IllegalAccessException e) {
+        throw new IllegalStateException("Unable to instantiate connector: illegal access", e);
+      } catch (InvocationTargetException e) {
+        throw new IllegalStateException("Unable to instantiate connector", e);
+      } catch (NoSuchMethodException e) {
+        throw new IllegalStateException("Unable to instantiate connector: no such method", e);
+      } catch (ClassNotFoundException e) {
+        throw new IllegalStateException("Unable to instantiate connector: class not found", e);
+      }
+    }
+  }
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/9621c55e/metron-streaming/Metron-EnrichmentAdapters/src/main/java/org/apache/metron/threatintel/ThreatIntelAdapter.java
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-EnrichmentAdapters/src/main/java/org/apache/metron/threatintel/ThreatIntelAdapter.java b/metron-streaming/Metron-EnrichmentAdapters/src/main/java/org/apache/metron/threatintel/ThreatIntelAdapter.java
deleted file mode 100644
index b95f4b8..0000000
--- a/metron-streaming/Metron-EnrichmentAdapters/src/main/java/org/apache/metron/threatintel/ThreatIntelAdapter.java
+++ /dev/null
@@ -1,108 +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.threatintel;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.HBaseConfiguration;
-import org.apache.metron.enrichment.interfaces.EnrichmentAdapter;
-import org.apache.metron.hbase.converters.threatintel.ThreatIntelKey;
-import org.apache.metron.reference.lookup.accesstracker.BloomAccessTracker;
-import org.apache.metron.reference.lookup.accesstracker.PersistentAccessTracker;
-import org.apache.metron.threatintel.hbase.ThreatIntelLookup;
-import org.json.simple.JSONObject;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-import java.io.Serializable;
-import java.util.UUID;
-
-public class ThreatIntelAdapter implements EnrichmentAdapter<String>,Serializable {
-    protected static final Logger _LOG = LoggerFactory.getLogger(ThreatIntelAdapter.class);
-    protected ThreatIntelConfig config;
-    protected ThreatIntelLookup lookup;
-
-    public ThreatIntelAdapter() {
-    }
-    public ThreatIntelAdapter(ThreatIntelConfig config) {
-        withConfig(config);
-    }
-
-    public ThreatIntelAdapter withConfig(ThreatIntelConfig config) {
-        this.config = config;
-        return this;
-    }
-
-    @Override
-    public void logAccess(String value) {
-        lookup.getAccessTracker().logAccess(new ThreatIntelKey(value));
-    }
-
-    @Override
-    public JSONObject enrich(String value) {
-        JSONObject enriched = new JSONObject();
-        boolean isThreat = false;
-        try {
-            isThreat = lookup.exists(new ThreatIntelKey(value), lookup.getTable(), false);
-        } catch (IOException e) {
-            throw new RuntimeException("Unable to retrieve value", e);
-        }
-        if(isThreat) {
-            enriched.put(config.getHBaseTable(), "alert");
-            _LOG.trace("Enriched value => " + enriched);
-        }
-        //throw new RuntimeException("Unable to retrieve value " + value);
-        return enriched;
-    }
-
-    @Override
-    public boolean initializeAdapter() {
-        PersistentAccessTracker accessTracker;
-        String hbaseTable = config.getHBaseTable();
-        int expectedInsertions = config.getExpectedInsertions();
-        double falsePositives = config.getFalsePositiveRate();
-        String trackerHBaseTable = config.getTrackerHBaseTable();
-        String trackerHBaseCF = config.getTrackerHBaseCF();
-        long millisecondsBetweenPersist = config.getMillisecondsBetweenPersists();
-        BloomAccessTracker bat = new BloomAccessTracker(hbaseTable, expectedInsertions, falsePositives);
-        Configuration hbaseConfig = HBaseConfiguration.create();
-        try {
-            accessTracker = new PersistentAccessTracker( hbaseTable
-                                                        , UUID.randomUUID().toString()
-                                                        , config.getProvider().getTable(hbaseConfig, trackerHBaseTable)
-                                                        , trackerHBaseCF
-                                                        , bat
-                                                        , millisecondsBetweenPersist
-                                                        );
-            lookup = new ThreatIntelLookup(config.getProvider().getTable(hbaseConfig, hbaseTable), config.getHBaseCF(), accessTracker);
-        } catch (IOException e) {
-            throw new IllegalStateException("Unable to initialize ThreatIntelAdapter", e);
-        }
-
-        return true;
-    }
-
-    @Override
-    public void cleanup() {
-        try {
-            lookup.close();
-        } catch (Exception e) {
-            throw new RuntimeException("Unable to cleanup access tracker", e);
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/9621c55e/metron-streaming/Metron-EnrichmentAdapters/src/main/java/org/apache/metron/threatintel/ThreatIntelConfig.java
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-EnrichmentAdapters/src/main/java/org/apache/metron/threatintel/ThreatIntelConfig.java b/metron-streaming/Metron-EnrichmentAdapters/src/main/java/org/apache/metron/threatintel/ThreatIntelConfig.java
deleted file mode 100644
index bb45468..0000000
--- a/metron-streaming/Metron-EnrichmentAdapters/src/main/java/org/apache/metron/threatintel/ThreatIntelConfig.java
+++ /dev/null
@@ -1,125 +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.threatintel;
-
-import org.apache.metron.hbase.HTableProvider;
-import org.apache.metron.hbase.TableProvider;
-
-import java.io.Serializable;
-import java.lang.reflect.InvocationTargetException;
-
-public class ThreatIntelConfig implements Serializable {
-    public static final long MS_IN_HOUR = 10000*60*60;
-    private String hBaseTable;
-    private String hBaseCF;
-    private double falsePositiveRate = 0.03;
-    private int expectedInsertions = 100000;
-    private String trackerHBaseTable;
-    private String trackerHBaseCF;
-    private long millisecondsBetweenPersists = 2*MS_IN_HOUR;
-    private TableProvider provider = new HTableProvider();
-
-    public String getHBaseTable() {
-        return hBaseTable;
-    }
-
-    public int getExpectedInsertions() {
-        return expectedInsertions;
-    }
-
-    public double getFalsePositiveRate() {
-        return falsePositiveRate;
-    }
-
-    public String getTrackerHBaseTable() {
-        return trackerHBaseTable;
-    }
-
-    public String getTrackerHBaseCF() {
-        return trackerHBaseCF;
-    }
-
-    public long getMillisecondsBetweenPersists() {
-        return millisecondsBetweenPersists;
-    }
-
-    public String getHBaseCF() {
-        return hBaseCF;
-    }
-
-    public TableProvider getProvider() {
-        return provider;
-    }
-
-    public ThreatIntelConfig withProviderImpl(String connectorImpl) {
-        if(connectorImpl == null || connectorImpl.length() == 0 || connectorImpl.charAt(0) == '$') {
-            provider = new HTableProvider();
-        }
-        else {
-            try {
-                Class<? extends TableProvider> clazz = (Class<? extends TableProvider>) Class.forName(connectorImpl);
-                provider = clazz.getConstructor().newInstance();
-            } catch (InstantiationException e) {
-                throw new IllegalStateException("Unable to instantiate connector.", e);
-            } catch (IllegalAccessException e) {
-                throw new IllegalStateException("Unable to instantiate connector: illegal access", e);
-            } catch (InvocationTargetException e) {
-                throw new IllegalStateException("Unable to instantiate connector", e);
-            } catch (NoSuchMethodException e) {
-                throw new IllegalStateException("Unable to instantiate connector: no such method", e);
-            } catch (ClassNotFoundException e) {
-                throw new IllegalStateException("Unable to instantiate connector: class not found", e);
-            }
-        }
-        return this;
-    }
-
-    public ThreatIntelConfig withTrackerHBaseTable(String hBaseTable) {
-        this.trackerHBaseTable = hBaseTable;
-        return this;
-    }
-
-    public ThreatIntelConfig withTrackerHBaseCF(String cf) {
-        this.trackerHBaseCF = cf;
-        return this;
-    }
-    public ThreatIntelConfig withHBaseTable(String hBaseTable) {
-        this.hBaseTable = hBaseTable;
-        return this;
-    }
-
-    public ThreatIntelConfig withHBaseCF(String cf) {
-        this.hBaseCF= cf;
-        return this;
-    }
-
-    public ThreatIntelConfig withFalsePositiveRate(double falsePositiveRate) {
-        this.falsePositiveRate = falsePositiveRate;
-        return this;
-    }
-
-    public ThreatIntelConfig withExpectedInsertions(int expectedInsertions) {
-        this.expectedInsertions = expectedInsertions;
-        return this;
-    }
-
-    public ThreatIntelConfig withMillisecondsBetweenPersists(long millisecondsBetweenPersists) {
-        this.millisecondsBetweenPersists = millisecondsBetweenPersists;
-        return this;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/9621c55e/metron-streaming/Metron-EnrichmentAdapters/src/test/java/org/apache/metron/enrichment/adapters/cif/CIFHbaseAdapterTest.java
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-EnrichmentAdapters/src/test/java/org/apache/metron/enrichment/adapters/cif/CIFHbaseAdapterTest.java b/metron-streaming/Metron-EnrichmentAdapters/src/test/java/org/apache/metron/enrichment/adapters/cif/CIFHbaseAdapterTest.java
index 6421a21..6ccc21e 100644
--- a/metron-streaming/Metron-EnrichmentAdapters/src/test/java/org/apache/metron/enrichment/adapters/cif/CIFHbaseAdapterTest.java
+++ b/metron-streaming/Metron-EnrichmentAdapters/src/test/java/org/apache/metron/enrichment/adapters/cif/CIFHbaseAdapterTest.java
@@ -19,6 +19,7 @@ package org.apache.metron.enrichment.adapters.cif;
 import java.net.InetAddress;
 import java.util.Properties;
 
+import org.apache.metron.enrichment.bolt.CacheKey;
 import org.apache.metron.enrichment.interfaces.EnrichmentAdapter;
 import org.apache.metron.test.AbstractTestContext;
 import org.junit.Assert;
@@ -174,17 +175,17 @@ public class CIFHbaseAdapterTest extends AbstractTestContext {
     }
 
     /**
-     * Test method for {@link org.apache.metron.enrichment.adapters.cif.CIFHbaseAdapter#enrich(java.lang.String)}.
+     * Test method for {@link org.apache.metron.enrichment.adapters.cif.CIFHbaseAdapter#enrich(CacheKey)}.
      */
     public void testEnrich() {
         if(skipTests(this.getMode())){
             return;//skip tests
        }else{
             cifHbaseAdapter.initializeAdapter();
-            Assert.assertNotNull(cifHbaseAdapter.enrich("testinvalid.metadata"));
+            Assert.assertNotNull(cifHbaseAdapter.enrich(new CacheKey("cif", "testinvalid.metadata", null)));
             
-            Assert.assertNotNull(cifHbaseAdapter.enrich("ivalid.ip"));
-            Assert.assertNotNull(cifHbaseAdapter.enrich("1.1.1.10"));
+            Assert.assertNotNull(cifHbaseAdapter.enrich(new CacheKey("cif", "ivalid.ip", null)));
+            Assert.assertNotNull(cifHbaseAdapter.enrich(new CacheKey("cif", "1.1.1.10", null)));
        }
     }
     

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/9621c55e/metron-streaming/Metron-EnrichmentAdapters/src/test/java/org/apache/metron/enrichment/adapters/geo/GeoMysqlAdapterTest.java
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-EnrichmentAdapters/src/test/java/org/apache/metron/enrichment/adapters/geo/GeoMysqlAdapterTest.java b/metron-streaming/Metron-EnrichmentAdapters/src/test/java/org/apache/metron/enrichment/adapters/geo/GeoMysqlAdapterTest.java
index 9a0fe2a..8898c64 100644
--- a/metron-streaming/Metron-EnrichmentAdapters/src/test/java/org/apache/metron/enrichment/adapters/geo/GeoMysqlAdapterTest.java
+++ b/metron-streaming/Metron-EnrichmentAdapters/src/test/java/org/apache/metron/enrichment/adapters/geo/GeoMysqlAdapterTest.java
@@ -21,6 +21,7 @@ import java.util.Properties;
 
 import org.apache.metron.enrichment.adapters.jdbc.JdbcAdapter;
 import org.apache.metron.enrichment.adapters.jdbc.MySqlConfig;
+import org.apache.metron.enrichment.bolt.CacheKey;
 import org.apache.metron.enrichment.interfaces.EnrichmentAdapter;
 import org.json.simple.JSONObject;
 
@@ -113,7 +114,7 @@ public class GeoMysqlAdapterTest extends AbstractSchemaTest {
        }else{
            
          try {           
-                JSONObject json = geoMySqlAdapter.enrich("72.163.4.161");
+                JSONObject json = geoMySqlAdapter.enrich(new CacheKey("dummy", "72.163.4.161", null));
                 
                 //assert Geo Response is not null
                 System.out.println("json ="+json);

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/9621c55e/metron-streaming/Metron-EnrichmentAdapters/src/test/java/org/apache/metron/enrichment/adapters/whois/WhoisHBaseAdapterTest.java
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-EnrichmentAdapters/src/test/java/org/apache/metron/enrichment/adapters/whois/WhoisHBaseAdapterTest.java b/metron-streaming/Metron-EnrichmentAdapters/src/test/java/org/apache/metron/enrichment/adapters/whois/WhoisHBaseAdapterTest.java
index a56f5ba..ccf2ea3 100644
--- a/metron-streaming/Metron-EnrichmentAdapters/src/test/java/org/apache/metron/enrichment/adapters/whois/WhoisHBaseAdapterTest.java
+++ b/metron-streaming/Metron-EnrichmentAdapters/src/test/java/org/apache/metron/enrichment/adapters/whois/WhoisHBaseAdapterTest.java
@@ -19,6 +19,7 @@ package org.apache.metron.enrichment.adapters.whois;
 import java.net.InetAddress;
 import java.util.Properties;
 
+import org.apache.metron.enrichment.bolt.CacheKey;
 import org.apache.metron.enrichment.interfaces.EnrichmentAdapter;
 import org.json.simple.JSONObject;
 
@@ -125,13 +126,13 @@ public class WhoisHBaseAdapterTest extends AbstractTestContext {
     }
 
     /**
-     * Test method for {@link org.apache.metron.enrichment.adapters.whois.WhoisHBaseAdapter#enrich(java.lang.String)}.
+     * Test method for {@link org.apache.metron.enrichment.adapters.whois.WhoisHBaseAdapter#enrich(CacheKey)}.
      */
     public void testEnrich() {
         if(skipTests(this.getMode())){
             return;//skip tests
        }else{
-            JSONObject json = whoisHbaseAdapter.enrich("72.163.4.161");
+            JSONObject json = whoisHbaseAdapter.enrich(new CacheKey("whois", "72.163.4.161", null));
             
             //assert Geo Response is not null
             Assert.assertNotNull(json);

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/9621c55e/metron-streaming/Metron-MessageParsers/pom.xml
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-MessageParsers/pom.xml b/metron-streaming/Metron-MessageParsers/pom.xml
index a697aa8..2687213 100644
--- a/metron-streaming/Metron-MessageParsers/pom.xml
+++ b/metron-streaming/Metron-MessageParsers/pom.xml
@@ -37,6 +37,17 @@
 			<version>${project.parent.version}</version>
 		</dependency>
 		<dependency>
+            <groupId>org.apache.hadoop</groupId>
+            <artifactId>hadoop-common</artifactId>
+            <version>${global_hadoop_version}</version>
+            <exclusions>
+                <exclusion>
+                    <artifactId>servlet-api</artifactId>
+                    <groupId>javax.servlet</groupId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+		<dependency>
 			<groupId>org.apache.storm</groupId>
 			<artifactId>storm-core</artifactId>
 			<version>${global_storm_version}</version>
@@ -56,11 +67,12 @@
 			<groupId>junit</groupId>
 			<artifactId>junit</artifactId>
 			<version>${global_junit_version}</version>
+			<scope>test</scope>
 		</dependency>
 		<dependency>
 			<groupId>com.google.guava</groupId>
 			<artifactId>guava</artifactId>
-			<version>${global_guava_version}</version>
+			<version>${global_hbase_guava_version}</version>
 		</dependency>
 
 		<dependency>

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/9621c55e/metron-streaming/Metron-Solr/pom.xml
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-Solr/pom.xml b/metron-streaming/Metron-Solr/pom.xml
index cbb7395..925c219 100644
--- a/metron-streaming/Metron-Solr/pom.xml
+++ b/metron-streaming/Metron-Solr/pom.xml
@@ -27,6 +27,11 @@
     </properties>
     <dependencies>
         <dependency>
+            <groupId>com.google.guava</groupId>
+            <artifactId>guava</artifactId>
+            <version>${global_hbase_guava_version}</version>
+        </dependency>
+        <dependency>
             <groupId>org.apache.metron</groupId>
             <artifactId>Metron-Common</artifactId>
             <version>${project.parent.version}</version>