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:59 UTC

[17/43] incubator-metron git commit: METRON-56 Create unified enrichment topology (merrimanr via cestella) closes apache/incubator-metron#33

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/9f96399d/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
new file mode 100644
index 0000000..0e530f5
--- /dev/null
+++ b/metron-streaming/Metron-Topologies/src/main/resources/Metron_Configs/topologies/enrichment/test.yaml
@@ -0,0 +1,314 @@
+# 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.
+
+name: "enrichment"
+config:
+    topology.workers: 1
+
+components:
+# Enrichment
+    -   id: "geoEnrichmentAdapter"
+        className: "org.apache.metron.integration.util.mock.MockGeoAdapter"
+    -   id: "geoEnrichment"
+        className: "org.apache.metron.domain.Enrichment"
+        constructorArgs:
+            -   "geo"
+            -   ref: "geoEnrichmentAdapter"
+    -   id: "hostEnrichmentAdapter"
+        className: "org.apache.metron.enrichment.adapters.host.HostFromJSONListAdapter"
+        constructorArgs:
+            - '${org.apache.metron.enrichment.host.known_hosts}'
+    -   id: "hostEnrichment"
+        className: "org.apache.metron.domain.Enrichment"
+        constructorArgs:
+            -   "host"
+            -   ref: "hostEnrichmentAdapter"
+    -   id: "enrichments"
+        className: "java.util.ArrayList"
+        configMethods:
+            -   name: "add"
+                args:
+                    - ref: "geoEnrichment"
+            -   name: "add"
+                args:
+                    - ref: "hostEnrichment"
+
+# Threat Intel
+    -   id: "ipThreatIntelConfig"
+        className: "org.apache.metron.threatintel.ThreatIntelConfig"
+        configMethods:
+            -   name: "withProviderImpl"
+                args:
+                    - "${hbase.provider.impl}"
+            -   name: "withTrackerHBaseTable"
+                args:
+                    - "${threat.intel.tracker.table}"
+            -   name: "withTrackerHBaseCF"
+                args:
+                    - "${threat.intel.tracker.cf}"
+            -   name: "withHBaseTable"
+                args:
+                    - "${threat.intel.ip.table}"
+            -   name: "withHBaseCF"
+                args:
+                    - "${threat.intel.ip.cf}"
+    -   id: "ipThreatIntelAdapter"
+        className: "org.apache.metron.threatintel.ThreatIntelAdapter"
+        configMethods:
+           -    name: "withConfig"
+                args:
+                    - ref: "ipThreatIntelConfig"
+    -   id: "ipThreatIntelEnrichment"
+        className: "org.apache.metron.domain.Enrichment"
+        constructorArgs:
+          -   "ip"
+          -   ref: "ipThreatIntelAdapter"
+    -   id: "threatIntels"
+        className: "java.util.ArrayList"
+        configMethods:
+            -   name: "add"
+                args:
+                    - ref: "ipThreatIntelEnrichment"
+
+#indexing
+    -   id: "indexWriter"
+        className: "org.apache.metron.writer.ElasticsearchWriter"
+        constructorArgs:
+            - "${es.clustername}"
+            - "${es.ip}"
+            - ${es.port}
+            - "${index.date.format}"
+
+#kafka/zookeeper
+    -   id: "zkHosts"
+        className: "storm.kafka.ZkHosts"
+        constructorArgs:
+            - "${kafka.zk}"
+    -   id: "kafkaConfig"
+        className: "storm.kafka.SpoutConfig"
+        constructorArgs:
+            # zookeeper hosts
+            - ref: "zkHosts"
+            # topic name
+            - "enrichments"
+            # zk root
+            - ""
+            # id
+            - "enrichments"
+        properties:
+            -   name: "ignoreZkOffsets"
+                value: true
+            -   name: "startOffsetTime"
+                value: -2
+
+spouts:
+    -   id: "testingSpout"
+        className: "org.apache.metron.test.spouts.GenericInternalTestSpout"
+        parallelism: 1
+        configMethods:
+            -   name: "withFilename"
+                args:
+                    - "SampleInput/YafExampleOutput"
+            -   name: "withRepeating"
+                args:
+                    - true
+    -   id: "kafkaSpout"
+        className: "storm.kafka.KafkaSpout"
+        constructorArgs:
+            - ref: "kafkaConfig"
+bolts:
+# Enrichment Bolts
+    -   id: "enrichmentSplitBolt"
+        className: "org.apache.metron.enrichment.bolt.EnrichmentSplitterBolt"
+        constructorArgs:
+            - "${kafka.zk}"
+        configMethods:
+            -   name: "withEnrichments"
+                args:
+                    - ref: "enrichments"
+    -   id: "geoEnrichmentBolt"
+        className: "org.apache.metron.enrichment.bolt.GenericEnrichmentBolt"
+        constructorArgs:
+            - "${kafka.zk}"
+        configMethods:
+            -   name: "withEnrichment"
+                args:
+                    - ref: "geoEnrichment"
+            -   name: "withMaxCacheSize"
+                args: [10000]
+            -   name: "withMaxTimeRetain"
+                args: [10]
+    -   id: "hostEnrichmentBolt"
+        className: "org.apache.metron.enrichment.bolt.GenericEnrichmentBolt"
+        constructorArgs:
+            - "${kafka.zk}"
+        configMethods:
+            -   name: "withEnrichment"
+                args:
+                    - ref: "hostEnrichment"
+            -   name: "withMaxCacheSize"
+                args: [10000]
+            -   name: "withMaxTimeRetain"
+                args: [10]
+    -   id: "enrichmentJoinBolt"
+        className: "org.apache.metron.enrichment.bolt.EnrichmentJoinBolt"
+        constructorArgs:
+            - "${kafka.zk}"
+        configMethods:
+            -   name: "withEnrichments"
+                args:
+                    - ref: "enrichments"
+            -   name: "withMaxCacheSize"
+                args: [10000]
+            -   name: "withMaxTimeRetain"
+                args: [10]
+
+# Threat Intel Bolts
+    -   id: "threatIntelSplitBolt"
+        className: "org.apache.metron.enrichment.bolt.ThreatIntelSplitterBolt"
+        constructorArgs:
+            - "${kafka.zk}"
+        configMethods:
+            -   name: "withEnrichments"
+                args:
+                    - ref: "threatIntels"
+            -   name: "withMessageFieldName"
+                args: ["message"]
+    -   id: "ipThreatIntelBolt"
+        className: "org.apache.metron.enrichment.bolt.GenericEnrichmentBolt"
+        constructorArgs:
+            - "${kafka.zk}"
+        configMethods:
+            -   name: "withEnrichment"
+                args:
+                    - ref: "ipThreatIntelEnrichment"
+            -   name: "withMaxCacheSize"
+                args: [10000]
+            -   name: "withMaxTimeRetain"
+                args: [10]
+    -   id: "threatIntelJoinBolt"
+        className: "org.apache.metron.enrichment.bolt.ThreatIntelJoinBolt"
+        constructorArgs:
+            - "${kafka.zk}"
+        configMethods:
+            -   name: "withEnrichments"
+                args:
+                    - ref: "threatIntels"
+            -   name: "withMaxCacheSize"
+                args: [10000]
+            -   name: "withMaxTimeRetain"
+                args: [10]
+# Indexing Bolts
+    -   id: "indexingBolt"
+        className: "org.apache.metron.bolt.BulkMessageWriterBolt"
+        constructorArgs:
+            - "${kafka.zk}"
+        configMethods:
+            -   name: "withBulkMessageWriter"
+                args:
+                    - ref: "indexWriter"
+
+
+streams:
+#parser
+    -   name: "spout -> enrichmentSplit"
+        from: "kafkaSpout"
+        to: "enrichmentSplitBolt"
+        grouping:
+            type: SHUFFLE
+
+#enrichment
+    -   name: "enrichmentSplit -> host"
+        from: "enrichmentSplitBolt"
+        to: "hostEnrichmentBolt"
+        grouping:
+            streamId: "host"
+            type: FIELDS
+            args: ["key"]
+    -   name: "enrichmentSplit -> geo"
+        from: "enrichmentSplitBolt"
+        to: "geoEnrichmentBolt"
+        grouping:
+            streamId: "geo"
+            type: FIELDS
+            args: ["key"]
+    -   name: "splitter -> join"
+        from: "enrichmentSplitBolt"
+        to: "enrichmentJoinBolt"
+        grouping:
+            streamId: "message"
+            type: FIELDS
+            args: ["key"]
+    -   name: "geo -> join"
+        from: "geoEnrichmentBolt"
+        to: "enrichmentJoinBolt"
+        grouping:
+            streamId: "geo"
+            type: FIELDS
+            args: ["key"]
+    -   name: "host -> join"
+        from: "hostEnrichmentBolt"
+        to: "enrichmentJoinBolt"
+        grouping:
+            streamId: "host"
+            type: FIELDS
+            args: ["key"]
+
+#threat intel
+    -   name: "enrichmentJoin -> threatSplit"
+        from: "enrichmentJoinBolt"
+        to: "threatIntelSplitBolt"
+        grouping:
+            streamId: "message"
+            type: FIELDS
+            args: ["key"]
+
+    -   name: "threatSplit -> ip"
+        from: "threatIntelSplitBolt"
+        to: "ipThreatIntelBolt"
+        grouping:
+            streamId: "ip"
+            type: FIELDS
+            args: ["key"]
+
+    -   name: "ip -> join"
+        from: "ipThreatIntelBolt"
+        to: "threatIntelJoinBolt"
+        grouping:
+            streamId: "ip"
+            type: FIELDS
+            args: ["key"]
+    -   name: "threatIntelSplit -> threatIntelJoin"
+        from: "threatIntelSplitBolt"
+        to: "threatIntelJoinBolt"
+        grouping:
+            streamId: "message"
+            type: FIELDS
+            args: ["key"]
+#indexing
+    -   name: "threatIntelJoin -> indexing"
+        from: "threatIntelJoinBolt"
+        to: "indexingBolt"
+        grouping:
+            streamId: "message"
+            type: FIELDS
+            args: ["key"]
+    -   name: "indexingBolt -> errorIndexingBolt"
+        from: "indexingBolt"
+        to: "indexingBolt"
+        grouping:
+            streamId: "error"
+            type: SHUFFLE

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/9f96399d/metron-streaming/Metron-Topologies/src/main/resources/Metron_Configs/topologies/fireeye/local.yaml
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-Topologies/src/main/resources/Metron_Configs/topologies/fireeye/local.yaml b/metron-streaming/Metron-Topologies/src/main/resources/Metron_Configs/topologies/fireeye/local.yaml
deleted file mode 100644
index 9a3c471..0000000
--- a/metron-streaming/Metron-Topologies/src/main/resources/Metron_Configs/topologies/fireeye/local.yaml
+++ /dev/null
@@ -1,401 +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.
-
-name: "fireeye-local"
-config:
-    topology.workers: 1
-
-components:
-    -   id: "parser"
-        className: "org.apache.metron.parsing.parsers.BasicFireEyeParser"
-    -   id: "jdbcConfig"
-        className: "org.apache.metron.enrichment.adapters.jdbc.MySqlConfig"
-        properties:
-            -   name: "host"
-                value: "${mysql.ip}"
-            -   name: "port"
-                value: ${mysql.port}
-            -   name: "username"
-                value: "${mysql.username}"
-            -   name: "password"
-                value: "${mysql.password}"
-            -   name: "table"
-                value: "GEO"
-    -   id: "geoEnrichmentAdapter"
-        className: "org.apache.metron.enrichment.adapters.geo.GeoAdapter"
-        configMethods:
-            -   name: "withJdbcConfig"
-                args:
-                    - ref: "jdbcConfig"
-    -   id: "geoEnrichment"
-        className: "org.apache.metron.domain.Enrichment"
-        properties:
-            -   name: "name"
-                value:  "geo"
-            -   name: "fields"
-                value: ["ip_src_addr", "ip_dst_addr"]
-            -   name: "adapter"
-                ref: "geoEnrichmentAdapter"
-    -   id: "hostEnrichmentAdapter"
-        className: "org.apache.metron.enrichment.adapters.host.HostFromJSONListAdapter"
-        constructorArgs:
-            - '${org.apache.metron.enrichment.host.known_hosts}'
-    -   id: "hostEnrichment"
-        className: "org.apache.metron.domain.Enrichment"
-        properties:
-            -   name: "name"
-                value:  "host"
-            -   name: "fields"
-                value: ["ip_src_addr", "ip_dst_addr"]
-            -   name: "adapter"
-                ref: "hostEnrichmentAdapter"
-    -   id: "enrichments"
-        className: "java.util.ArrayList"
-        configMethods:
-            -   name: "add"
-                args:
-                    - ref: "geoEnrichment"
-            -   name: "add"
-                args:
-                    - ref: "hostEnrichment"
-    -   id: "indexAdapter"
-        className: "org.apache.metron.indexing.adapters.ESTimedRotatingAdapter"
-    -   id: "alertsConfig"
-        className: "java.util.HashMap"
-        configMethods:
-            -   name: "put"
-                args: ["whitelist_table_name", "ip_whitelist"]
-            -   name: "put"
-                args: ["blacklist_table_name", "ip_blacklist"]
-            -   name: "put"
-                args: ["quorum", "mon.cluster2.ctolab.hortonworks.com, nn1.cluster2.ctolab.hortonworks.com, nn2.cluster2.ctolab.hortonworks.com"]
-            -   name: "put"
-                args: ["port", "2181"]
-            -   name: "put"
-                args: ["_MAX_CACHE_SIZE_OBJECTS_NUM", "3600"]
-            -   name: "put"
-                args: ["_MAX_TIME_RETAIN_MINUTES", "1000"]
-    -   id: "alertsAdapter"
-        className: "org.apache.metron.alerts.adapters.CIFAlertsAdapter"
-        constructorArgs:
-            - ref: "alertsConfig"
-    -   id: "alertsIdentifier"
-        className: "org.json.simple.JSONObject"
-        configMethods:
-            -   name: "put"
-                args: ["environment", "local"]
-            -   name: "put"
-                args: ["topology", "fireeye"]
-    -   id: "metricConfig"
-        className: "org.apache.commons.configuration.BaseConfiguration"
-        configMethods:
-            -   name: "setProperty"
-                args:
-                    - "org.apache.metron.metrics.reporter.graphite"
-                    - "${org.apache.metron.metrics.reporter.graphite}"
-            -   name: "setProperty"
-                args:
-                    - "org.apache.metron.metrics.reporter.console"
-                    - "${org.apache.metron.metrics.reporter.console}"
-            -   name: "setProperty"
-                args:
-                    - "org.apache.metron.metrics.reporter.jmx"
-                    - "${org.apache.metron.metrics.reporter.jmx}"
-            -   name: "setProperty"
-                args:
-                    - "org.apache.metron.metrics.graphite.address"
-                    - "${org.apache.metron.metrics.graphite.address}"
-            -   name: "setProperty"
-                args:
-                    - "org.apache.metron.metrics.graphite.port"
-                    - "${org.apache.metron.metrics.graphite.port}"
-            -   name: "setProperty"
-                args:
-                    - "org.apache.metron.metrics.TelemetryParserBolt.acks"
-                    - "${org.apache.metron.metrics.TelemetryParserBolt.acks}"
-            -   name: "setProperty"
-                args:
-                    - "org.apache.metron.metrics.TelemetryParserBolt.emits"
-                    - "${org.apache.metron.metrics.TelemetryParserBolt.emits}"
-            -   name: "setProperty"
-                args:
-                    - "org.apache.metron.metrics.TelemetryParserBolt.fails"
-                    - "${org.apache.metron.metrics.TelemetryParserBolt.fails}"
-            -   name: "setProperty"
-                args:
-                    - "org.apache.metron.metrics.GenericEnrichmentBolt.acks"
-                    - "${org.apache.metron.metrics.GenericEnrichmentBolt.acks}"
-            -   name: "setProperty"
-                args:
-                    - "org.apache.metron.metrics.GenericEnrichmentBolt.emits"
-                    - "${org.apache.metron.metrics.GenericEnrichmentBolt.emits}"
-            -   name: "setProperty"
-                args:
-                    - "org.apache.metron.metrics.GenericEnrichmentBolt.fails"
-                    - "${org.apache.metron.metrics.GenericEnrichmentBolt.fails}"
-            -   name: "setProperty"
-                args:
-                    - "org.apache.metron.metrics.TelemetryIndexingBolt.acks"
-                    - "${org.apache.metron.metrics.TelemetryIndexingBolt.acks}"
-            -   name: "setProperty"
-                args:
-                    - "org.apache.metron.metrics.TelemetryIndexingBolt.emits"
-                    - "${org.apache.metron.metrics.TelemetryIndexingBolt.emits}"
-            -   name: "setProperty"
-                args:
-                    - "org.apache.metron.metrics.TelemetryIndexingBolt.fails"
-                    - "${org.apache.metron.metrics.TelemetryIndexingBolt.fails}"
-
-spouts:
-    -   id: "testingSpout"
-        className: "org.apache.metron.test.spouts.GenericInternalTestSpout"
-        parallelism: 1
-        configMethods:
-            -   name: "withFilename"
-                args:
-                    - "SampleInput/FireeyeExampleOutput"
-            -   name: "withRepeating"
-                args:
-                    - true
-
-bolts:
-    -   id: "parserBolt"
-        className: "org.apache.metron.bolt.TelemetryParserBolt"
-        configMethods:
-            -   name: "withMessageParser"
-                args:
-                    - ref: "parser"
-            -   name: "withEnrichments"
-                args:
-                    - ref: "enrichments"
-    -   id: "indexingBolt"
-        className: "org.apache.metron.indexing.TelemetryIndexingBolt"
-        configMethods:
-            -   name: "withIndexIP"
-                args:
-                    - "${es.ip}"
-            -   name: "withIndexPort"
-                args:
-                    - ${es.port}
-            -   name: "withClusterName"
-                args:
-                    - "${es.clustername}"
-            -   name: "withIndexName"
-                args:
-                    - "fireeye_index"
-            -   name: "withIndexTimestamp"
-                args:
-                    - "yyyy.MM.dd.hh"
-            -   name: "withDocumentName"
-                args:
-                    - "fireeye_doc"
-            -   name: "withBulk"
-                args:
-                    - 1
-            -   name: "withIndexAdapter"
-                args:
-                    - ref: "indexAdapter"
-            -   name: "withMetricConfiguration"
-                args:
-                    - ref: "metricConfig"
-    -   id: "alertsBolt"
-        className: "org.apache.metron.alerts.TelemetryAlertsBolt"
-        configMethods:
-            -   name: "withIdentifier"
-                args:
-                    - ref: "alertsIdentifier"
-            -   name: "withMaxCacheSize"
-                args: [1000]
-            -   name: "withMaxTimeRetain"
-                args: [3600]
-            -   name: "withAlertsAdapter"
-                args:
-                    - ref: "alertsAdapter"
-            -   name: "withOutputFieldName"
-                args: ["message"]
-            -   name: "withMetricConfiguration"
-                args:
-                    - ref: "metricConfig"
-    -   id: "alertsIndexingBolt"
-        className: "org.apache.metron.indexing.TelemetryIndexingBolt"
-        configMethods:
-            -   name: "withIndexIP"
-                args:
-                    - "${es.ip}"
-            -   name: "withIndexPort"
-                args:
-                    - ${es.port}
-            -   name: "withClusterName"
-                args:
-                    - "${es.clustername}"
-            -   name: "withIndexName"
-                args:
-                    - "alert"
-            -   name: "withIndexTimestamp"
-                args:
-                    - "yyyy.MM.ww"
-            -   name: "withDocumentName"
-                args:
-                    - "fireeye_alert"
-            -   name: "withBulk"
-                args:
-                    - 1
-            -   name: "withIndexAdapter"
-                args:
-                    - ref: "indexAdapter"
-            -   name: "withMetricConfiguration"
-                args:
-                    - ref: "metricConfig"
-    -   id: "errorIndexingBolt"
-        className: "org.apache.metron.indexing.TelemetryIndexingBolt"
-        configMethods:
-            -   name: "withIndexIP"
-                args:
-                    - "${es.ip}"
-            -   name: "withIndexPort"
-                args:
-                    - ${es.port}
-            -   name: "withClusterName"
-                args:
-                    - "${es.clustername}"
-            -   name: "withIndexName"
-                args:
-                    - "error"
-            -   name: "withIndexTimestamp"
-                args:
-                    - "yyyy.MM"
-            -   name: "withDocumentName"
-                args:
-                    - "fireeye_error"
-            -   name: "withBulk"
-                args:
-                    - 1
-            -   name: "withIndexAdapter"
-                args:
-                    - ref: "indexAdapter"
-            -   name: "withMetricConfiguration"
-                args:
-                    - ref: "metricConfig"
-    -   id: "geoEnrichmentBolt"
-        className: "org.apache.metron.enrichment.bolt.GenericEnrichmentBolt"
-        configMethods:
-            -   name: "withEnrichment"
-                args:
-                    - ref: "geoEnrichment"
-            -   name: "withMaxCacheSize"
-                args: [10000]
-            -   name: "withMaxTimeRetain"
-                args: [10]
-    -   id: "hostEnrichmentBolt"
-        className: "org.apache.metron.enrichment.bolt.GenericEnrichmentBolt"
-        configMethods:
-            -   name: "withEnrichment"
-                args:
-                    - ref: "hostEnrichment"
-            -   name: "withMaxCacheSize"
-                args: [10000]
-            -   name: "withMaxTimeRetain"
-                args: [10]
-    -   id: "joinBolt"
-        className: "org.apache.metron.enrichment.bolt.EnrichmentJoinBolt"
-        configMethods:
-        -   name: "withEnrichments"
-            args:
-                - ref: "enrichments"
-        -   name: "withMaxCacheSize"
-            args: [10000]
-        -   name: "withMaxTimeRetain"
-            args: [10]
-
-streams:
-    -   name: "spout -> parser"
-        from: "testingSpout"
-        to: "parserBolt"
-        grouping:
-            type: SHUFFLE
-    -   name: "parser -> host"
-        from: "parserBolt"
-        to: "hostEnrichmentBolt"
-        grouping:
-            streamId: "host"
-            type: FIELDS
-            args: ["key"]
-    -   name: "parser -> geo"
-        from: "parserBolt"
-        to: "geoEnrichmentBolt"
-        grouping:
-            streamId: "geo"
-            type: FIELDS
-            args: ["key"]
-    -   name: "parser -> join"
-        from: "parserBolt"
-        to: "joinBolt"
-        grouping:
-            streamId: "message"
-            type: FIELDS
-            args: ["key"]
-    -   name: "geo -> join"
-        from: "geoEnrichmentBolt"
-        to: "joinBolt"
-        grouping:
-            streamId: "geo"
-            type: FIELDS
-            args: ["key"]
-    -   name: "host -> join"
-        from: "hostEnrichmentBolt"
-        to: "joinBolt"
-        grouping:
-            streamId: "host"
-            type: FIELDS
-            args: ["key"]
-    -   name: "join -> alerts"
-        from: "joinBolt"
-        to: "alertsBolt"
-        grouping:
-            streamId: "message"
-            type: FIELDS
-            args: ["key"]
-    -   name: "alerts -> alertsIndexing"
-        from: "alertsBolt"
-        to: "alertsIndexingBolt"
-        grouping:
-            streamId: "message"
-            type: SHUFFLE
-    -   name: "join -> indexing"
-        from: "joinBolt"
-        to: "indexingBolt"
-        grouping:
-            streamId: "message"
-            type: FIELDS
-            args: ["key"]
-    -   name: "parser -> errors"
-        from: "parserBolt"
-        to: "errorIndexingBolt"
-        grouping:
-            streamId: "error"
-            type: SHUFFLE
-    -   name: "indexing -> errors"
-        from: "indexingBolt"
-        to: "errorIndexingBolt"
-        grouping:
-            streamId: "error"
-            type: SHUFFLE
-    -   name: "alerts -> errors"
-        from: "alertsBolt"
-        to: "errorIndexingBolt"
-        grouping:
-            streamId: "error"
-            type: SHUFFLE

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/9f96399d/metron-streaming/Metron-Topologies/src/main/resources/Metron_Configs/topologies/fireeye/remote.yaml
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-Topologies/src/main/resources/Metron_Configs/topologies/fireeye/remote.yaml b/metron-streaming/Metron-Topologies/src/main/resources/Metron_Configs/topologies/fireeye/remote.yaml
index cea5990..59cc372 100644
--- a/metron-streaming/Metron-Topologies/src/main/resources/Metron_Configs/topologies/fireeye/remote.yaml
+++ b/metron-streaming/Metron-Topologies/src/main/resources/Metron_Configs/topologies/fireeye/remote.yaml
@@ -21,143 +21,10 @@ config:
 components:
     -   id: "parser"
         className: "org.apache.metron.parsing.parsers.BasicFireEyeParser"
-    -   id: "jdbcConfig"
-        className: "org.apache.metron.enrichment.adapters.jdbc.MySqlConfig"
-        properties:
-            -   name: "host"
-                value: "${mysql.ip}"
-            -   name: "port"
-                value: ${mysql.port}
-            -   name: "username"
-                value: "${mysql.username}"
-            -   name: "password"
-                value: "${mysql.password}"
-            -   name: "table"
-                value: "GEO"
-    -   id: "geoEnrichmentAdapter"
-        className: "org.apache.metron.enrichment.adapters.geo.GeoAdapter"
-        configMethods:
-            -   name: "withJdbcConfig"
-                args:
-                    - ref: "jdbcConfig"
-    -   id: "geoEnrichment"
-        className: "org.apache.metron.domain.Enrichment"
-        properties:
-            -   name: "name"
-                value:  "geo"
-            -   name: "fields"
-                value: ["ip_src_addr", "ip_dst_addr"]
-            -   name: "adapter"
-                ref: "geoEnrichmentAdapter"
-    -   id: "hostEnrichmentAdapter"
-        className: "org.apache.metron.enrichment.adapters.host.HostFromJSONListAdapter"
+    -   id: "writer"
+        className: "org.apache.metron.writer.KafkaWriter"
         constructorArgs:
-            - '${org.apache.metron.enrichment.host.known_hosts}'
-    -   id: "hostEnrichment"
-        className: "org.apache.metron.domain.Enrichment"
-        properties:
-            -   name: "name"
-                value:  "host"
-            -   name: "fields"
-                value: ["ip_src_addr", "ip_dst_addr"]
-            -   name: "adapter"
-                ref: "hostEnrichmentAdapter"
-    -   id: "enrichments"
-        className: "java.util.ArrayList"
-        configMethods:
-            -   name: "add"
-                args:
-                    - ref: "geoEnrichment"
-            -   name: "add"
-                args:
-                    - ref: "hostEnrichment"
-    -   id: "indexAdapter"
-        className: "org.apache.metron.indexing.adapters.ESTimedRotatingAdapter"
-    -   id: "alertsConfig"
-        className: "java.util.HashMap"
-        configMethods:
-            -   name: "put"
-                args: ["whitelist_table_name", "ip_whitelist"]
-            -   name: "put"
-                args: ["blacklist_table_name", "ip_blacklist"]
-            -   name: "put"
-                args: ["quorum", "mon.cluster2.ctolab.hortonworks.com, nn1.cluster2.ctolab.hortonworks.com, nn2.cluster2.ctolab.hortonworks.com"]
-            -   name: "put"
-                args: ["port", "2181"]
-            -   name: "put"
-                args: ["_MAX_CACHE_SIZE_OBJECTS_NUM", "3600"]
-            -   name: "put"
-                args: ["_MAX_TIME_RETAIN_MINUTES", "1000"]
-    -   id: "alertsAdapter"
-        className: "org.apache.metron.alerts.adapters.CIFAlertsAdapter"
-        constructorArgs:
-            - ref: "alertsConfig"
-    -   id: "alertsIdentifier"
-        className: "org.json.simple.JSONObject"
-        configMethods:
-            -   name: "put"
-                args: ["environment", "local"]
-            -   name: "put"
-                args: ["topology", "fireeye"]
-    -   id: "metricConfig"
-        className: "org.apache.commons.configuration.BaseConfiguration"
-        configMethods:
-            -   name: "setProperty"
-                args:
-                    - "org.apache.metron.metrics.reporter.graphite"
-                    - "${org.apache.metron.metrics.reporter.graphite}"
-            -   name: "setProperty"
-                args:
-                    - "org.apache.metron.metrics.reporter.console"
-                    - "${org.apache.metron.metrics.reporter.console}"
-            -   name: "setProperty"
-                args:
-                    - "org.apache.metron.metrics.reporter.jmx"
-                    - "${org.apache.metron.metrics.reporter.jmx}"
-            -   name: "setProperty"
-                args:
-                    - "org.apache.metron.metrics.graphite.address"
-                    - "${org.apache.metron.metrics.graphite.address}"
-            -   name: "setProperty"
-                args:
-                    - "org.apache.metron.metrics.graphite.port"
-                    - "${org.apache.metron.metrics.graphite.port}"
-            -   name: "setProperty"
-                args:
-                    - "org.apache.metron.metrics.TelemetryParserBolt.acks"
-                    - "${org.apache.metron.metrics.TelemetryParserBolt.acks}"
-            -   name: "setProperty"
-                args:
-                    - "org.apache.metron.metrics.TelemetryParserBolt.emits"
-                    - "${org.apache.metron.metrics.TelemetryParserBolt.emits}"
-            -   name: "setProperty"
-                args:
-                    - "org.apache.metron.metrics.TelemetryParserBolt.fails"
-                    - "${org.apache.metron.metrics.TelemetryParserBolt.fails}"
-            -   name: "setProperty"
-                args:
-                    - "org.apache.metron.metrics.GenericEnrichmentBolt.acks"
-                    - "${org.apache.metron.metrics.GenericEnrichmentBolt.acks}"
-            -   name: "setProperty"
-                args:
-                    - "org.apache.metron.metrics.GenericEnrichmentBolt.emits"
-                    - "${org.apache.metron.metrics.GenericEnrichmentBolt.emits}"
-            -   name: "setProperty"
-                args:
-                    - "org.apache.metron.metrics.GenericEnrichmentBolt.fails"
-                    - "${org.apache.metron.metrics.GenericEnrichmentBolt.fails}"
-            -   name: "setProperty"
-                args:
-                    - "org.apache.metron.metrics.TelemetryIndexingBolt.acks"
-                    - "${org.apache.metron.metrics.TelemetryIndexingBolt.acks}"
-            -   name: "setProperty"
-                args:
-                    - "org.apache.metron.metrics.TelemetryIndexingBolt.emits"
-                    - "${org.apache.metron.metrics.TelemetryIndexingBolt.emits}"
-            -   name: "setProperty"
-                args:
-                    - "org.apache.metron.metrics.TelemetryIndexingBolt.fails"
-                    - "${org.apache.metron.metrics.TelemetryIndexingBolt.fails}"
+            - "${kafka.broker}"
     -   id: "zkHosts"
         className: "storm.kafka.ZkHosts"
         constructorArgs:
@@ -168,18 +35,28 @@ components:
             # zookeeper hosts
             - ref: "zkHosts"
             # topic name
-            - "${spout.kafka.topic.pcap}"
+            - "${spout.kafka.topic.snort}"
             # zk root
             - ""
             # id
-            - "${spout.kafka.topic.pcap}"
+            - "${spout.kafka.topic.snort}"
         properties:
-            -   name: "forceFromStart"
+            -   name: "ignoreZkOffsets"
                 value: true
             -   name: "startOffsetTime"
                 value: -1
 
 spouts:
+    -   id: "testingSpout"
+        className: "org.apache.metron.test.spouts.GenericInternalTestSpout"
+        parallelism: 1
+        configMethods:
+            -   name: "withFilename"
+                args:
+                    - "SampleInput/YafExampleOutput"
+            -   name: "withRepeating"
+                args:
+                    - true
     -   id: "kafkaSpout"
         className: "storm.kafka.KafkaSpout"
         constructorArgs:
@@ -187,229 +64,16 @@ spouts:
 
 bolts:
     -   id: "parserBolt"
-        className: "org.apache.metron.bolt.TelemetryParserBolt"
-        configMethods:
-            -   name: "withMessageParser"
-                args:
-                    - ref: "parser"
-            -   name: "withEnrichments"
-                args:
-                    - ref: "enrichments"
-    -   id: "indexingBolt"
-        className: "org.apache.metron.indexing.TelemetryIndexingBolt"
-        configMethods:
-            -   name: "withIndexIP"
-                args:
-                    - "${es.ip}"
-            -   name: "withIndexPort"
-                args:
-                    - ${es.port}
-            -   name: "withClusterName"
-                args:
-                    - "${es.clustername}"
-            -   name: "withIndexName"
-                args:
-                    - "fireeye_index"
-            -   name: "withIndexTimestamp"
-                args:
-                    - "yyyy.MM.dd.hh"
-            -   name: "withDocumentName"
-                args:
-                    - "fireeye_doc"
-            -   name: "withBulk"
-                args:
-                    - 1
-            -   name: "withIndexAdapter"
-                args:
-                    - ref: "indexAdapter"
-            -   name: "withMetricConfiguration"
-                args:
-                    - ref: "metricConfig"
-    -   id: "alertsBolt"
-        className: "org.apache.metron.alerts.TelemetryAlertsBolt"
-        configMethods:
-            -   name: "withIdentifier"
-                args:
-                    - ref: "alertsIdentifier"
-            -   name: "withMaxCacheSize"
-                args: [1000]
-            -   name: "withMaxTimeRetain"
-                args: [3600]
-            -   name: "withAlertsAdapter"
-                args:
-                    - ref: "alertsAdapter"
-            -   name: "withOutputFieldName"
-                args: ["message"]
-            -   name: "withMetricConfiguration"
-                args:
-                    - ref: "metricConfig"
-    -   id: "alertsIndexingBolt"
-        className: "org.apache.metron.indexing.TelemetryIndexingBolt"
-        configMethods:
-            -   name: "withIndexIP"
-                args:
-                    - "${es.ip}"
-            -   name: "withIndexPort"
-                args:
-                    - ${es.port}
-            -   name: "withClusterName"
-                args:
-                    - "${es.clustername}"
-            -   name: "withIndexName"
-                args:
-                    - "alert"
-            -   name: "withIndexTimestamp"
-                args:
-                    - "yyyy.MM.ww"
-            -   name: "withDocumentName"
-                args:
-                    - "fireeye_alert"
-            -   name: "withBulk"
-                args:
-                    - 1
-            -   name: "withIndexAdapter"
-                args:
-                    - ref: "indexAdapter"
-            -   name: "withMetricConfiguration"
-                args:
-                    - ref: "metricConfig"
-    -   id: "errorIndexingBolt"
-        className: "org.apache.metron.indexing.TelemetryIndexingBolt"
-        configMethods:
-            -   name: "withIndexIP"
-                args:
-                    - "${es.ip}"
-            -   name: "withIndexPort"
-                args:
-                    - ${es.port}
-            -   name: "withClusterName"
-                args:
-                    - "${es.clustername}"
-            -   name: "withIndexName"
-                args:
-                    - "error"
-            -   name: "withIndexTimestamp"
-                args:
-                    - "yyyy.MM"
-            -   name: "withDocumentName"
-                args:
-                    - "fireeye_error"
-            -   name: "withBulk"
-                args:
-                    - 1
-            -   name: "withIndexAdapter"
-                args:
-                    - ref: "indexAdapter"
-            -   name: "withMetricConfiguration"
-                args:
-                    - ref: "metricConfig"
-    -   id: "geoEnrichmentBolt"
-        className: "org.apache.metron.enrichment.bolt.GenericEnrichmentBolt"
-        configMethods:
-            -   name: "withEnrichment"
-                args:
-                    - ref: "geoEnrichment"
-            -   name: "withMaxCacheSize"
-                args: [10000]
-            -   name: "withMaxTimeRetain"
-                args: [10]
-    -   id: "hostEnrichmentBolt"
-        className: "org.apache.metron.enrichment.bolt.GenericEnrichmentBolt"
-        configMethods:
-            -   name: "withEnrichment"
-                args:
-                    - ref: "hostEnrichment"
-            -   name: "withMaxCacheSize"
-                args: [10000]
-            -   name: "withMaxTimeRetain"
-                args: [10]
-    -   id: "joinBolt"
-        className: "org.apache.metron.enrichment.bolt.EnrichmentJoinBolt"
-        configMethods:
-        -   name: "withEnrichments"
-            args:
-                - ref: "enrichments"
-        -   name: "withMaxCacheSize"
-            args: [10000]
-        -   name: "withMaxTimeRetain"
-            args: [10]
+        className: "org.apache.metron.bolt.ParserBolt"
+        constructorArgs:
+            - "${kafka.zk}"
+            - "${spout.kafka.topic.snort}"
+            - ref: "parser"
+            - ref: "writer"
 
 streams:
-    -   name: "spout -> parser"
+    -   name: "spout -> bolt"
         from: "kafkaSpout"
         to: "parserBolt"
         grouping:
             type: SHUFFLE
-    -   name: "parser -> host"
-        from: "parserBolt"
-        to: "hostEnrichmentBolt"
-        grouping:
-            streamId: "host"
-            type: FIELDS
-            args: ["key"]
-    -   name: "parser -> geo"
-        from: "parserBolt"
-        to: "geoEnrichmentBolt"
-        grouping:
-            streamId: "geo"
-            type: FIELDS
-            args: ["key"]
-    -   name: "parser -> join"
-        from: "parserBolt"
-        to: "joinBolt"
-        grouping:
-            streamId: "message"
-            type: FIELDS
-            args: ["key"]
-    -   name: "geo -> join"
-        from: "geoEnrichmentBolt"
-        to: "joinBolt"
-        grouping:
-            streamId: "geo"
-            type: FIELDS
-            args: ["key"]
-    -   name: "host -> join"
-        from: "hostEnrichmentBolt"
-        to: "joinBolt"
-        grouping:
-            streamId: "host"
-            type: FIELDS
-            args: ["key"]
-    -   name: "join -> alerts"
-        from: "joinBolt"
-        to: "alertsBolt"
-        grouping:
-            streamId: "message"
-            type: FIELDS
-            args: ["key"]
-    -   name: "alerts -> alertsIndexing"
-        from: "alertsBolt"
-        to: "alertsIndexingBolt"
-        grouping:
-            streamId: "message"
-            type: SHUFFLE
-    -   name: "join -> indexing"
-        from: "joinBolt"
-        to: "indexingBolt"
-        grouping:
-            streamId: "message"
-            type: FIELDS
-            args: ["key"]
-    -   name: "parser -> errors"
-        from: "parserBolt"
-        to: "errorIndexingBolt"
-        grouping:
-            streamId: "error"
-            type: SHUFFLE
-    -   name: "indexing -> errors"
-        from: "indexingBolt"
-        to: "errorIndexingBolt"
-        grouping:
-            streamId: "error"
-            type: SHUFFLE
-    -   name: "alerts -> errors"
-        from: "alertsBolt"
-        to: "errorIndexingBolt"
-        grouping:
-            streamId: "error"
-            type: SHUFFLE

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/9f96399d/metron-streaming/Metron-Topologies/src/main/resources/Metron_Configs/topologies/fireeye/test.yaml
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-Topologies/src/main/resources/Metron_Configs/topologies/fireeye/test.yaml b/metron-streaming/Metron-Topologies/src/main/resources/Metron_Configs/topologies/fireeye/test.yaml
new file mode 100644
index 0000000..c014f86
--- /dev/null
+++ b/metron-streaming/Metron-Topologies/src/main/resources/Metron_Configs/topologies/fireeye/test.yaml
@@ -0,0 +1,79 @@
+# 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.
+
+name: "fireeye-test"
+config:
+    topology.workers: 1
+
+components:
+    -   id: "parser"
+        className: "org.apache.metron.parsing.parsers.BasicFireEyeParser"
+    -   id: "writer"
+        className: "org.apache.metron.writer.KafkaWriter"
+        constructorArgs:
+            - "${kafka.broker}"
+    -   id: "zkHosts"
+        className: "storm.kafka.ZkHosts"
+        constructorArgs:
+            - "${kafka.zk}"
+    -   id: "kafkaConfig"
+        className: "storm.kafka.SpoutConfig"
+        constructorArgs:
+            # zookeeper hosts
+            - ref: "zkHosts"
+            # topic name
+            - "${spout.kafka.topic.snort}"
+            # zk root
+            - ""
+            # id
+            - "${spout.kafka.topic.snort}"
+        properties:
+            -   name: "ignoreZkOffsets"
+                value: true
+            -   name: "startOffsetTime"
+                value: -2
+
+spouts:
+    -   id: "testingSpout"
+        className: "org.apache.metron.test.spouts.GenericInternalTestSpout"
+        parallelism: 1
+        configMethods:
+            -   name: "withFilename"
+                args:
+                    - "SampleInput/YafExampleOutput"
+            -   name: "withRepeating"
+                args:
+                    - false
+    -   id: "kafkaSpout"
+        className: "storm.kafka.KafkaSpout"
+        constructorArgs:
+            - ref: "kafkaConfig"
+
+bolts:
+    -   id: "parserBolt"
+        className: "org.apache.metron.bolt.ParserBolt"
+        constructorArgs:
+            - "${kafka.zk}"
+            - "${spout.kafka.topic.snort}"
+            - ref: "parser"
+            - ref: "writer"
+
+streams:
+    -   name: "spout -> bolt"
+        from: "kafkaSpout"
+        to: "parserBolt"
+        grouping:
+            type: SHUFFLE

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/9f96399d/metron-streaming/Metron-Topologies/src/main/resources/Metron_Configs/topologies/ise/local.yaml
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-Topologies/src/main/resources/Metron_Configs/topologies/ise/local.yaml b/metron-streaming/Metron-Topologies/src/main/resources/Metron_Configs/topologies/ise/local.yaml
deleted file mode 100644
index 2afba20..0000000
--- a/metron-streaming/Metron-Topologies/src/main/resources/Metron_Configs/topologies/ise/local.yaml
+++ /dev/null
@@ -1,192 +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.
-
-name: "ise-local"
-config:
-    topology.workers: 1
-
-components:
-    -   id: "iseParser"
-        className: "org.apache.metron.parsing.parsers.BasicIseParser"
-    -   id: "genericMessageFilter"
-        className: "org.apache.metron.filters.GenericMessageFilter"
-    -   id: "indexAdapter"
-        className: "org.apache.metron.indexing.adapters.ESTimedRotatingAdapter"
-    -   id: "metricConfig"
-        className: "org.apache.commons.configuration.BaseConfiguration"
-        configMethods:
-            -   name: "setProperty"
-                args:
-                    - "org.apache.metron.metrics.reporter.graphite"
-                    - "${org.apache.metron.metrics.reporter.graphite}"
-            -   name: "setProperty"
-                args:
-                    - "org.apache.metron.metrics.reporter.console"
-                    - "${org.apache.metron.metrics.reporter.console}"
-            -   name: "setProperty"
-                args:
-                    - "org.apache.metron.metrics.reporter.jmx"
-                    - "${org.apache.metron.metrics.reporter.jmx}"
-            -   name: "setProperty"
-                args:
-                    - "org.apache.metron.metrics.graphite.address"
-                    - "${org.apache.metron.metrics.graphite.address}"
-            -   name: "setProperty"
-                args:
-                    - "org.apache.metron.metrics.graphite.port"
-                    - "${org.apache.metron.metrics.graphite.port}"
-            -   name: "setProperty"
-                args:
-                    - "org.apache.metron.metrics.TelemetryParserBolt.acks"
-                    - "${org.apache.metron.metrics.TelemetryParserBolt.acks}"
-            -   name: "setProperty"
-                args:
-                    - "org.apache.metron.metrics.TelemetryParserBolt.emits"
-                    - "${org.apache.metron.metrics.TelemetryParserBolt.emits}"
-            -   name: "setProperty"
-                args:
-                    - "org.apache.metron.metrics.TelemetryParserBolt.fails"
-                    - "${org.apache.metron.metrics.TelemetryParserBolt.fails}"
-            -   name: "setProperty"
-                args:
-                    - "org.apache.metron.metrics.GenericEnrichmentBolt.acks"
-                    - "${org.apache.metron.metrics.GenericEnrichmentBolt.acks}"
-            -   name: "setProperty"
-                args:
-                    - "org.apache.metron.metrics.GenericEnrichmentBolt.emits"
-                    - "${org.apache.metron.metrics.GenericEnrichmentBolt.emits}"
-            -   name: "setProperty"
-                args:
-                    - "org.apache.metron.metrics.GenericEnrichmentBolt.fails"
-                    - "${org.apache.metron.metrics.GenericEnrichmentBolt.fails}"
-            -   name: "setProperty"
-                args:
-                    - "org.apache.metron.metrics.TelemetryIndexingBolt.acks"
-                    - "${org.apache.metron.metrics.TelemetryIndexingBolt.acks}"
-            -   name: "setProperty"
-                args:
-                    - "org.apache.metron.metrics.TelemetryIndexingBolt.emits"
-                    - "${org.apache.metron.metrics.TelemetryIndexingBolt.emits}"
-            -   name: "setProperty"
-                args:
-                    - "org.apache.metron.metrics.TelemetryIndexingBolt.fails"
-                    - "${org.apache.metron.metrics.TelemetryIndexingBolt.fails}"
-
-spouts:
-    -   id: "testingSpout"
-        className: "org.apache.metron.test.spouts.GenericInternalTestSpout"
-        parallelism: 1
-        configMethods:
-            -   name: "withFilename"
-                args:
-                    - "SampleInput/ISESampleOutput"
-            -   name: "withRepeating"
-                args:
-                    - true
-
-bolts:
-    -   id: "parserBolt"
-        className: "org.apache.metron.bolt.TelemetryParserBolt"
-        configMethods:
-            -   name: "withMessageParser"
-                args:
-                    - ref: "iseParser"
-    -   id: "indexingBolt"
-        className: "org.apache.metron.indexing.TelemetryIndexingBolt"
-        configMethods:
-            -   name: "withIndexIP"
-                args:
-                    - "${es.ip}"
-            -   name: "withIndexPort"
-                args:
-                    - ${es.port}
-            -   name: "withClusterName"
-                args:
-                    - "${es.clustername}"
-            -   name: "withIndexName"
-                args:
-                    - "ise_index"
-            -   name: "withIndexTimestamp"
-                args:
-                    - "yyyy.MM.dd.hh"
-            -   name: "withDocumentName"
-                args:
-                    - "ise_doc"
-            -   name: "withBulk"
-                args:
-                    - 1
-            -   name: "withIndexAdapter"
-                args:
-                    - ref: "indexAdapter"
-            -   name: "withMetricConfiguration"
-                args:
-                    - ref: "metricConfig"
-    -   id: "errorIndexingBolt"
-        className: "org.apache.metron.indexing.TelemetryIndexingBolt"
-        configMethods:
-            -   name: "withIndexIP"
-                args:
-                    - "${es.ip}"
-            -   name: "withIndexPort"
-                args:
-                    - ${es.port}
-            -   name: "withClusterName"
-                args:
-                    - "${es.clustername}"
-            -   name: "withIndexName"
-                args:
-                    - "error"
-            -   name: "withIndexTimestamp"
-                args:
-                    - "yyyy.MM"
-            -   name: "withDocumentName"
-                args:
-                    - "ise_error"
-            -   name: "withBulk"
-                args:
-                    - 1
-            -   name: "withIndexAdapter"
-                args:
-                    - ref: "indexAdapter"
-            -   name: "withMetricConfiguration"
-                args:
-                    - ref: "metricConfig"
-
-streams:
-    -   name: "spout -> parser"
-        from: "testingSpout"
-        to: "parserBolt"
-        grouping:
-            type: SHUFFLE
-    -   name: "parser -> indexing"
-        from: "parserBolt"
-        to: "indexingBolt"
-        grouping:
-            streamId: "message"
-            type: FIELDS
-            args: ["key"]
-    -   name: "parser -> errors"
-        from: "parserBolt"
-        to: "errorIndexingBolt"
-        grouping:
-            streamId: "error"
-            type: SHUFFLE
-    -   name: "indexing -> errors"
-        from: "indexingBolt"
-        to: "errorIndexingBolt"
-        grouping:
-            streamId: "error"
-            type: SHUFFLE

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/9f96399d/metron-streaming/Metron-Topologies/src/main/resources/Metron_Configs/topologies/ise/remote.yaml
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-Topologies/src/main/resources/Metron_Configs/topologies/ise/remote.yaml b/metron-streaming/Metron-Topologies/src/main/resources/Metron_Configs/topologies/ise/remote.yaml
index 0196ae6..78cd779 100644
--- a/metron-streaming/Metron-Topologies/src/main/resources/Metron_Configs/topologies/ise/remote.yaml
+++ b/metron-streaming/Metron-Topologies/src/main/resources/Metron_Configs/topologies/ise/remote.yaml
@@ -19,71 +19,12 @@ config:
     topology.workers: 1
 
 components:
-    -   id: "iseParser"
+    -   id: "parser"
         className: "org.apache.metron.parsing.parsers.BasicIseParser"
-    -   id: "genericMessageFilter"
-        className: "org.apache.metron.filters.GenericMessageFilter"
-    -   id: "indexAdapter"
-        className: "org.apache.metron.indexing.adapters.ESTimedRotatingAdapter"
-    -   id: "metricConfig"
-        className: "org.apache.commons.configuration.BaseConfiguration"
-        configMethods:
-            -   name: "setProperty"
-                args:
-                    - "org.apache.metron.metrics.reporter.graphite"
-                    - "${org.apache.metron.metrics.reporter.graphite}"
-            -   name: "setProperty"
-                args:
-                    - "org.apache.metron.metrics.reporter.console"
-                    - "${org.apache.metron.metrics.reporter.console}"
-            -   name: "setProperty"
-                args:
-                    - "org.apache.metron.metrics.reporter.jmx"
-                    - "${org.apache.metron.metrics.reporter.jmx}"
-            -   name: "setProperty"
-                args:
-                    - "org.apache.metron.metrics.graphite.address"
-                    - "${org.apache.metron.metrics.graphite.address}"
-            -   name: "setProperty"
-                args:
-                    - "org.apache.metron.metrics.graphite.port"
-                    - "${org.apache.metron.metrics.graphite.port}"
-            -   name: "setProperty"
-                args:
-                    - "org.apache.metron.metrics.TelemetryParserBolt.acks"
-                    - "${org.apache.metron.metrics.TelemetryParserBolt.acks}"
-            -   name: "setProperty"
-                args:
-                    - "org.apache.metron.metrics.TelemetryParserBolt.emits"
-                    - "${org.apache.metron.metrics.TelemetryParserBolt.emits}"
-            -   name: "setProperty"
-                args:
-                    - "org.apache.metron.metrics.TelemetryParserBolt.fails"
-                    - "${org.apache.metron.metrics.TelemetryParserBolt.fails}"
-            -   name: "setProperty"
-                args:
-                    - "org.apache.metron.metrics.GenericEnrichmentBolt.acks"
-                    - "${org.apache.metron.metrics.GenericEnrichmentBolt.acks}"
-            -   name: "setProperty"
-                args:
-                    - "org.apache.metron.metrics.GenericEnrichmentBolt.emits"
-                    - "${org.apache.metron.metrics.GenericEnrichmentBolt.emits}"
-            -   name: "setProperty"
-                args:
-                    - "org.apache.metron.metrics.GenericEnrichmentBolt.fails"
-                    - "${org.apache.metron.metrics.GenericEnrichmentBolt.fails}"
-            -   name: "setProperty"
-                args:
-                    - "org.apache.metron.metrics.TelemetryIndexingBolt.acks"
-                    - "${org.apache.metron.metrics.TelemetryIndexingBolt.acks}"
-            -   name: "setProperty"
-                args:
-                    - "org.apache.metron.metrics.TelemetryIndexingBolt.emits"
-                    - "${org.apache.metron.metrics.TelemetryIndexingBolt.emits}"
-            -   name: "setProperty"
-                args:
-                    - "org.apache.metron.metrics.TelemetryIndexingBolt.fails"
-                    - "${org.apache.metron.metrics.TelemetryIndexingBolt.fails}"
+    -   id: "writer"
+        className: "org.apache.metron.writer.KafkaWriter"
+        constructorArgs:
+            - "${kafka.broker}"
     -   id: "zkHosts"
         className: "storm.kafka.ZkHosts"
         constructorArgs:
@@ -94,18 +35,28 @@ components:
             # zookeeper hosts
             - ref: "zkHosts"
             # topic name
-            - "${spout.kafka.topic.ise}"
+            - "${spout.kafka.topic.snort}"
             # zk root
             - ""
             # id
-            - "${spout.kafka.topic.ise}"
+            - "${spout.kafka.topic.snort}"
         properties:
-            -   name: "forceFromStart"
+            -   name: "ignoreZkOffsets"
                 value: true
             -   name: "startOffsetTime"
                 value: -1
 
 spouts:
+    -   id: "testingSpout"
+        className: "org.apache.metron.test.spouts.GenericInternalTestSpout"
+        parallelism: 1
+        configMethods:
+            -   name: "withFilename"
+                args:
+                    - "SampleInput/YafExampleOutput"
+            -   name: "withRepeating"
+                args:
+                    - true
     -   id: "kafkaSpout"
         className: "storm.kafka.KafkaSpout"
         constructorArgs:
@@ -113,94 +64,16 @@ spouts:
 
 bolts:
     -   id: "parserBolt"
-        className: "org.apache.metron.bolt.TelemetryParserBolt"
-        configMethods:
-            -   name: "withMessageParser"
-                args:
-                    - ref: "iseParser"
-    -   id: "indexingBolt"
-        className: "org.apache.metron.indexing.TelemetryIndexingBolt"
-        configMethods:
-            -   name: "withIndexIP"
-                args:
-                    - "${es.ip}"
-            -   name: "withIndexPort"
-                args:
-                    - ${es.port}
-            -   name: "withClusterName"
-                args:
-                    - "${es.clustername}"
-            -   name: "withIndexName"
-                args:
-                    - "ise_index"
-            -   name: "withIndexTimestamp"
-                args:
-                    - "yyyy.MM.dd.hh"
-            -   name: "withDocumentName"
-                args:
-                    - "ise_doc"
-            -   name: "withBulk"
-                args:
-                    - 1
-            -   name: "withIndexAdapter"
-                args:
-                    - ref: "indexAdapter"
-            -   name: "withMetricConfiguration"
-                args:
-                    - ref: "metricConfig"
-    -   id: "errorIndexingBolt"
-        className: "org.apache.metron.indexing.TelemetryIndexingBolt"
-        configMethods:
-            -   name: "withIndexIP"
-                args:
-                    - "${es.ip}"
-            -   name: "withIndexPort"
-                args:
-                    - ${es.port}
-            -   name: "withClusterName"
-                args:
-                    - "${es.clustername}"
-            -   name: "withIndexName"
-                args:
-                    - "error"
-            -   name: "withIndexTimestamp"
-                args:
-                    - "yyyy.MM"
-            -   name: "withDocumentName"
-                args:
-                    - "ise_error"
-            -   name: "withBulk"
-                args:
-                    - 1
-            -   name: "withIndexAdapter"
-                args:
-                    - ref: "indexAdapter"
-            -   name: "withMetricConfiguration"
-                args:
-                    - ref: "metricConfig"
+        className: "org.apache.metron.bolt.ParserBolt"
+        constructorArgs:
+            - "${kafka.zk}"
+            - "${spout.kafka.topic.snort}"
+            - ref: "parser"
+            - ref: "writer"
 
 streams:
-    -   name: "spout -> parser"
+    -   name: "spout -> bolt"
         from: "kafkaSpout"
         to: "parserBolt"
         grouping:
             type: SHUFFLE
-    -   name: "parser -> indexing"
-        from: "parserBolt"
-        to: "indexingBolt"
-        grouping:
-            streamId: "message"
-            type: FIELDS
-            args: ["key"]
-    -   name: "parser -> errors"
-        from: "parserBolt"
-        to: "errorIndexingBolt"
-        grouping:
-            streamId: "error"
-            type: SHUFFLE
-    -   name: "indexing -> errors"
-        from: "indexingBolt"
-        to: "errorIndexingBolt"
-        grouping:
-            streamId: "error"
-            type: SHUFFLE

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/9f96399d/metron-streaming/Metron-Topologies/src/main/resources/Metron_Configs/topologies/ise/test.yaml
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-Topologies/src/main/resources/Metron_Configs/topologies/ise/test.yaml b/metron-streaming/Metron-Topologies/src/main/resources/Metron_Configs/topologies/ise/test.yaml
new file mode 100644
index 0000000..4d6239c
--- /dev/null
+++ b/metron-streaming/Metron-Topologies/src/main/resources/Metron_Configs/topologies/ise/test.yaml
@@ -0,0 +1,79 @@
+# 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.
+
+name: "ise-test"
+config:
+    topology.workers: 1
+
+components:
+    -   id: "parser"
+        className: "org.apache.metron.parsing.parsers.BasicIseParser"
+    -   id: "writer"
+        className: "org.apache.metron.writer.KafkaWriter"
+        constructorArgs:
+            - "${kafka.broker}"
+    -   id: "zkHosts"
+        className: "storm.kafka.ZkHosts"
+        constructorArgs:
+            - "${kafka.zk}"
+    -   id: "kafkaConfig"
+        className: "storm.kafka.SpoutConfig"
+        constructorArgs:
+            # zookeeper hosts
+            - ref: "zkHosts"
+            # topic name
+            - "${spout.kafka.topic.snort}"
+            # zk root
+            - ""
+            # id
+            - "${spout.kafka.topic.snort}"
+        properties:
+            -   name: "ignoreZkOffsets"
+                value: true
+            -   name: "startOffsetTime"
+                value: -2
+
+spouts:
+    -   id: "testingSpout"
+        className: "org.apache.metron.test.spouts.GenericInternalTestSpout"
+        parallelism: 1
+        configMethods:
+            -   name: "withFilename"
+                args:
+                    - "SampleInput/YafExampleOutput"
+            -   name: "withRepeating"
+                args:
+                    - false
+    -   id: "kafkaSpout"
+        className: "storm.kafka.KafkaSpout"
+        constructorArgs:
+            - ref: "kafkaConfig"
+
+bolts:
+    -   id: "parserBolt"
+        className: "org.apache.metron.bolt.ParserBolt"
+        constructorArgs:
+            - "${kafka.zk}"
+            - "${spout.kafka.topic.snort}"
+            - ref: "parser"
+            - ref: "writer"
+
+streams:
+    -   name: "spout -> bolt"
+        from: "kafkaSpout"
+        to: "parserBolt"
+        grouping:
+            type: SHUFFLE

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/9f96399d/metron-streaming/Metron-Topologies/src/main/resources/Metron_Configs/topologies/lancope/local.yaml
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-Topologies/src/main/resources/Metron_Configs/topologies/lancope/local.yaml b/metron-streaming/Metron-Topologies/src/main/resources/Metron_Configs/topologies/lancope/local.yaml
deleted file mode 100644
index 57a7344..0000000
--- a/metron-streaming/Metron-Topologies/src/main/resources/Metron_Configs/topologies/lancope/local.yaml
+++ /dev/null
@@ -1,401 +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.
-
-name: "lancope-local"
-config:
-    topology.workers: 1
-
-components:
-    -   id: "parser"
-        className: "org.apache.metron.parsing.parsers.BasicLancopeParser"
-    -   id: "jdbcConfig"
-        className: "org.apache.metron.enrichment.adapters.jdbc.MySqlConfig"
-        properties:
-            -   name: "host"
-                value: "${mysql.ip}"
-            -   name: "port"
-                value: ${mysql.port}
-            -   name: "username"
-                value: "${mysql.username}"
-            -   name: "password"
-                value: "${mysql.password}"
-            -   name: "table"
-                value: "GEO"
-    -   id: "geoEnrichmentAdapter"
-        className: "org.apache.metron.enrichment.adapters.geo.GeoAdapter"
-        configMethods:
-            -   name: "withJdbcConfig"
-                args:
-                    - ref: "jdbcConfig"
-    -   id: "geoEnrichment"
-        className: "org.apache.metron.domain.Enrichment"
-        properties:
-            -   name: "name"
-                value:  "geo"
-            -   name: "fields"
-                value: ["ip_src_addr", "ip_dst_addr"]
-            -   name: "adapter"
-                ref: "geoEnrichmentAdapter"
-    -   id: "hostEnrichmentAdapter"
-        className: "org.apache.metron.enrichment.adapters.host.HostFromJSONListAdapter"
-        constructorArgs:
-            - '${org.apache.metron.enrichment.host.known_hosts}'
-    -   id: "hostEnrichment"
-        className: "org.apache.metron.domain.Enrichment"
-        properties:
-            -   name: "name"
-                value:  "host"
-            -   name: "fields"
-                value: ["ip_src_addr", "ip_dst_addr"]
-            -   name: "adapter"
-                ref: "hostEnrichmentAdapter"
-    -   id: "enrichments"
-        className: "java.util.ArrayList"
-        configMethods:
-            -   name: "add"
-                args:
-                    - ref: "geoEnrichment"
-            -   name: "add"
-                args:
-                    - ref: "hostEnrichment"
-    -   id: "indexAdapter"
-        className: "org.apache.metron.indexing.adapters.ESTimedRotatingAdapter"
-    -   id: "alertsConfig"
-        className: "java.util.HashMap"
-        configMethods:
-            -   name: "put"
-                args: ["whitelist_table_name", "ip_whitelist"]
-            -   name: "put"
-                args: ["blacklist_table_name", "ip_blacklist"]
-            -   name: "put"
-                args: ["quorum", "mon.cluster2.ctolab.hortonworks.com, nn1.cluster2.ctolab.hortonworks.com, nn2.cluster2.ctolab.hortonworks.com"]
-            -   name: "put"
-                args: ["port", "2181"]
-            -   name: "put"
-                args: ["_MAX_CACHE_SIZE_OBJECTS_NUM", "3600"]
-            -   name: "put"
-                args: ["_MAX_TIME_RETAIN_MINUTES", "1000"]
-    -   id: "alertsAdapter"
-        className: "org.apache.metron.alerts.adapters.CIFAlertsAdapter"
-        constructorArgs:
-            - ref: "alertsConfig"
-    -   id: "alertsIdentifier"
-        className: "org.json.simple.JSONObject"
-        configMethods:
-            -   name: "put"
-                args: ["environment", "local"]
-            -   name: "put"
-                args: ["topology", "lancope"]
-    -   id: "metricConfig"
-        className: "org.apache.commons.configuration.BaseConfiguration"
-        configMethods:
-            -   name: "setProperty"
-                args:
-                    - "org.apache.metron.metrics.reporter.graphite"
-                    - "${org.apache.metron.metrics.reporter.graphite}"
-            -   name: "setProperty"
-                args:
-                    - "org.apache.metron.metrics.reporter.console"
-                    - "${org.apache.metron.metrics.reporter.console}"
-            -   name: "setProperty"
-                args:
-                    - "org.apache.metron.metrics.reporter.jmx"
-                    - "${org.apache.metron.metrics.reporter.jmx}"
-            -   name: "setProperty"
-                args:
-                    - "org.apache.metron.metrics.graphite.address"
-                    - "${org.apache.metron.metrics.graphite.address}"
-            -   name: "setProperty"
-                args:
-                    - "org.apache.metron.metrics.graphite.port"
-                    - "${org.apache.metron.metrics.graphite.port}"
-            -   name: "setProperty"
-                args:
-                    - "org.apache.metron.metrics.TelemetryParserBolt.acks"
-                    - "${org.apache.metron.metrics.TelemetryParserBolt.acks}"
-            -   name: "setProperty"
-                args:
-                    - "org.apache.metron.metrics.TelemetryParserBolt.emits"
-                    - "${org.apache.metron.metrics.TelemetryParserBolt.emits}"
-            -   name: "setProperty"
-                args:
-                    - "org.apache.metron.metrics.TelemetryParserBolt.fails"
-                    - "${org.apache.metron.metrics.TelemetryParserBolt.fails}"
-            -   name: "setProperty"
-                args:
-                    - "org.apache.metron.metrics.GenericEnrichmentBolt.acks"
-                    - "${org.apache.metron.metrics.GenericEnrichmentBolt.acks}"
-            -   name: "setProperty"
-                args:
-                    - "org.apache.metron.metrics.GenericEnrichmentBolt.emits"
-                    - "${org.apache.metron.metrics.GenericEnrichmentBolt.emits}"
-            -   name: "setProperty"
-                args:
-                    - "org.apache.metron.metrics.GenericEnrichmentBolt.fails"
-                    - "${org.apache.metron.metrics.GenericEnrichmentBolt.fails}"
-            -   name: "setProperty"
-                args:
-                    - "org.apache.metron.metrics.TelemetryIndexingBolt.acks"
-                    - "${org.apache.metron.metrics.TelemetryIndexingBolt.acks}"
-            -   name: "setProperty"
-                args:
-                    - "org.apache.metron.metrics.TelemetryIndexingBolt.emits"
-                    - "${org.apache.metron.metrics.TelemetryIndexingBolt.emits}"
-            -   name: "setProperty"
-                args:
-                    - "org.apache.metron.metrics.TelemetryIndexingBolt.fails"
-                    - "${org.apache.metron.metrics.TelemetryIndexingBolt.fails}"
-
-spouts:
-    -   id: "testingSpout"
-        className: "org.apache.metron.test.spouts.GenericInternalTestSpout"
-        parallelism: 1
-        configMethods:
-            -   name: "withFilename"
-                args:
-                    - "SampleInput/LancopeExampleOutput"
-            -   name: "withRepeating"
-                args:
-                    - true
-
-bolts:
-    -   id: "parserBolt"
-        className: "org.apache.metron.bolt.TelemetryParserBolt"
-        configMethods:
-            -   name: "withMessageParser"
-                args:
-                    - ref: "parser"
-            -   name: "withEnrichments"
-                args:
-                    - ref: "enrichments"
-    -   id: "indexingBolt"
-        className: "org.apache.metron.indexing.TelemetryIndexingBolt"
-        configMethods:
-            -   name: "withIndexIP"
-                args:
-                    - "${es.ip}"
-            -   name: "withIndexPort"
-                args:
-                    - ${es.port}
-            -   name: "withClusterName"
-                args:
-                    - "${es.clustername}"
-            -   name: "withIndexName"
-                args:
-                    - "lancope_index"
-            -   name: "withIndexTimestamp"
-                args:
-                    - "yyyy.MM.dd.hh"
-            -   name: "withDocumentName"
-                args:
-                    - "lancope_doc"
-            -   name: "withBulk"
-                args:
-                    - 1
-            -   name: "withIndexAdapter"
-                args:
-                    - ref: "indexAdapter"
-            -   name: "withMetricConfiguration"
-                args:
-                    - ref: "metricConfig"
-    -   id: "alertsBolt"
-        className: "org.apache.metron.alerts.TelemetryAlertsBolt"
-        configMethods:
-            -   name: "withIdentifier"
-                args:
-                    - ref: "alertsIdentifier"
-            -   name: "withMaxCacheSize"
-                args: [1000]
-            -   name: "withMaxTimeRetain"
-                args: [3600]
-            -   name: "withAlertsAdapter"
-                args:
-                    - ref: "alertsAdapter"
-            -   name: "withOutputFieldName"
-                args: ["message"]
-            -   name: "withMetricConfiguration"
-                args:
-                    - ref: "metricConfig"
-    -   id: "alertsIndexingBolt"
-        className: "org.apache.metron.indexing.TelemetryIndexingBolt"
-        configMethods:
-            -   name: "withIndexIP"
-                args:
-                    - "${es.ip}"
-            -   name: "withIndexPort"
-                args:
-                    - ${es.port}
-            -   name: "withClusterName"
-                args:
-                    - "${es.clustername}"
-            -   name: "withIndexName"
-                args:
-                    - "alert"
-            -   name: "withIndexTimestamp"
-                args:
-                    - "yyyy.MM.ww"
-            -   name: "withDocumentName"
-                args:
-                    - "lancope_alert"
-            -   name: "withBulk"
-                args:
-                    - 1
-            -   name: "withIndexAdapter"
-                args:
-                    - ref: "indexAdapter"
-            -   name: "withMetricConfiguration"
-                args:
-                    - ref: "metricConfig"
-    -   id: "errorIndexingBolt"
-        className: "org.apache.metron.indexing.TelemetryIndexingBolt"
-        configMethods:
-            -   name: "withIndexIP"
-                args:
-                    - "${es.ip}"
-            -   name: "withIndexPort"
-                args:
-                    - ${es.port}
-            -   name: "withClusterName"
-                args:
-                    - "${es.clustername}"
-            -   name: "withIndexName"
-                args:
-                    - "error"
-            -   name: "withIndexTimestamp"
-                args:
-                    - "yyyy.MM"
-            -   name: "withDocumentName"
-                args:
-                    - "lancope_error"
-            -   name: "withBulk"
-                args:
-                    - 1
-            -   name: "withIndexAdapter"
-                args:
-                    - ref: "indexAdapter"
-            -   name: "withMetricConfiguration"
-                args:
-                    - ref: "metricConfig"
-    -   id: "geoEnrichmentBolt"
-        className: "org.apache.metron.enrichment.bolt.GenericEnrichmentBolt"
-        configMethods:
-            -   name: "withEnrichment"
-                args:
-                    - ref: "geoEnrichment"
-            -   name: "withMaxCacheSize"
-                args: [10000]
-            -   name: "withMaxTimeRetain"
-                args: [10]
-    -   id: "hostEnrichmentBolt"
-        className: "org.apache.metron.enrichment.bolt.GenericEnrichmentBolt"
-        configMethods:
-            -   name: "withEnrichment"
-                args:
-                    - ref: "hostEnrichment"
-            -   name: "withMaxCacheSize"
-                args: [10000]
-            -   name: "withMaxTimeRetain"
-                args: [10]
-    -   id: "joinBolt"
-        className: "org.apache.metron.enrichment.bolt.EnrichmentJoinBolt"
-        configMethods:
-        -   name: "withEnrichments"
-            args:
-                - ref: "enrichments"
-        -   name: "withMaxCacheSize"
-            args: [10000]
-        -   name: "withMaxTimeRetain"
-            args: [10]
-
-streams:
-    -   name: "spout -> parser"
-        from: "testingSpout"
-        to: "parserBolt"
-        grouping:
-            type: SHUFFLE
-    -   name: "parser -> host"
-        from: "parserBolt"
-        to: "hostEnrichmentBolt"
-        grouping:
-            streamId: "host"
-            type: FIELDS
-            args: ["key"]
-    -   name: "parser -> geo"
-        from: "parserBolt"
-        to: "geoEnrichmentBolt"
-        grouping:
-            streamId: "geo"
-            type: FIELDS
-            args: ["key"]
-    -   name: "parser -> join"
-        from: "parserBolt"
-        to: "joinBolt"
-        grouping:
-            streamId: "message"
-            type: FIELDS
-            args: ["key"]
-    -   name: "geo -> join"
-        from: "geoEnrichmentBolt"
-        to: "joinBolt"
-        grouping:
-            streamId: "geo"
-            type: FIELDS
-            args: ["key"]
-    -   name: "host -> join"
-        from: "hostEnrichmentBolt"
-        to: "joinBolt"
-        grouping:
-            streamId: "host"
-            type: FIELDS
-            args: ["key"]
-    -   name: "join -> alerts"
-        from: "joinBolt"
-        to: "alertsBolt"
-        grouping:
-            streamId: "message"
-            type: FIELDS
-            args: ["key"]
-    -   name: "alerts -> alertsIndexing"
-        from: "alertsBolt"
-        to: "alertsIndexingBolt"
-        grouping:
-            streamId: "message"
-            type: SHUFFLE
-    -   name: "join -> indexing"
-        from: "joinBolt"
-        to: "indexingBolt"
-        grouping:
-            streamId: "message"
-            type: FIELDS
-            args: ["key"]
-    -   name: "parser -> errors"
-        from: "parserBolt"
-        to: "errorIndexingBolt"
-        grouping:
-            streamId: "error"
-            type: SHUFFLE
-    -   name: "indexing -> errors"
-        from: "indexingBolt"
-        to: "errorIndexingBolt"
-        grouping:
-            streamId: "error"
-            type: SHUFFLE
-    -   name: "alerts -> errors"
-        from: "alertsBolt"
-        to: "errorIndexingBolt"
-        grouping:
-            streamId: "error"
-            type: SHUFFLE