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/03/21 18:05:51 UTC

[09/43] incubator-metron git commit: METRON-50: Ingest threat intel data from Taxii feeds closes apache/incubator-metron#29

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/0e1055aa/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
new file mode 100644
index 0000000..4f307c5
--- /dev/null
+++ b/metron-streaming/Metron-DataLoads/src/main/java/org/apache/metron/dataloads/taxii/TaxiiLoader.java
@@ -0,0 +1,180 @@
+/**
+ * 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/0e1055aa/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 519b95a..a124710 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,7 +18,9 @@
 package org.apache.metron.dataloads.extractor;
 
 import com.google.common.collect.Iterables;
-import org.apache.metron.threatintel.ThreatIntelKey;
+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.apache.metron.threatintel.ThreatIntelResults;
 import org.junit.Assert;
 import org.junit.Test;
@@ -33,12 +35,12 @@ public class ExtractorTest {
     {
 
         @Override
-        public Iterable<ThreatIntelResults> extract(String line) throws IOException {
+        public Iterable<LookupKV> extract(String line) throws IOException {
             ThreatIntelKey key = new ThreatIntelKey();
             key.indicator = "dummy";
             Map<String, String> value = new HashMap<>();
             value.put("indicator", "dummy");
-            return Arrays.asList(new ThreatIntelResults(key, value));
+            return Arrays.asList(new LookupKV(key, new ThreatIntelValue(value)));
         }
 
         @Override
@@ -49,9 +51,11 @@ public class ExtractorTest {
     @Test
     public void testDummyExtractor() throws IllegalAccessException, InstantiationException, ClassNotFoundException, IOException {
         Extractor extractor = Extractors.create(DummyExtractor.class.getName());
-        ThreatIntelResults results = Iterables.getFirst(extractor.extract(null), null);
-        Assert.assertEquals("dummy", results.getKey().indicator);
-        Assert.assertEquals("dummy", results.getValue().get("indicator"));
+        LookupKV results = Iterables.getFirst(extractor.extract(null), null);
+        ThreatIntelKey key = (ThreatIntelKey) results.getKey();
+        ThreatIntelValue value = (ThreatIntelValue) results.getValue();
+        Assert.assertEquals("dummy", key.indicator);
+        Assert.assertEquals("dummy", value.getMetadata().get("indicator"));
     }
 
     @Test
@@ -68,8 +72,10 @@ public class ExtractorTest {
                 "            ,\"extractor\" : \"org.apache.metron.dataloads.extractor.ExtractorTest$DummyExtractor\"\n" +
                 "         }";
         ExtractorHandler handler = ExtractorHandler.load(config);
-        ThreatIntelResults results = Iterables.getFirst(handler.getExtractor().extract(null), null);
-        Assert.assertEquals("dummy", results.getKey().indicator);
-        Assert.assertEquals("dummy", results.getValue().get("indicator"));
+        LookupKV results = Iterables.getFirst(handler.getExtractor().extract(null), null);
+        ThreatIntelKey key = (ThreatIntelKey) results.getKey();
+        ThreatIntelValue value = (ThreatIntelValue) results.getValue();
+        Assert.assertEquals("dummy", key.indicator);
+        Assert.assertEquals("dummy", value.getMetadata().get("indicator"));
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/0e1055aa/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 96d9046..682b230 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
@@ -19,6 +19,9 @@ package org.apache.metron.dataloads.extractor.csv;
 
 import com.google.common.collect.Iterables;
 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.reference.lookup.LookupKV;
 import org.apache.metron.threatintel.ThreatIntelResults;
 import org.junit.Assert;
 import org.junit.Test;
@@ -102,14 +105,16 @@ public class CSVExtractorTest {
 
     public void validate(ExtractorHandler handler) throws IOException {
         {
-            ThreatIntelResults results = Iterables.getFirst(handler.getExtractor().extract("google.com,1.0,foo"), null);
-            Assert.assertEquals("google.com", results.getKey().indicator);
-            Assert.assertEquals("google.com", results.getValue().get("host"));
-            Assert.assertEquals("foo", results.getValue().get("meta"));
-            Assert.assertEquals(2, results.getValue().size());
+            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<ThreatIntelResults> results = handler.getExtractor().extract("#google.com,1.0,foo");
+            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/0e1055aa/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 f25a5a0..eaf6677 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,15 +18,13 @@
 package org.apache.metron.dataloads.extractor.stix;
 
 import com.google.common.collect.Iterables;
-import org.apache.metron.dataloads.ThreatIntelBulkLoader;
 import org.apache.metron.dataloads.extractor.Extractor;
 import org.apache.metron.dataloads.extractor.ExtractorHandler;
-import org.apache.metron.threatintel.ThreatIntelResults;
+import org.apache.metron.hbase.converters.threatintel.ThreatIntelKey;
+import org.apache.metron.reference.lookup.LookupKV;
 import org.junit.Assert;
 import org.junit.Test;
 
-import java.util.List;
-
 public class StixExtractorTest {
     @Test
     public void testStixAddresses() throws Exception {
@@ -148,11 +146,12 @@ public class StixExtractorTest {
                     "         }";
             ExtractorHandler handler = ExtractorHandler.load(config);
             Extractor extractor = handler.getExtractor();
-            Iterable<ThreatIntelResults> results = extractor.extract(stixDoc);
+            Iterable<LookupKV> results = extractor.extract(stixDoc);
+
             Assert.assertEquals(3, Iterables.size(results));
-            Assert.assertEquals("10.0.0.0", Iterables.get(results, 0).getKey().indicator);
-            Assert.assertEquals("10.0.0.1", Iterables.get(results, 1).getKey().indicator);
-            Assert.assertEquals("10.0.0.2", Iterables.get(results, 2).getKey().indicator);
+            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);
         }
         {
             /**
@@ -169,11 +168,11 @@ public class StixExtractorTest {
                     "         }";
             ExtractorHandler handler = ExtractorHandler.load(config);
             Extractor extractor = handler.getExtractor();
-            Iterable<ThreatIntelResults> results = extractor.extract(stixDoc);
+            Iterable<LookupKV> results = extractor.extract(stixDoc);
             Assert.assertEquals(3, Iterables.size(results));
-            Assert.assertEquals("10.0.0.0", Iterables.get(results, 0).getKey().indicator);
-            Assert.assertEquals("10.0.0.1", Iterables.get(results, 1).getKey().indicator);
-            Assert.assertEquals("10.0.0.2", Iterables.get(results, 2).getKey().indicator);
+            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);
         }
         {
             /**
@@ -192,7 +191,7 @@ public class StixExtractorTest {
                     "         }";
             ExtractorHandler handler = ExtractorHandler.load(config);
             Extractor extractor = handler.getExtractor();
-            Iterable<ThreatIntelResults> results = extractor.extract(stixDoc);
+            Iterable<LookupKV> results = extractor.extract(stixDoc);
             Assert.assertEquals(0, Iterables.size(results));
         }
     }

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/0e1055aa/metron-streaming/Metron-DataLoads/src/test/java/org/apache/metron/dataloads/hbase/HBaseConverterTest.java
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-DataLoads/src/test/java/org/apache/metron/dataloads/hbase/HBaseConverterTest.java b/metron-streaming/Metron-DataLoads/src/test/java/org/apache/metron/dataloads/hbase/HBaseConverterTest.java
deleted file mode 100644
index 3587e59..0000000
--- a/metron-streaming/Metron-DataLoads/src/test/java/org/apache/metron/dataloads/hbase/HBaseConverterTest.java
+++ /dev/null
@@ -1,67 +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.threatintel.hbase.Converter;
-import org.apache.metron.threatintel.ThreatIntelKey;
-import org.apache.metron.threatintel.ThreatIntelResults;
-import org.junit.Assert;
-import org.junit.Test;
-
-import java.io.IOException;
-import java.util.AbstractMap;
-import java.util.HashMap;
-import java.util.Map;
-
-public class HBaseConverterTest {
-    ThreatIntelKey key = new ThreatIntelKey("google");
-    Map<String, String> value = new HashMap<String, String>() {{
-        put("foo", "bar");
-        put("grok", "baz");
-    }};
-    Long timestamp = 7L;
-    ThreatIntelResults results = new ThreatIntelResults(key, value);
-    @Test
-    public void testKeySerialization() {
-        byte[] serialized = key.toBytes();
-        ThreatIntelKey deserialized = ThreatIntelKey.fromBytes(serialized);
-        Assert.assertEquals(key, deserialized);
-    }
-
-    @Test
-    public void testPut() throws IOException {
-        Put put = Converter.INSTANCE.toPut("cf", key, value, timestamp);
-        Map.Entry<ThreatIntelResults, Long> converted= Converter.INSTANCE.fromPut(put, "cf");
-        Assert.assertEquals(new AbstractMap.SimpleEntry<>(results, timestamp), converted);
-    }
-    @Test
-    public void testResult() throws IOException {
-        Result r = Converter.INSTANCE.toResult("cf", key, value, timestamp);
-        Map.Entry<ThreatIntelResults, Long> converted= Converter.INSTANCE.fromResult(r, "cf");
-        Assert.assertEquals(new AbstractMap.SimpleEntry<>(results, timestamp), converted);
-    }
-
-    @Test
-    public void testGet() throws Exception {
-        Get get = Converter.INSTANCE.toGet("cf", key);
-        Assert.assertArrayEquals(key.toBytes(), get.getRow());
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/0e1055aa/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
new file mode 100644
index 0000000..21a1d26
--- /dev/null
+++ b/metron-streaming/Metron-DataLoads/src/test/java/org/apache/metron/dataloads/hbase/HBaseThreatIntelConverterTest.java
@@ -0,0 +1,76 @@
+/**
+ * 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/0e1055aa/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 2824de8..be9ac90 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
@@ -26,9 +26,11 @@ import org.apache.hadoop.hbase.client.Result;
 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.ThreatIntelBulkLoader;
-import org.apache.metron.threatintel.ThreatIntelResults;
-import org.apache.metron.threatintel.hbase.Converter;
+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.reference.lookup.LookupKV;
+import org.apache.metron.hbase.converters.threatintel.ThreatIntelConverter;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
@@ -87,19 +89,19 @@ public class BulkLoadMapperIntegrationTest {
     Assert.assertNotNull(testTable);
     FileSystem fs = FileSystem.get(config);
     String contents = "google.com,1,foo";
+    ThreatIntelConverter converter = new ThreatIntelConverter();
     HBaseUtil.INSTANCE.writeFile(contents, new Path("input.csv"), fs);
-    Job job = ThreatIntelBulkLoader.createJob(config, "input.csv", tableName, cf, extractorConfig, 0L);
+    Job job = ThreatIntelBulkLoader.createJob(config, "input.csv", tableName, cf, extractorConfig, 0L, new ThreatIntelConverter());
     Assert.assertTrue(job.waitForCompletion(true));
     ResultScanner scanner = testTable.getScanner(Bytes.toBytes(cf));
-    List<Map.Entry<ThreatIntelResults, Long>> results = new ArrayList<>();
+    List<LookupKV<ThreatIntelKey, ThreatIntelValue>> results = new ArrayList<>();
     for(Result r : scanner) {
-      results.add(Converter.INSTANCE.fromResult(r, cf));
+      results.add(converter.fromResult(r, cf));
     }
     Assert.assertEquals(1, results.size());
-    Assert.assertEquals(0L, (long)results.get(0).getValue());
-    Assert.assertEquals(results.get(0).getKey().getKey().indicator, "google.com");
-    Assert.assertEquals(results.get(0).getKey().getValue().size(), 2);
-    Assert.assertEquals(results.get(0).getKey().getValue().get("meta"), "foo");
-    Assert.assertEquals(results.get(0).getKey().getValue().get("host"), "google.com");
+    Assert.assertEquals(results.get(0).getKey().indicator, "google.com");
+    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/0e1055aa/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 c98c256..8ace7bc 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
@@ -21,9 +21,11 @@ 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.reference.lookup.LookupKV;
 import org.apache.metron.threatintel.ThreatIntelResults;
-import org.apache.metron.threatintel.hbase.Converter;
-import org.apache.metron.threatintel.ThreatIntelKey;
+import org.apache.metron.hbase.converters.threatintel.ThreatIntelConverter;
+import org.apache.metron.hbase.converters.threatintel.ThreatIntelKey;
 import org.junit.Assert;
 import org.junit.Test;
 
@@ -67,6 +69,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());
         }});
         {
             mapper.map(null, new Text("#google.com,1,foo"), null);
@@ -78,14 +81,14 @@ public class BulkLoadMapperTest {
             ThreatIntelKey expectedKey = new ThreatIntelKey() {{
                 indicator = "google.com";
             }};
+            ThreatIntelConverter converter = new ThreatIntelConverter();
             Put put = puts.get(new ImmutableBytesWritable(expectedKey.toBytes()));
             Assert.assertNotNull(puts);
-            Map.Entry<ThreatIntelResults, Long> results = Converter.INSTANCE.fromPut(put, "cf");
-            Assert.assertEquals(0L, (long)results.getValue());
-            Assert.assertEquals(results.getKey().getKey().indicator, "google.com");
-            Assert.assertEquals(results.getKey().getValue().size(), 2);
-            Assert.assertEquals(results.getKey().getValue().get("meta"), "foo");
-            Assert.assertEquals(results.getKey().getValue().get("host"), "google.com");
+            LookupKV<ThreatIntelKey, ThreatIntelValue> 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");
+            Assert.assertEquals(results.getValue().getMetadata().get("host"), "google.com");
         }
 
     }

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/0e1055aa/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 6d5c71c..a7d28fb 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
@@ -19,19 +19,17 @@ package org.apache.metron.dataloads.hbase.mr;
 
 import com.google.common.collect.Iterables;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 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.LeastRecentlyUsedPruner;
+import org.apache.metron.dataloads.bulk.LeastRecentlyUsedPruner;
+import org.apache.metron.hbase.converters.threatintel.ThreatIntelValue;
 import org.apache.metron.reference.lookup.LookupKey;
-import org.apache.metron.reference.lookup.accesstracker.AccessTrackerUtil;
 import org.apache.metron.reference.lookup.accesstracker.BloomAccessTracker;
 import org.apache.metron.reference.lookup.accesstracker.PersistentAccessTracker;
-import org.apache.metron.threatintel.ThreatIntelKey;
-import org.apache.metron.threatintel.hbase.Converter;
+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;
@@ -84,24 +82,26 @@ public class LeastRecentlyUsedPrunerIntegrationTest {
         List<LookupKey> goodKeysOtherHalf = getKeys(5, 10);
         Iterable<LookupKey> goodKeys = Iterables.concat(goodKeysHalf, goodKeysOtherHalf);
         List<LookupKey> badKey = getKeys(10, 11);
+        ThreatIntelConverter converter = new ThreatIntelConverter();
         for(LookupKey k : goodKeysHalf) {
-            testTable.put(Converter.INSTANCE.toPut(cf, (ThreatIntelKey) k
-                                                  , new HashMap<String, String>() {{
+            testTable.put(converter.toPut(cf, (ThreatIntelKey) k
+                                            , new ThreatIntelValue(
+                                                  new HashMap<String, String>() {{
                                                     put("k", "dummy");
                                                     }}
-                                                  , 1L
                                                   )
+                                          )
                          );
             Assert.assertTrue(lookup.exists((ThreatIntelKey)k, testTable, true));
         }
         pat.persist(true);
         for(LookupKey k : goodKeysOtherHalf) {
-            testTable.put(Converter.INSTANCE.toPut(cf, (ThreatIntelKey) k
-                                                  , new HashMap<String, String>() {{
+            testTable.put(converter.toPut(cf, (ThreatIntelKey) k
+                                            , new ThreatIntelValue(new HashMap<String, String>() {{
                                                     put("k", "dummy");
                                                     }}
-                                                  , 1L
-                                                  )
+                                                                  )
+                                         )
                          );
             Assert.assertTrue(lookup.exists((ThreatIntelKey)k, testTable, true));
         }
@@ -112,11 +112,11 @@ public class LeastRecentlyUsedPrunerIntegrationTest {
         }
         pat.persist(true);
         {
-            testTable.put(Converter.INSTANCE.toPut(cf, (ThreatIntelKey) badKey.get(0)
-                    , new HashMap<String, String>() {{
+            testTable.put(converter.toPut(cf, (ThreatIntelKey) badKey.get(0)
+                    , new ThreatIntelValue(new HashMap<String, String>() {{
                         put("k", "dummy");
                     }}
-                    , 1L
+                    )
                     )
             );
         }

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/0e1055aa/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
new file mode 100644
index 0000000..64e4618
--- /dev/null
+++ b/metron-streaming/Metron-DataLoads/src/test/java/org/apache/metron/dataloads/taxii/MockTaxiiService.java
@@ -0,0 +1,99 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.metron.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);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/0e1055aa/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
new file mode 100644
index 0000000..f61d729
--- /dev/null
+++ b/metron-streaming/Metron-DataLoads/src/test/java/org/apache/metron/dataloads/taxii/TaxiiIntegrationTest.java
@@ -0,0 +1,121 @@
+/**
+ * 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/0e1055aa/metron-streaming/Metron-DataLoads/src/test/resources/taxii-messages/message.discovery
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-DataLoads/src/test/resources/taxii-messages/message.discovery b/metron-streaming/Metron-DataLoads/src/test/resources/taxii-messages/message.discovery
new file mode 100644
index 0000000..2f7e788
--- /dev/null
+++ b/metron-streaming/Metron-DataLoads/src/test/resources/taxii-messages/message.discovery
@@ -0,0 +1,21 @@
+<taxii_11:Discovery_Response in_response_to="urn:uuid:695149bd-72ab-41dd-a7e0-7d0b2a73e81f" message_id="65257" xmlns:xmldsig="http://www.w3.org/2000/09/xmldsig#" xmlns:taxii_11="http://taxii.mitre.org/messages/taxii_xml_binding-1.1">
+    <taxii_11:Service_Instance service_type="DISCOVERY" available="true" service_version="urn:taxii.mitre.org:services:1.1">
+        <taxii_11:Protocol_Binding>urn:taxii.mitre.org:protocol:https:1.0</taxii_11:Protocol_Binding>
+        <taxii_11:Address>http://localhost:PORT/taxii-data</taxii_11:Address>
+        <taxii_11:Message_Binding>urn:taxii.mitre.org:message:xml:1.1</taxii_11:Message_Binding>
+        <taxii_11:Message></taxii_11:Message>
+    </taxii_11:Service_Instance>
+    <taxii_11:Service_Instance service_type="COLLECTION_MANAGEMENT" available="true" service_version="urn:taxii.mitre.org:services:1.1">
+        <taxii_11:Protocol_Binding>urn:taxii.mitre.org:protocol:https:1.0</taxii_11:Protocol_Binding>
+        <taxii_11:Address>http://localhost:PORT/taxii-collections</taxii_11:Address>
+        <taxii_11:Message_Binding>urn:taxii.mitre.org:message:xml:1.1</taxii_11:Message_Binding>
+        <taxii_11:Message></taxii_11:Message>
+    </taxii_11:Service_Instance>
+    <taxii_11:Service_Instance service_type="POLL" available="true" service_version="urn:taxii.mitre.org:services:1.1">
+        <taxii_11:Protocol_Binding>urn:taxii.mitre.org:protocol:https:1.0</taxii_11:Protocol_Binding>
+        <taxii_11:Address>http://localhost:PORT/taxii-data</taxii_11:Address>
+        <taxii_11:Message_Binding>urn:taxii.mitre.org:message:xml:1.1</taxii_11:Message_Binding>
+        <taxii_11:Message></taxii_11:Message>
+    </taxii_11:Service_Instance>
+</taxii_11:Discovery_Response>
+