You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@metron.apache.org by om...@apache.org on 2015/12/08 07:38:02 UTC

[38/51] [partial] incubator-metron git commit: Initial import of code from https://github.com/OpenSOC/opensoc at ac0b00373f8f56dfae03a8109af5feb373ea598e.

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/05e188ba/opensoc-streaming/OpenSOC-Topologies/src/main/java/com/opensoc/topology/runner/TopologyRunner.java
----------------------------------------------------------------------
diff --git a/opensoc-streaming/OpenSOC-Topologies/src/main/java/com/opensoc/topology/runner/TopologyRunner.java b/opensoc-streaming/OpenSOC-Topologies/src/main/java/com/opensoc/topology/runner/TopologyRunner.java
new file mode 100644
index 0000000..095d3be
--- /dev/null
+++ b/opensoc-streaming/OpenSOC-Topologies/src/main/java/com/opensoc/topology/runner/TopologyRunner.java
@@ -0,0 +1,931 @@
+/*
+ * 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 com.opensoc.topology.runner;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Stack;
+
+import org.apache.commons.configuration.Configuration;
+import org.apache.commons.configuration.ConfigurationException;
+import org.apache.commons.configuration.PropertiesConfiguration;
+import org.apache.commons.lang.StringUtils;
+import org.apache.storm.hdfs.bolt.HdfsBolt;
+import org.apache.storm.hdfs.bolt.format.DefaultFileNameFormat;
+import org.apache.storm.hdfs.bolt.format.DelimitedRecordFormat;
+import org.apache.storm.hdfs.bolt.format.FileNameFormat;
+import org.apache.storm.hdfs.bolt.format.RecordFormat;
+import org.apache.storm.hdfs.bolt.rotation.FileRotationPolicy;
+import org.apache.storm.hdfs.bolt.rotation.FileSizeRotationPolicy;
+import org.apache.storm.hdfs.bolt.rotation.FileSizeRotationPolicy.Units;
+import org.apache.storm.hdfs.bolt.sync.CountSyncPolicy;
+import org.apache.storm.hdfs.bolt.sync.SyncPolicy;
+import org.apache.storm.hdfs.common.rotation.MoveFileAction;
+import org.json.simple.JSONObject;
+
+import storm.kafka.BrokerHosts;
+import storm.kafka.KafkaSpout;
+import storm.kafka.SpoutConfig;
+import storm.kafka.ZkHosts;
+import storm.kafka.bolt.KafkaBolt;
+import backtype.storm.Config;
+import backtype.storm.LocalCluster;
+import backtype.storm.StormSubmitter;
+import backtype.storm.generated.AlreadyAliveException;
+import backtype.storm.generated.Grouping;
+import backtype.storm.generated.InvalidTopologyException;
+import backtype.storm.spout.RawScheme;
+import backtype.storm.spout.SchemeAsMultiScheme;
+import backtype.storm.topology.BoltDeclarer;
+import backtype.storm.topology.TopologyBuilder;
+import backtype.storm.tuple.Fields;
+
+import com.opensoc.alerts.TelemetryAlertsBolt;
+import com.opensoc.alerts.adapters.HbaseWhiteAndBlacklistAdapter;
+import com.opensoc.alerts.interfaces.AlertsAdapter;
+import com.opensoc.enrichment.adapters.cif.CIFHbaseAdapter;
+import com.opensoc.enrichment.adapters.geo.GeoMysqlAdapter;
+import com.opensoc.enrichment.adapters.host.HostFromPropertiesFileAdapter;
+import com.opensoc.enrichment.adapters.whois.WhoisHBaseAdapter;
+import com.opensoc.enrichment.common.GenericEnrichmentBolt;
+import com.opensoc.enrichment.interfaces.EnrichmentAdapter;
+import com.opensoc.hbase.HBaseBolt;
+import com.opensoc.hbase.HBaseStreamPartitioner;
+import com.opensoc.hbase.TupleTableConfig;
+import com.opensoc.indexing.TelemetryIndexingBolt;
+import com.opensoc.indexing.adapters.ESBaseBulkAdapter;
+import com.opensoc.indexing.adapters.ESTimedRotatingAdapter;
+import com.opensoc.json.serialization.JSONKryoSerializer;
+import com.opensoc.topologyhelpers.Cli;
+import com.opensoc.topologyhelpers.SettingsLoader;
+
+public abstract class TopologyRunner {
+
+	protected Configuration config;
+	protected TopologyBuilder builder;
+	protected Config conf;
+	protected boolean local_mode = true;
+	protected boolean debug = true;
+	protected String config_path = null;
+	protected String default_config_path = "OpenSOC_Configs";
+	protected boolean success = false;
+	protected Stack<String> messageComponents = new Stack<String>();
+	protected Stack<String> errorComponents = new Stack<String>();
+	protected Stack<String> alertComponents = new Stack<String>();
+	protected Stack<String> dataComponents = new Stack<String>();
+	protected Stack<String> terminalComponents = new Stack<String>();
+
+	public void initTopology(String args[], String subdir)
+			throws ConfigurationException, AlreadyAliveException,
+			InvalidTopologyException {
+		Cli command_line = new Cli(args);
+		command_line.parse();
+
+		System.out.println("[OpenSOC] Starting topology deployment...");
+
+		debug = command_line.isDebug();
+		System.out.println("[OpenSOC] Debug mode set to: " + debug);
+
+		local_mode = command_line.isLocal_mode();
+		System.out.println("[OpenSOC] Local mode set to: " + local_mode);
+
+		if (command_line.getPath() != null) {
+			config_path = command_line.getPath();
+			System.out
+					.println("[OpenSOC] Setting config path to external config path: "
+							+ config_path);
+		} else {
+			config_path = default_config_path;
+			System.out
+					.println("[OpenSOC] Initializing from default internal config path: "
+							+ config_path);
+		}
+
+		String topology_conf_path = config_path + "/topologies/" + subdir
+				+ "/topology.conf";
+
+		String environment_identifier_path = config_path
+				+ "/topologies/environment_identifier.conf";
+		String topology_identifier_path = config_path + "/topologies/" + subdir
+				+ "/topology_identifier.conf";
+
+		System.out.println("[OpenSOC] Looking for environment identifier: "
+				+ environment_identifier_path);
+		System.out.println("[OpenSOC] Looking for topology identifier: "
+				+ topology_identifier_path);
+		System.out.println("[OpenSOC] Looking for topology config: "
+				+ topology_conf_path);
+
+		config = new PropertiesConfiguration(topology_conf_path);
+
+		JSONObject environment_identifier = SettingsLoader
+				.loadEnvironmentIdnetifier(environment_identifier_path);
+		JSONObject topology_identifier = SettingsLoader
+				.loadTopologyIdnetifier(topology_identifier_path);
+
+		String topology_name = SettingsLoader.generateTopologyName(
+				environment_identifier, topology_identifier);
+
+		System.out.println("[OpenSOC] Initializing Topology: " + topology_name);
+
+		builder = new TopologyBuilder();
+
+		conf = new Config();
+		conf.registerSerialization(JSONObject.class, JSONKryoSerializer.class);
+		conf.setDebug(debug);
+
+		System.out.println("[OpenSOC] Initializing Spout: " + topology_name);
+
+		if (command_line.isGenerator_spout()) {
+			String component_name = config.getString("spout.test.name",
+					"DefaultTopologySpout");
+			success = initializeTestingSpout(component_name);
+			messageComponents.add(component_name);
+
+			System.out.println("[OpenSOC] ------Component " + component_name
+					+ " initialized with the following settings:");
+
+			SettingsLoader.printConfigOptions((PropertiesConfiguration) config,
+					"spout.test");
+		}
+
+		if (!command_line.isGenerator_spout()) {
+			String component_name = config.getString("spout.kafka.name",
+					"DefaultTopologyKafkaSpout");
+
+			success = initializeKafkaSpout(component_name);
+			messageComponents.add(component_name);
+
+			System.out.println("[OpenSOC] ------Component " + component_name
+					+ " initialized with the following settings:");
+
+			SettingsLoader.printConfigOptions((PropertiesConfiguration) config,
+					"spout.kafka");
+		}
+
+		if (config.getBoolean("parser.bolt.enabled", true)) {
+			String component_name = config.getString("parser.bolt.name",
+					"DefaultTopologyParserBot");
+
+			success = initializeParsingBolt(topology_name, component_name);
+			messageComponents.add(component_name);
+			errorComponents.add(component_name);
+
+			dataComponents.add(component_name);
+
+			System.out.println("[OpenSOC] ------Component " + component_name
+					+ " initialized with the following settings:");
+
+			SettingsLoader.printConfigOptions((PropertiesConfiguration) config,
+					"parser.bolt");
+		}
+
+		if (config.getBoolean("bolt.enrichment.geo.enabled", false)) {
+			String component_name = config.getString(
+					"bolt.enrichment.geo.name", "DefaultGeoEnrichmentBolt");
+
+			success = initializeGeoEnrichment(topology_name, component_name);
+			messageComponents.add(component_name);
+			errorComponents.add(component_name);
+
+			System.out.println("[OpenSOC] ------Component " + component_name
+					+ " initialized with the following settings:");
+
+			SettingsLoader.printConfigOptions((PropertiesConfiguration) config,
+					"bolt.enrichment.geo");
+			SettingsLoader.printConfigOptions((PropertiesConfiguration) config,
+					"mysql");
+		}
+
+		if (config.getBoolean("bolt.enrichment.host.enabled", false)) {
+			String component_name = config.getString(
+					"bolt.enrichment.host.name", "DefaultHostEnrichmentBolt");
+
+			success = initializeHostsEnrichment(topology_name, component_name,
+					"OpenSOC_Configs/etc/whitelists/known_hosts.conf");
+			messageComponents.add(component_name);
+			errorComponents.add(component_name);
+
+			System.out.println("[OpenSOC] ------Component " + component_name
+					+ " initialized with the following settings:");
+
+			SettingsLoader.printConfigOptions((PropertiesConfiguration) config,
+					"bolt.enrichment.host");
+		}
+
+		if (config.getBoolean("bolt.enrichment.whois.enabled", false)) {
+			String component_name = config.getString(
+					"bolt.enrichment.whois.name", "DefaultWhoisEnrichmentBolt");
+
+			success = initializeWhoisEnrichment(topology_name, component_name);
+			messageComponents.add(component_name);
+			errorComponents.add(component_name);
+
+			System.out.println("[OpenSOC] ------Component " + component_name
+					+ " initialized with the following settings:");
+
+			SettingsLoader.printConfigOptions((PropertiesConfiguration) config,
+					"bolt.enrichment.whois");
+		}
+
+		if (config.getBoolean("bolt.enrichment.cif.enabled", false)) {
+			String component_name = config.getString(
+					"bolt.enrichment.cif.name", "DefaultCIFEnrichmentBolt");
+
+			success = initializeCIFEnrichment(topology_name, component_name);
+			messageComponents.add(component_name);
+			errorComponents.add(component_name);
+
+			System.out.println("[OpenSOC] ------Component " + component_name
+					+ " initialized with the following settings:");
+
+			SettingsLoader.printConfigOptions((PropertiesConfiguration) config,
+					"bolt.enrichment.cif");
+		}
+
+		if (config.getBoolean("bolt.alerts.enabled", false)) {
+			String component_name = config.getString("bolt.alerts.name",
+					"DefaultAlertsBolt");
+
+			success = initializeAlerts(topology_name, component_name,
+					config_path + "/topologies/" + subdir + "/alerts.xml",
+					environment_identifier, topology_identifier);
+
+			messageComponents.add(component_name);
+			errorComponents.add(component_name);
+			alertComponents.add(component_name);
+
+			System.out.println("[OpenSOC] ------Component " + component_name
+					+ " initialized with the following settings:");
+
+			SettingsLoader.printConfigOptions((PropertiesConfiguration) config,
+					"bolt.alerts");
+		}
+
+		if (config.getBoolean("bolt.alerts.indexing.enabled") && config.getBoolean("bolt.alerts.enabled")) {
+
+			String component_name = config.getString(
+					"bolt.alerts.indexing.name", "DefaultAlertsBolt");
+
+			success = initializeAlertIndexing(component_name);
+			terminalComponents.add(component_name);
+
+			System.out.println("[OpenSOC] ------Component " + component_name
+					+ " initialized with the following settings:");
+
+			SettingsLoader.printConfigOptions((PropertiesConfiguration) config,
+					"bolt.alerts.indexing");
+		}
+
+		if (config.getBoolean("bolt.kafka.enabled", false)) {
+			String component_name = config.getString("bolt.kafka.name",
+					"DefaultKafkaBolt");
+
+			success = initializeKafkaBolt(component_name);
+			terminalComponents.add(component_name);
+
+			System.out.println("[OpenSOC] Component " + component_name
+					+ " initialized");
+
+			System.out.println("[OpenSOC] ------Component " + component_name
+					+ " initialized with the following settings:");
+
+			SettingsLoader.printConfigOptions((PropertiesConfiguration) config,
+					"bolt.kafka");
+		}
+
+		if (config.getBoolean("bolt.indexing.enabled", true)) {
+			String component_name = config.getString("bolt.indexing.name",
+					"DefaultIndexingBolt");
+
+			success = initializeIndexingBolt(component_name);
+			errorComponents.add(component_name);
+			terminalComponents.add(component_name);
+
+			System.out.println("[OpenSOC] ------Component " + component_name
+					+ " initialized with the following settings:");
+
+			SettingsLoader.printConfigOptions((PropertiesConfiguration) config,
+					"bolt.indexing");
+		}
+
+		if (config.getBoolean("bolt.hdfs.enabled", false)) {
+			String component_name = config.getString("bolt.hdfs.name",
+					"DefaultHDFSBolt");
+
+			success = initializeHDFSBolt(topology_name, component_name);
+			terminalComponents.add(component_name);
+
+			System.out.println("[OpenSOC] ------Component " + component_name
+					+ " initialized with the following settings:");
+
+			SettingsLoader.printConfigOptions((PropertiesConfiguration) config,
+					"bolt.hdfs");
+		}
+
+		if (config.getBoolean("bolt.error.indexing.enabled")) {
+			String component_name = config.getString(
+					"bolt.error.indexing.name", "DefaultErrorIndexingBolt");
+
+			success = initializeErrorIndexBolt(component_name);
+			terminalComponents.add(component_name);
+
+			System.out.println("[OpenSOC] ------Component " + component_name
+					+ " initialized with the following settings:");
+
+			SettingsLoader.printConfigOptions((PropertiesConfiguration) config,
+					"bolt.error");
+		}
+
+		if (config.containsKey("bolt.hbase.enabled")
+				&& config.getBoolean("bolt.hbase.enabled")) {
+			String component_name = config.getString("bolt.hbase.name",
+					"DefaultHbaseBolt");
+
+			String shuffleType = config.getString("bolt.hbase.shuffle.type",
+					"direct");
+			success = initializeHbaseBolt(component_name, shuffleType);
+			terminalComponents.add(component_name);
+
+			System.out.println("[OpenSOC] ------Component " + component_name
+					+ " initialized with the following settings:");
+
+			SettingsLoader.printConfigOptions((PropertiesConfiguration) config,
+					"bolt.hbase");
+		}
+
+		System.out.println("[OpenSOC] Topology Summary: ");
+		System.out.println("[OpenSOC] Message Stream: "
+				+ printComponentStream(messageComponents));
+		System.out.println("[OpenSOC] Alerts Stream: "
+				+ printComponentStream(alertComponents));
+		System.out.println("[OpenSOC] Error Stream: "
+				+ printComponentStream(errorComponents));
+		System.out.println("[OpenSOC] Data Stream: "
+				+ printComponentStream(dataComponents));
+		System.out.println("[OpenSOC] Terminal Components: "
+				+ printComponentStream(terminalComponents));
+
+		if (local_mode) {
+			conf.setNumWorkers(config.getInt("num.workers"));
+			conf.setMaxTaskParallelism(1);
+			LocalCluster cluster = new LocalCluster();
+			cluster.submitTopology(topology_name, conf,
+					builder.createTopology());
+		} else {
+
+			conf.setNumWorkers(config.getInt("num.workers"));
+			StormSubmitter.submitTopology(topology_name, conf,
+					builder.createTopology());
+		}
+
+	}
+
+	private String printComponentStream(List<String> messageComponents) {
+		StringBuilder print_string = new StringBuilder();
+
+		for (String component : messageComponents) {
+			print_string.append(component + " -> ");
+		}
+
+		print_string.append("[TERMINAL COMPONENT]");
+
+		return print_string.toString();
+	}
+
+	public boolean initializeHbaseBolt(String name, String shuffleType) {
+
+		try {
+
+			String messageUpstreamComponent = dataComponents.get(dataComponents
+					.size()-1);
+
+			System.out.println("[OpenSOC] ------" + name
+					+ " is initializing from " + messageUpstreamComponent);
+
+			String tableName = config.getString("bolt.hbase.table.name")
+					.toString();
+			TupleTableConfig hbaseBoltConfig = new TupleTableConfig(tableName,
+					config.getString("bolt.hbase.table.key.tuple.field.name")
+							.toString(), config.getString(
+							"bolt.hbase.table.timestamp.tuple.field.name")
+							.toString());
+
+			String allColumnFamiliesColumnQualifiers = config.getString(
+					"bolt.hbase.table.fields").toString();
+			// This is expected in the form
+			// "<cf1>:<cq11>,<cq12>,<cq13>|<cf2>:<cq21>,<cq22>|......."
+			String[] tokenizedColumnFamiliesWithColumnQualifiers = StringUtils
+					.split(allColumnFamiliesColumnQualifiers, "\\|");
+			for (String tokenizedColumnFamilyWithColumnQualifiers : tokenizedColumnFamiliesWithColumnQualifiers) {
+				String[] cfCqTokens = StringUtils.split(
+						tokenizedColumnFamilyWithColumnQualifiers, ":");
+				String columnFamily = cfCqTokens[0];
+				String[] columnQualifiers = StringUtils.split(cfCqTokens[1],
+						",");
+				for (String columnQualifier : columnQualifiers) {
+					hbaseBoltConfig.addColumn(columnFamily, columnQualifier);
+				}
+
+				// hbaseBoltConfig.setDurability(Durability.valueOf(conf.get(
+				// "storm.topology.pcap.bolt.hbase.durability").toString()));
+
+				hbaseBoltConfig.setBatch(Boolean.valueOf(config.getString(
+						"bolt.hbase.enable.batching").toString()));
+
+				HBaseBolt hbase_bolt = new HBaseBolt(hbaseBoltConfig,
+						config.getString("kafka.zk.list"),
+						config.getString("kafka.zk.port"));
+				hbase_bolt.setAutoAck(true);
+
+				BoltDeclarer declarer = builder.setBolt(name, hbase_bolt,
+						config.getInt("bolt.hbase.parallelism.hint"))
+						.setNumTasks(config.getInt("bolt.hbase.num.tasks"));
+
+				if (Grouping._Fields.CUSTOM_OBJECT.toString().equalsIgnoreCase(
+						shuffleType)) {
+					declarer.customGrouping(
+							messageUpstreamComponent,
+							"pcap_data_stream",
+							new HBaseStreamPartitioner(
+									hbaseBoltConfig.getTableName(),
+									0,
+									Integer.parseInt(conf
+											.get("bolt.hbase.partitioner.region.info.refresh.interval.mins")
+											.toString())));
+				} else if (Grouping._Fields.DIRECT.toString().equalsIgnoreCase(
+						shuffleType)) {
+					declarer.fieldsGrouping(messageUpstreamComponent,
+							"pcap_data_stream", new Fields("pcap_id"));
+				}
+
+			}
+		} catch (Exception e) {
+			e.printStackTrace();
+			System.exit(0);
+		}
+		return true;
+	}
+
+	private boolean initializeErrorIndexBolt(String component_name) {
+		try {
+
+			TelemetryIndexingBolt indexing_bolt = new TelemetryIndexingBolt()
+					.withIndexIP(config.getString("es.ip"))
+					.withIndexPort(config.getInt("es.port"))
+					.withClusterName(config.getString("es.clustername"))
+					.withIndexName(
+							config.getString("bolt.error.indexing.indexname"))
+					.withDocumentName(
+							config.getString("bolt.error.indexing.documentname"))
+					.withBulk(config.getInt("bolt.error.indexing.bulk"))
+					.withIndexAdapter(new ESBaseBulkAdapter())
+					.withMetricConfiguration(config);
+
+			BoltDeclarer declarer = builder
+					.setBolt(
+							component_name,
+							indexing_bolt,
+							config.getInt("bolt.error.indexing.parallelism.hint"))
+					.setNumTasks(config.getInt("bolt.error.indexing.num.tasks"));
+
+			for (String component : errorComponents)
+				declarer.shuffleGrouping(component, "error");
+
+			return true;
+		} catch (Exception e) {
+			e.printStackTrace();
+			return false;
+		}
+
+	}
+
+	private boolean initializeKafkaSpout(String name) {
+		try {
+
+			BrokerHosts zk = new ZkHosts(config.getString("kafka.zk"));
+			String input_topic = config.getString("spout.kafka.topic");
+			SpoutConfig kafkaConfig = new SpoutConfig(zk, input_topic, "",
+					input_topic);
+			kafkaConfig.scheme = new SchemeAsMultiScheme(new RawScheme());
+			kafkaConfig.forceFromStart = Boolean.valueOf("True");
+			kafkaConfig.startOffsetTime = -1;
+
+			builder.setSpout(name, new KafkaSpout(kafkaConfig),
+					config.getInt("spout.kafka.parallelism.hint")).setNumTasks(
+					config.getInt("spout.kafka.num.tasks"));
+
+		} catch (Exception e) {
+			e.printStackTrace();
+			System.exit(0);
+		}
+
+		return true;
+	}
+
+	abstract boolean initializeParsingBolt(String topology_name, String name);
+
+	abstract boolean initializeTestingSpout(String name);
+
+	private boolean initializeGeoEnrichment(String topology_name, String name) {
+
+		try {
+			String messageUpstreamComponent = messageComponents
+					.get(messageComponents.size() - 1);
+
+			System.out.println("[OpenSOC] ------" + name
+					+ " is initializing from " + messageUpstreamComponent);
+
+			List<String> geo_keys = new ArrayList<String>();
+			geo_keys.add(config.getString("source.ip"));
+			geo_keys.add(config.getString("dest.ip"));
+
+			GeoMysqlAdapter geo_adapter = new GeoMysqlAdapter(
+					config.getString("mysql.ip"), config.getInt("mysql.port"),
+					config.getString("mysql.username"),
+					config.getString("mysql.password"),
+					config.getString("bolt.enrichment.geo.adapter.table"));
+
+			GenericEnrichmentBolt geo_enrichment = new GenericEnrichmentBolt()
+					.withEnrichmentTag(
+							config.getString("bolt.enrichment.geo.enrichment_tag"))
+					.withOutputFieldName(topology_name)
+					.withAdapter(geo_adapter)
+					.withMaxTimeRetain(
+							config.getInt("bolt.enrichment.geo.MAX_TIME_RETAIN"))
+					.withMaxCacheSize(
+							config.getInt("bolt.enrichment.geo.MAX_CACHE_SIZE"))
+					.withKeys(geo_keys).withMetricConfiguration(config);
+
+			builder.setBolt(name, geo_enrichment,
+					config.getInt("bolt.enrichment.geo.parallelism.hint"))
+					.fieldsGrouping(messageUpstreamComponent, "message",
+							new Fields("key"))
+					.setNumTasks(config.getInt("bolt.enrichment.geo.num.tasks"));
+
+		} catch (Exception e) {
+			e.printStackTrace();
+			System.exit(0);
+		}
+
+		return true;
+	}
+
+	private boolean initializeHostsEnrichment(String topology_name,
+			String name, String hosts_path) {
+
+		try {
+
+			String messageUpstreamComponent = messageComponents
+					.get(messageComponents.size() - 1);
+
+			System.out.println("[OpenSOC] ------" + name
+					+ " is initializing from " + messageUpstreamComponent);
+
+			List<String> hosts_keys = new ArrayList<String>();
+			hosts_keys.add(config.getString("source.ip"));
+			hosts_keys.add(config.getString("dest.ip"));
+
+			Map<String, JSONObject> known_hosts = SettingsLoader
+					.loadKnownHosts(hosts_path);
+
+			HostFromPropertiesFileAdapter host_adapter = new HostFromPropertiesFileAdapter(
+					known_hosts);
+
+			GenericEnrichmentBolt host_enrichment = new GenericEnrichmentBolt()
+					.withEnrichmentTag(
+							config.getString("bolt.enrichment.host.enrichment_tag"))
+					.withAdapter(host_adapter)
+					.withMaxTimeRetain(
+							config.getInt("bolt.enrichment.host.MAX_TIME_RETAIN"))
+					.withMaxCacheSize(
+							config.getInt("bolt.enrichment.host.MAX_CACHE_SIZE"))
+					.withOutputFieldName(topology_name).withKeys(hosts_keys)
+					.withMetricConfiguration(config);
+
+			builder.setBolt(name, host_enrichment,
+					config.getInt("bolt.enrichment.host.parallelism.hint"))
+					.fieldsGrouping(messageUpstreamComponent, "message",
+							new Fields("key"))
+					.setNumTasks(
+							config.getInt("bolt.enrichment.host.num.tasks"));
+
+		} catch (Exception e) {
+			e.printStackTrace();
+			System.exit(0);
+		}
+
+		return true;
+	}
+
+	private boolean initializeAlerts(String topology_name, String name,
+			String alerts_path, JSONObject environment_identifier,
+			JSONObject topology_identifier) {
+		try {
+
+			String messageUpstreamComponent = messageComponents
+					.get(messageComponents.size() - 1);
+
+			System.out.println("[OpenSOC] ------" + name
+					+ " is initializing from " + messageUpstreamComponent);
+
+			JSONObject alerts_identifier = SettingsLoader
+					.generateAlertsIdentifier(environment_identifier,
+							topology_identifier);
+
+			AlertsAdapter alerts_adapter = new HbaseWhiteAndBlacklistAdapter(
+					"ip_whitelist", "ip_blacklist",
+					config.getString("kafka.zk.list"),
+					config.getString("kafka.zk.port"), 3600, 1000);
+
+			TelemetryAlertsBolt alerts_bolt = new TelemetryAlertsBolt()
+					.withIdentifier(alerts_identifier).withMaxCacheSize(1000)
+					.withMaxTimeRetain(3600).withAlertsAdapter(alerts_adapter)
+					.withOutputFieldName("message")
+					.withMetricConfiguration(config);
+
+			builder.setBolt(name, alerts_bolt,
+					config.getInt("bolt.alerts.parallelism.hint"))
+					.fieldsGrouping(messageUpstreamComponent, "message",
+							new Fields("key"))
+					.setNumTasks(config.getInt("bolt.alerts.num.tasks"));
+
+		} catch (Exception e) {
+			e.printStackTrace();
+			System.exit(0);
+		}
+		return true;
+	}
+
+	private boolean initializeAlertIndexing(String name) {
+		String messageUpstreamComponent = alertComponents.get(alertComponents
+				.size() - 1);
+
+		System.out.println("[OpenSOC] ------" + name + " is initializing from "
+				+ messageUpstreamComponent);
+
+		TelemetryIndexingBolt indexing_bolt = new TelemetryIndexingBolt()
+				.withIndexIP(config.getString("es.ip"))
+				.withIndexPort(config.getInt("es.port"))
+				.withClusterName(config.getString("es.clustername"))
+				.withIndexName(
+						config.getString("bolt.alerts.indexing.indexname"))
+				.withDocumentName(
+						config.getString("bolt.alerts.indexing.documentname"))
+				.withBulk(config.getInt("bolt.alerts.indexing.bulk"))
+				.withIndexAdapter(new ESBaseBulkAdapter())
+				.withMetricConfiguration(config);
+
+		String alerts_name = config.getString("bolt.alerts.indexing.name");
+		builder.setBolt(alerts_name, indexing_bolt,
+				config.getInt("bolt.indexing.parallelism.hint"))
+				.shuffleGrouping(messageUpstreamComponent, "alert")
+				.setNumTasks(config.getInt("bolt.indexing.num.tasks"));
+
+		return true;
+	}
+
+	private boolean initializeKafkaBolt(String name) {
+		try {
+
+			String messageUpstreamComponent = messageComponents
+					.get(messageComponents.size() - 1);
+
+			System.out.println("[OpenSOC] ------" + name
+					+ " is initializing from " + messageUpstreamComponent);
+
+			Map<String, String> kafka_broker_properties = new HashMap<String, String>();
+			kafka_broker_properties.put("zk.connect",
+					config.getString("kafka.zk"));
+			kafka_broker_properties.put("metadata.broker.list",
+					config.getString("kafka.br"));
+
+			kafka_broker_properties.put("serializer.class",
+					"com.opensoc.json.serialization.JSONKafkaSerializer");
+
+			kafka_broker_properties.put("key.serializer.class",
+					"kafka.serializer.StringEncoder");
+
+			String output_topic = config.getString("bolt.kafka.topic");
+
+			conf.put("kafka.broker.properties", kafka_broker_properties);
+			conf.put("topic", output_topic);
+
+			builder.setBolt(name, new KafkaBolt<String, JSONObject>(),
+					config.getInt("bolt.kafka.parallelism.hint"))
+					.shuffleGrouping(messageUpstreamComponent, "message")
+					.setNumTasks(config.getInt("bolt.kafka.num.tasks"));
+		} catch (Exception e) {
+			e.printStackTrace();
+			System.exit(0);
+		}
+		return true;
+	}
+
+	private boolean initializeWhoisEnrichment(String topology_name, String name) {
+		try {
+
+			String messageUpstreamComponent = messageComponents
+					.get(messageComponents.size() - 1);
+
+			System.out.println("[OpenSOC] ------" + name
+					+ " is initializing from " + messageUpstreamComponent);
+
+			List<String> whois_keys = new ArrayList<String>();
+			String[] keys_from_settings = config.getString(
+					"bolt.enrichment.whois.source").split(",");
+
+			for (String key : keys_from_settings)
+				whois_keys.add(key);
+
+			EnrichmentAdapter whois_adapter = new WhoisHBaseAdapter(
+					config.getString("bolt.enrichment.whois.hbase.table.name"),
+					config.getString("kafka.zk.list"),
+					config.getString("kafka.zk.port"));
+
+			GenericEnrichmentBolt whois_enrichment = new GenericEnrichmentBolt()
+					.withEnrichmentTag(
+							config.getString("bolt.enrichment.whois.enrichment_tag"))
+					.withOutputFieldName(topology_name)
+					.withAdapter(whois_adapter)
+					.withMaxTimeRetain(
+							config.getInt("bolt.enrichment.whois.MAX_TIME_RETAIN"))
+					.withMaxCacheSize(
+							config.getInt("bolt.enrichment.whois.MAX_CACHE_SIZE"))
+					.withKeys(whois_keys).withMetricConfiguration(config);
+
+			builder.setBolt(name, whois_enrichment,
+					config.getInt("bolt.enrichment.whois.parallelism.hint"))
+					.fieldsGrouping(messageUpstreamComponent, "message",
+							new Fields("key"))
+					.setNumTasks(
+							config.getInt("bolt.enrichment.whois.num.tasks"));
+
+		} catch (Exception e) {
+			e.printStackTrace();
+			System.exit(0);
+		}
+
+		return true;
+	}
+
+	private boolean initializeIndexingBolt(String name) {
+		try {
+
+			String messageUpstreamComponent = messageComponents
+					.get(messageComponents.size() - 1);
+
+			System.out.println("[OpenSOC] ------" + name
+					+ " is initializing from " + messageUpstreamComponent);
+
+			TelemetryIndexingBolt indexing_bolt = new TelemetryIndexingBolt()
+					.withIndexIP(config.getString("es.ip"))
+					.withIndexPort(config.getInt("es.port"))
+					.withClusterName(config.getString("es.clustername"))
+					.withIndexName(config.getString("bolt.indexing.indexname"))
+					.withDocumentName(
+							config.getString("bolt.indexing.documentname"))
+					.withBulk(config.getInt("bolt.indexing.bulk"))
+					.withIndexAdapter(new ESTimedRotatingAdapter())
+					.withMetricConfiguration(config);
+
+			builder.setBolt(name, indexing_bolt,
+					config.getInt("bolt.indexing.parallelism.hint"))
+					.fieldsGrouping(messageUpstreamComponent, "message",
+							new Fields("key"))
+					.setNumTasks(config.getInt("bolt.indexing.num.tasks"));
+
+		} catch (Exception e) {
+			e.printStackTrace();
+			System.exit(0);
+		}
+
+		return true;
+	}
+
+	private boolean initializeCIFEnrichment(String topology_name, String name) {
+		try {
+
+			String messageUpstreamComponent = messageComponents
+					.get(messageComponents.size() - 1);
+
+			System.out.println("[OpenSOC] ------" + name
+					+ " is initializing from " + messageUpstreamComponent);
+
+			List<String> cif_keys = new ArrayList<String>();
+
+			cif_keys.add(config.getString("source.ip"));
+			cif_keys.add(config.getString("dest.ip"));
+			cif_keys.add(config.getString("bolt.enrichment.cif.host"));
+			cif_keys.add(config.getString("bolt.enrichment.cif.email"));
+
+			GenericEnrichmentBolt cif_enrichment = new GenericEnrichmentBolt()
+					.withEnrichmentTag(
+							config.getString("bolt.enrichment.cif.enrichment_tag"))
+					.withAdapter(
+							new CIFHbaseAdapter(config
+									.getString("kafka.zk.list"), config
+									.getString("kafka.zk.port"), config
+									.getString("bolt.enrichment.cif.tablename")))
+					.withOutputFieldName(topology_name)
+					.withEnrichmentTag("CIF_Enrichment")
+					.withKeys(cif_keys)
+					.withMaxTimeRetain(
+							config.getInt("bolt.enrichment.cif.MAX_TIME_RETAIN"))
+					.withMaxCacheSize(
+							config.getInt("bolt.enrichment.cif.MAX_CACHE_SIZE"))
+					.withMetricConfiguration(config);
+
+			builder.setBolt(name, cif_enrichment,
+					config.getInt("bolt.enrichment.cif.parallelism.hint"))
+					.fieldsGrouping(messageUpstreamComponent, "message",
+							new Fields("key"))
+					.setNumTasks(config.getInt("bolt.enrichment.cif.num.tasks"));
+
+		} catch (Exception e) {
+			e.printStackTrace();
+			System.exit(0);
+		}
+
+		return true;
+	}
+
+	private boolean initializeHDFSBolt(String topology_name, String name) {
+		try {
+
+			String messageUpstreamComponent = messageComponents
+					.get(messageComponents.size() - 1);
+
+			System.out.println("[OpenSOC] ------" + name
+					+ " is initializing from " + messageUpstreamComponent);
+
+			RecordFormat format = new DelimitedRecordFormat()
+					.withFieldDelimiter(
+							config.getString("bolt.hdfs.field.delimiter")
+									.toString()).withFields(
+							new Fields("message"));
+
+			// sync the file system after every x number of tuples
+			SyncPolicy syncPolicy = new CountSyncPolicy(Integer.valueOf(config
+					.getString("bolt.hdfs.batch.size").toString()));
+
+			// rotate files when they reach certain size
+			FileRotationPolicy rotationPolicy = new FileSizeRotationPolicy(
+					Float.valueOf(config.getString(
+							"bolt.hdfs.file.rotation.size.in.mb").toString()),
+					Units.MB);
+
+			FileNameFormat fileNameFormat = new DefaultFileNameFormat()
+					.withPath(config.getString("bolt.hdfs.wip.file.path")
+							.toString());
+
+			// Post rotate action
+			MoveFileAction moveFileAction = (new MoveFileAction())
+					.toDestination(config.getString(
+							"bolt.hdfs.finished.file.path").toString());
+
+			HdfsBolt hdfsBolt = new HdfsBolt()
+					.withFsUrl(
+							config.getString("bolt.hdfs.file.system.url")
+									.toString())
+					.withFileNameFormat(fileNameFormat)
+					.withRecordFormat(format)
+					.withRotationPolicy(rotationPolicy)
+					.withSyncPolicy(syncPolicy)
+					.addRotationAction(moveFileAction);
+			if (config.getString("bolt.hdfs.compression.codec.class") != null) {
+				hdfsBolt.withCompressionCodec(config.getString(
+						"bolt.hdfs.compression.codec.class").toString());
+			}
+
+			builder.setBolt(name, hdfsBolt,
+					config.getInt("bolt.hdfs.parallelism.hint"))
+					.shuffleGrouping(messageUpstreamComponent, "message")
+					.setNumTasks(config.getInt("bolt.hdfs.num.tasks"));
+
+		} catch (Exception e) {
+			e.printStackTrace();
+			System.exit(0);
+		}
+
+		return true;
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/05e188ba/opensoc-streaming/OpenSOC-Topologies/src/main/resources/OpenSOC_Configs/etc/env/environment_common.conf
----------------------------------------------------------------------
diff --git a/opensoc-streaming/OpenSOC-Topologies/src/main/resources/OpenSOC_Configs/etc/env/environment_common.conf b/opensoc-streaming/OpenSOC-Topologies/src/main/resources/OpenSOC_Configs/etc/env/environment_common.conf
new file mode 100644
index 0000000..f93921a
--- /dev/null
+++ b/opensoc-streaming/OpenSOC-Topologies/src/main/resources/OpenSOC_Configs/etc/env/environment_common.conf
@@ -0,0 +1,4 @@
+kafka.zk.port=2181
+kafka.zk.list=zkpr1,zkpr2,zkpr3
+kafka.zk=zkpr1:2181,zkpr2:2181,zkpr3:2181
+kafka.br=dn01:9092,dn02:9092,dn03:9092,dn04:9092,dn05:9092,dn06:9092,dn07:9092,dn08:9092
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/05e188ba/opensoc-streaming/OpenSOC-Topologies/src/main/resources/OpenSOC_Configs/etc/env/es_connection.conf
----------------------------------------------------------------------
diff --git a/opensoc-streaming/OpenSOC-Topologies/src/main/resources/OpenSOC_Configs/etc/env/es_connection.conf b/opensoc-streaming/OpenSOC-Topologies/src/main/resources/OpenSOC_Configs/etc/env/es_connection.conf
new file mode 100644
index 0000000..bb8861c
--- /dev/null
+++ b/opensoc-streaming/OpenSOC-Topologies/src/main/resources/OpenSOC_Configs/etc/env/es_connection.conf
@@ -0,0 +1,3 @@
+es.ip=esp
+es.port=9300
+es.clustername=devo_es
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/05e188ba/opensoc-streaming/OpenSOC-Topologies/src/main/resources/OpenSOC_Configs/etc/env/hdfs_connection.conf
----------------------------------------------------------------------
diff --git a/opensoc-streaming/OpenSOC-Topologies/src/main/resources/OpenSOC_Configs/etc/env/hdfs_connection.conf b/opensoc-streaming/OpenSOC-Topologies/src/main/resources/OpenSOC_Configs/etc/env/hdfs_connection.conf
new file mode 100644
index 0000000..f7e7f2b
--- /dev/null
+++ b/opensoc-streaming/OpenSOC-Topologies/src/main/resources/OpenSOC_Configs/etc/env/hdfs_connection.conf
@@ -0,0 +1,2 @@
+bolt.hdfs.IP=nn1
+bolt.hdfs.port=8020
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/05e188ba/opensoc-streaming/OpenSOC-Topologies/src/main/resources/OpenSOC_Configs/etc/env/mysql_connection.conf
----------------------------------------------------------------------
diff --git a/opensoc-streaming/OpenSOC-Topologies/src/main/resources/OpenSOC_Configs/etc/env/mysql_connection.conf b/opensoc-streaming/OpenSOC-Topologies/src/main/resources/OpenSOC_Configs/etc/env/mysql_connection.conf
new file mode 100644
index 0000000..15690af
--- /dev/null
+++ b/opensoc-streaming/OpenSOC-Topologies/src/main/resources/OpenSOC_Configs/etc/env/mysql_connection.conf
@@ -0,0 +1,4 @@
+mysql.ip=mysql
+mysql.port=0
+mysql.username=test
+mysql.password=123123
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/05e188ba/opensoc-streaming/OpenSOC-Topologies/src/main/resources/OpenSOC_Configs/etc/hbase-site.xml
----------------------------------------------------------------------
diff --git a/opensoc-streaming/OpenSOC-Topologies/src/main/resources/OpenSOC_Configs/etc/hbase-site.xml b/opensoc-streaming/OpenSOC-Topologies/src/main/resources/OpenSOC_Configs/etc/hbase-site.xml
new file mode 100644
index 0000000..5c3c819
--- /dev/null
+++ b/opensoc-streaming/OpenSOC-Topologies/src/main/resources/OpenSOC_Configs/etc/hbase-site.xml
@@ -0,0 +1,127 @@
+<!--Tue Apr  1 18:16:39 2014-->
+  <configuration>
+    <property>
+    <name>hbase.tmp.dir</name>
+    <value>/disk/h/hbase</value>
+  </property>
+    <property>
+    <name>hbase.hregion.memstore.chunkpool.maxsize</name>
+    <value>0.5</value>
+  </property>
+    <property>
+    <name>hbase.regionserver.codecs</name>
+    <value>lzo,gz,snappy</value>
+  </property>
+    <property>
+    <name>hbase.hstore.flush.retries.number</name>
+    <value>120</value>
+  </property>
+    <property>
+    <name>hbase.client.keyvalue.maxsize</name>
+    <value>10485760</value>
+  </property>
+    <property>
+    <name>hbase.rootdir</name>
+    <value>hdfs://nn1:8020/apps/hbase/data</value>
+  </property>
+    <property>
+    <name>hbase.defaults.for.version.skip</name>
+    <value>true</value>
+  </property>
+    <property>
+    <name>hbase.client.scanner.caching</name>
+    <value>100</value>
+  </property>
+    <property>
+    <name>hbase.superuser</name>
+    <value>hbase</value>
+  </property>
+    <property>
+    <name>hfile.block.cache.size</name>
+    <value>0.40</value>
+  </property>
+    <property>
+    <name>hbase.regionserver.checksum.verify</name>
+    <value>true</value>
+  </property>
+    <property>
+    <name>hbase.hregion.memstore.mslab.enabled</name>
+    <value>true</value>
+  </property>
+    <property>
+    <name>hbase.hregion.max.filesize</name>
+    <value>107374182400</value>
+  </property>
+    <property>
+    <name>hbase.cluster.distributed</name>
+    <value>true</value>
+  </property>
+    <property>
+    <name>zookeeper.session.timeout</name>
+    <value>30000</value>
+  </property>
+    <property>
+    <name>zookeeper.znode.parent</name>
+    <value>/hbase-unsecure</value>
+  </property>
+    <property>
+    <name>hbase.regionserver.global.memstore.lowerLimit</name>
+    <value>0.38</value>
+  </property>
+    <property>
+    <name>hbase.regionserver.handler.count</name>
+    <value>240</value>
+  </property>
+    <property>
+    <name>hbase.hregion.memstore.mslab.chunksize</name>
+    <value>8388608</value>
+  </property>
+    <property>
+    <name>hbase.zookeeper.quorum</name>
+    <value>zkpr1,zkpr2,zkpr3</value>
+  </property>
+    <property>
+    <name>hbase.zookeeper.useMulti</name>
+    <value>true</value>
+  </property>
+    <property>
+    <name>hbase.hregion.majorcompaction</name>
+    <value>86400000</value>
+  </property>
+    <property>
+    <name>hbase.hstore.blockingStoreFiles</name>
+    <value>200</value>
+  </property>
+    <property>
+    <name>hbase.zookeeper.property.clientPort</name>
+    <value>2181</value>
+  </property>
+    <property>
+    <name>hbase.hregion.memstore.flush.size</name>
+    <value>134217728</value>
+  </property>
+    <property>
+    <name>hbase.security.authorization</name>
+    <value>false</value>
+  </property>
+    <property>
+    <name>hbase.regionserver.global.memstore.upperLimit</name>
+    <value>0.4</value>
+  </property>
+    <property>
+    <name>hbase.hstore.compactionThreshold</name>
+    <value>4</value>
+  </property>
+    <property>
+    <name>hbase.hregion.memstore.block.multiplier</name>
+    <value>8</value>
+  </property>
+    <property>
+    <name>hbase.security.authentication</name>
+    <value>simple</value>
+  </property>
+    <property>
+    <name>dfs.client.read.shortcircuit</name>
+    <value>true</value>
+  </property>
+  </configuration>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/05e188ba/opensoc-streaming/OpenSOC-Topologies/src/main/resources/OpenSOC_Configs/etc/whitelists/known_hosts.conf
----------------------------------------------------------------------
diff --git a/opensoc-streaming/OpenSOC-Topologies/src/main/resources/OpenSOC_Configs/etc/whitelists/known_hosts.conf b/opensoc-streaming/OpenSOC-Topologies/src/main/resources/OpenSOC_Configs/etc/whitelists/known_hosts.conf
new file mode 100644
index 0000000..1c11207
--- /dev/null
+++ b/opensoc-streaming/OpenSOC-Topologies/src/main/resources/OpenSOC_Configs/etc/whitelists/known_hosts.conf
@@ -0,0 +1,3 @@
+10.1.128.236={"local":"YES", "type":"webserver", "asset_value" : "important"}
+10.1.128.237={"local":"UNKNOWN", "type":"unknown", "asset_value" : "important"}
+10.60.10.254={"local":"YES", "type":"printer", "asset_value" : "important"}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/05e188ba/opensoc-streaming/OpenSOC-Topologies/src/main/resources/OpenSOC_Configs/topologies/bro/alerts.xml
----------------------------------------------------------------------
diff --git a/opensoc-streaming/OpenSOC-Topologies/src/main/resources/OpenSOC_Configs/topologies/bro/alerts.xml b/opensoc-streaming/OpenSOC-Topologies/src/main/resources/OpenSOC_Configs/topologies/bro/alerts.xml
new file mode 100644
index 0000000..3016afb
--- /dev/null
+++ b/opensoc-streaming/OpenSOC-Topologies/src/main/resources/OpenSOC_Configs/topologies/bro/alerts.xml
@@ -0,0 +1,24 @@
+<?xml version="1.0" encoding="ISO-8859-1" ?>
+<rule-definitions>
+	<rule>
+		<pattern>.*host\"\:\{"ip_dst_addr\"\:\{\},\"ip_src_addr\"\:\{\}.*</pattern>
+		<alert>{"type":"error","priority":5, "title":"No Local Hostname Present", "body":
+			"We don't have a record for source or destination IPs in our internal database."}
+		</alert>
+	</rule>
+	<rule>
+		<pattern>.*whois\"\:\{\"tld\"\:\{\}.*</pattern>
+		<alert>{"type":"warning","priority":10, "title":"Whois domain unknown", "body":
+			"Could not locate whois information for tld"}</alert>
+	</rule>
+	<rule>
+		<pattern>^((?!country\"\:\"US\").)*$</pattern>
+		<alert>{"type":"warning","priority":10, "title":"NOT US IP", "body": "Communication contains a non-US IP"}</alert>
+	</rule>
+	<rule>
+		<pattern>.*geo.*</pattern>
+		<alert>{"type":"error","priority":1, "title":"test", "body": "test alert"}</alert>
+	</rule>
+</rule-definitions>
+
+

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/05e188ba/opensoc-streaming/OpenSOC-Topologies/src/main/resources/OpenSOC_Configs/topologies/bro/features_enabled.conf
----------------------------------------------------------------------
diff --git a/opensoc-streaming/OpenSOC-Topologies/src/main/resources/OpenSOC_Configs/topologies/bro/features_enabled.conf b/opensoc-streaming/OpenSOC-Topologies/src/main/resources/OpenSOC_Configs/topologies/bro/features_enabled.conf
new file mode 100644
index 0000000..ef677f3
--- /dev/null
+++ b/opensoc-streaming/OpenSOC-Topologies/src/main/resources/OpenSOC_Configs/topologies/bro/features_enabled.conf
@@ -0,0 +1,105 @@
+#Enable and disable features for each topology
+
+#Feature: Test spout 
+##Feature Description: Reads telemetry from file and ingests it into topology.  Used for testing or bulk loading the topology
+
+spout.test.name=TestSpout
+spout.test.enabled=true
+spout.test.num.tasks=1
+spout.test.parallelism.hint=1
+
+#Feature: Kafka spout
+##Feature Description: Acts as a Kafka consumer.  Takes messages from a Kafka topic and ingests them into a topology
+
+spout.kafka.name=KafkaSpout
+spout.kafka.enabled=false
+spout.kafka.num.tasks=1
+spout.kafka.parallelism.hint=1
+
+#Feature: Parser Bolt
+##Feature Description: Parses telemetry from its native format into a native JSON
+
+parser.bolt.name=ParserBolt
+bolt.parser.enabled=true
+bolt.parser.num.tasks=1
+bolt.parser.parallelism.hint=1
+
+#Feature: Host Enrichment
+##Feature Description: Appends information about known hosts to a telemetry message
+
+bolt.enrichment.host.name=HostEnrichment
+bolt.enrichment.host.enabled=true
+bolt.enrichment.host.num.tasks=1
+bolt.enrichment.host.parallelism.hint=1
+
+#Feature: Geo Enrichment
+##Feature Description: Appends geo information about known non-local IPs to a telemetry message
+
+bolt.enrichment.geo.name=GeoEnrichment 
+bolt.enrichment.geo.enabled=true
+bolt.enrichment.geo.num.tasks=1
+bolt.enrichment.geo.parallelism.hint=1
+
+#Feature: Whois Enrichment
+##Feature Description: Appends whois information about known domains to a telemetry message
+
+bolt.enrichment.whois.name=WhoisEnrichment
+bolt.enrichment.whois.enabled=false
+bolt.enrichment.whois.num.tasks=1
+bolt.enrichment.whois.parallelism.hint=1
+
+#Feature: CIF Enrichment
+##Feature Description: Appends information from CIF threat intelligence feeds to a telemetry message
+
+bolt.enrichment.cif.name=SIFBolt
+bolt.enrichment.cif.enabled=false
+bolt.enrichment.cif.num.tasks=1
+bolt.enrichment.cif.parallelism.hint=1
+
+#Feature: Rules-Based Alerts
+##Feature Description: Tags messages with rules-based alerts
+
+bolt.alerts.name=Alerts
+bolt.alerts.enabled=true
+bolt.alerts.num.tasks=1
+bolt.alerts.parallelism.hint=1
+
+#Feature: Indexer
+##Feature Description: Indexes telemetry messages in ElasticSearch or Solr
+
+bolt.indexing.name=IndexBolt
+bolt.indexing.enabled=true
+bolt.indexing.num.tasks=1
+bolt.indexing.parallelism.hint=1
+
+#Feature: Alerts Indexer
+##Feature Description: Indexes alert messages in ElasticSearch or Solr
+
+bolt.alerts.indexing.name=AlertIndexBolt
+bolt.alerts.indexing.enabled=true
+bolt.alerts.indexing.num.tasks=1
+bolt.alerts.indexing.parallelism.hint=1
+
+#Feature: Error Indexer
+##Feature Description: Indexes error messages in ElasticSearch or Solr
+
+bolt.error.indexing.name=ErrorIndexBolt
+bolt.error.indexing.enabled=true
+bolt.error.indexing.num.tasks=1
+bolt.error.indexing.parallelism.hint=1
+
+#Feature: Kafka Bolt
+##Feature Description: Writes telemetry messages back into a Kafka topic
+
+bolt.kafka.name=KafkaBolt
+bolt.kafka.enabled=true
+bolt.kafka.num.tasks=1
+bolt.kafka.parallelism.hint=1
+
+#Feature: HDFS Bolt
+##Feature Description: Writes telemetry messages into HDFS
+
+bolt.hdfs.name=HDFSBolt
+bolt.hdfs.enabled=true
+bolt.hdfs.num.tasks=1
+bolt.hdfs.parallelism.hint=1
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/05e188ba/opensoc-streaming/OpenSOC-Topologies/src/main/resources/OpenSOC_Configs/topologies/bro/metrics.conf
----------------------------------------------------------------------
diff --git a/opensoc-streaming/OpenSOC-Topologies/src/main/resources/OpenSOC_Configs/topologies/bro/metrics.conf b/opensoc-streaming/OpenSOC-Topologies/src/main/resources/OpenSOC_Configs/topologies/bro/metrics.conf
new file mode 100644
index 0000000..1daef3d
--- /dev/null
+++ b/opensoc-streaming/OpenSOC-Topologies/src/main/resources/OpenSOC_Configs/topologies/bro/metrics.conf
@@ -0,0 +1,26 @@
+#reporters
+com.opensoc.metrics.reporter.graphite=true
+com.opensoc.metrics.reporter.console=false
+com.opensoc.metrics.reporter.jmx=false
+
+#Graphite Addresses
+
+com.opensoc.metrics.graphite.address=localhost
+com.opensoc.metrics.graphite.port=2023
+
+#TelemetryParserBolt
+com.opensoc.metrics.TelemetryParserBolt.acks=true
+com.opensoc.metrics.TelemetryParserBolt.emits=true
+com.opensoc.metrics.TelemetryParserBolt.fails=true
+
+
+#GenericEnrichmentBolt
+com.opensoc.metrics.GenericEnrichmentBolt.acks=true
+com.opensoc.metrics.GenericEnrichmentBolt.emits=true
+com.opensoc.metrics.GenericEnrichmentBolt.fails=true
+
+
+#TelemetryIndexingBolt
+com.opensoc.metrics.TelemetryIndexingBolt.acks=true
+com.opensoc.metrics.TelemetryIndexingBolt.emits=true
+com.opensoc.metrics.TelemetryIndexingBolt.fails=true

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/05e188ba/opensoc-streaming/OpenSOC-Topologies/src/main/resources/OpenSOC_Configs/topologies/bro/topology.conf
----------------------------------------------------------------------
diff --git a/opensoc-streaming/OpenSOC-Topologies/src/main/resources/OpenSOC_Configs/topologies/bro/topology.conf b/opensoc-streaming/OpenSOC-Topologies/src/main/resources/OpenSOC_Configs/topologies/bro/topology.conf
new file mode 100644
index 0000000..6012056
--- /dev/null
+++ b/opensoc-streaming/OpenSOC-Topologies/src/main/resources/OpenSOC_Configs/topologies/bro/topology.conf
@@ -0,0 +1,88 @@
+include = ../../etc/env/environment_common.conf
+include = ../../etc/env/es_connection.conf
+include = ../../etc/env/hdfs_connection.conf
+include = ../../etc/env/mysql_connection.conf
+include = metrics.conf
+include = features_enabled.conf
+
+#Global Properties
+
+debug.mode=true
+local.mode=true
+num.workers=1
+
+#Standard 5-tuple fields
+
+source.ip=ip_src_addr
+source.port=ip_src_port
+dest.ip=ip_dst_addr
+dest.port=ip_dst_port
+protocol=protocol
+
+#Test Spout
+spout.test.parallelism.repeat=false
+
+#Kafka Spout
+spout.kafka.topic=bro_raw
+
+#Parsing Bolt
+source.include.protocols=snmp,http,ftp,ssh,ssl,dns,socks,dnp3,smtp,dhcp,modbus,radius,irc
+source.exclude.protocols=x509,files,app_stats
+
+#Host Enrichment
+
+bolt.enrichment.host.MAX_CACHE_SIZE=10000
+bolt.enrichment.host.MAX_TIME_RETAIN=10
+bolt.enrichment.host.enrichment_tag=host
+
+
+#GeoEnrichment
+
+bolt.enrichment.geo.enrichment_tag=geo
+bolt.enrichment.geo.adapter.table=GEO
+bolt.enrichment.geo.MAX_CACHE_SIZE=10000
+bolt.enrichment.geo.MAX_TIME_RETAIN=10
+
+#WhoisEnrichment
+
+bolt.enrichment.whois.hbase.table.name=whois
+bolt.enrichment.whois.enrichment_tag=whois
+bolt.enrichment.whois.source=tld
+bolt.enrichment.whois.MAX_CACHE_SIZE=10000
+bolt.enrichment.whois.MAX_TIME_RETAIN=10
+
+#CIF Enrichment
+bolt.enrichment.cif.tablename=cif_table
+bolt.enrichment.cif.host=tld
+bolt.enrichment.cif.email=email
+bolt.enrichment.cif.MAX_CACHE_SIZE=10000
+bolt.enrichment.cif.MAX_TIME_RETAIN=10
+bolt.enrichment.cif.enrichment_tag=cif
+
+
+#Indexing Bolt
+bolt.indexing.indexname=bro_index
+bolt.indexing.documentname=bro_doc
+bolt.indexing.bulk=200
+
+#Alerts Indexing Bolt
+bolt.alerts.indexing.indexname=alert
+bolt.alerts.indexing.documentname=bro_alert
+bolt.alerts.indexing.bulk=1
+
+#Error Indexing Bolt
+bolt.error.indexing.indexname=error
+bolt.error.indexing.documentname=bro_error
+bolt.error.indexing.bulk=1
+
+#HDFS Bolt
+bolt.hdfs.batch.size=5000
+bolt.hdfs.field.delimiter=|
+bolt.hdfs.file.rotation.size.in.mb=5
+bolt.hdfs.file.system.url=hdfs://nn1:8020
+bolt.hdfs.wip.file.path=/bro/wip
+bolt.hdfs.finished.file.path=/bro/rotated
+bolt.hdfs.compression.codec.class=org.apache.hadoop.io.compress.SnappyCodec
+
+#Kafka Bolt
+bolt.kafka.topic=bro_enriched
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/05e188ba/opensoc-streaming/OpenSOC-Topologies/src/main/resources/OpenSOC_Configs/topologies/bro/topology_identifier.conf
----------------------------------------------------------------------
diff --git a/opensoc-streaming/OpenSOC-Topologies/src/main/resources/OpenSOC_Configs/topologies/bro/topology_identifier.conf b/opensoc-streaming/OpenSOC-Topologies/src/main/resources/OpenSOC_Configs/topologies/bro/topology_identifier.conf
new file mode 100644
index 0000000..bb72783
--- /dev/null
+++ b/opensoc-streaming/OpenSOC-Topologies/src/main/resources/OpenSOC_Configs/topologies/bro/topology_identifier.conf
@@ -0,0 +1,4 @@
+#Each topology must have a unique identifier.  This setting is required
+
+topology.id=bro
+instance.id=B001
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/05e188ba/opensoc-streaming/OpenSOC-Topologies/src/main/resources/OpenSOC_Configs/topologies/environment_identifier.conf
----------------------------------------------------------------------
diff --git a/opensoc-streaming/OpenSOC-Topologies/src/main/resources/OpenSOC_Configs/topologies/environment_identifier.conf b/opensoc-streaming/OpenSOC-Topologies/src/main/resources/OpenSOC_Configs/topologies/environment_identifier.conf
new file mode 100644
index 0000000..4e8e005
--- /dev/null
+++ b/opensoc-streaming/OpenSOC-Topologies/src/main/resources/OpenSOC_Configs/topologies/environment_identifier.conf
@@ -0,0 +1,5 @@
+#This file identifies the cluster instance
+
+customer.id=mtd
+datacenter.id=allen
+instance.id=dev
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/05e188ba/opensoc-streaming/OpenSOC-Topologies/src/main/resources/OpenSOC_Configs/topologies/ise/alerts.xml
----------------------------------------------------------------------
diff --git a/opensoc-streaming/OpenSOC-Topologies/src/main/resources/OpenSOC_Configs/topologies/ise/alerts.xml b/opensoc-streaming/OpenSOC-Topologies/src/main/resources/OpenSOC_Configs/topologies/ise/alerts.xml
new file mode 100644
index 0000000..f36b881
--- /dev/null
+++ b/opensoc-streaming/OpenSOC-Topologies/src/main/resources/OpenSOC_Configs/topologies/ise/alerts.xml
@@ -0,0 +1,11 @@
+<?xml version="1.0" encoding="ISO-8859-1" ?>
+<rule-definitions>
+	<rule>
+		<pattern>.*message.*</pattern>
+		<alert>{"type":"alert","priority":5, "title":"ISE Alert", "body":
+			"Alert triggered by ISE"}
+		</alert>
+	</rule>
+</rule-definitions>
+
+

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/05e188ba/opensoc-streaming/OpenSOC-Topologies/src/main/resources/OpenSOC_Configs/topologies/ise/features_enabled.conf
----------------------------------------------------------------------
diff --git a/opensoc-streaming/OpenSOC-Topologies/src/main/resources/OpenSOC_Configs/topologies/ise/features_enabled.conf b/opensoc-streaming/OpenSOC-Topologies/src/main/resources/OpenSOC_Configs/topologies/ise/features_enabled.conf
new file mode 100644
index 0000000..486eea5
--- /dev/null
+++ b/opensoc-streaming/OpenSOC-Topologies/src/main/resources/OpenSOC_Configs/topologies/ise/features_enabled.conf
@@ -0,0 +1,105 @@
+#Enable and disable features for each topology
+
+#Feature: Test spout 
+##Feature Description: Reads telemetry from file and ingests it into topology.  Used for testing or bulk loading the topology
+
+spout.test.name=TestSpout
+spout.test.enabled=true
+spout.test.num.tasks=1
+spout.test.parallelism.hint=1
+
+#Feature: Kafka spout
+##Feature Description: Acts as a Kafka consumer.  Takes messages from a Kafka topic and ingests them into a topology
+
+spout.kafka.name=KafkaSpout
+spout.kafka.enabled=false
+spout.kafka.num.tasks=1
+spout.kafka.parallelism.hint=1
+
+#Feature: Parser Bolt
+##Feature Description: Parses telemetry from its native format into a native JSON
+
+parser.bolt.name=ParserBolt
+bolt.parser.enabled=true
+bolt.parser.num.tasks=1
+bolt.parser.parallelism.hint=1
+
+#Feature: Host Enrichment
+##Feature Description: Appends information about known hosts to a telemetry message
+
+bolt.enrichment.host.name=HostEnrichment
+bolt.enrichment.host.enabled=false
+bolt.enrichment.host.num.tasks=1
+bolt.enrichment.host.parallelism.hint=1
+
+#Feature: Geo Enrichment
+##Feature Description: Appends geo information about known non-local IPs to a telemetry message
+
+bolt.enrichment.geo.name=GeoEnrichment 
+bolt.enrichment.geo.enabled=false
+bolt.enrichment.geo.num.tasks=1
+bolt.enrichment.geo.parallelism.hint=1
+
+#Feature: Whois Enrichment
+##Feature Description: Appends whois information about known domains to a telemetry message
+
+bolt.enrichment.whois.name=WhoisEnrichment
+bolt.enrichment.whois.enabled=false
+bolt.enrichment.whois.num.tasks=1
+bolt.enrichment.whois.parallelism.hint=1
+
+#Feature: CIF Enrichment
+##Feature Description: Appends information from CIF threat intelligence feeds to a telemetry message
+
+bolt.enrichment.cif.name=SIFBolt
+bolt.enrichment.cif.enabled=false
+bolt.enrichment.cif.num.tasks=1
+bolt.enrichment.cif.parallelism.hint=1
+
+#Feature: Rules-Based Alerts
+##Feature Description: Tags messages with rules-based alerts
+
+bolt.alerts.name=Alerts
+bolt.alerts.enabled=false
+bolt.alerts.num.tasks=1
+bolt.alerts.parallelism.hint=1
+
+#Feature: Indexer
+##Feature Description: Indexes telemetry messages in ElasticSearch or Solr
+
+bolt.indexing.name=IndexBolt
+bolt.indexing.enabled=true
+bolt.indexing.num.tasks=1
+bolt.indexing.parallelism.hint=1
+
+#Feature: Alerts Indexer
+##Feature Description: Indexes alert messages in ElasticSearch or Solr
+
+bolt.alerts.indexing.name=AlertIndexBolt
+bolt.alerts.indexing.enabled=true
+bolt.alerts.indexing.num.tasks=1
+bolt.alerts.indexing.parallelism.hint=1
+
+#Feature: Error Indexer
+##Feature Description: Indexes error messages in ElasticSearch or Solr
+
+bolt.error.indexing.name=ErrorIndexBolt
+bolt.error.indexing.enabled=true
+bolt.error.indexing.num.tasks=1
+bolt.error.indexing.parallelism.hint=1
+
+#Feature: Kafka Bolt
+##Feature Description: Writes telemetry messages back into a Kafka topic
+
+bolt.kafka.name=KafkaBolt
+bolt.kafka.enabled=true
+bolt.kafka.num.tasks=1
+bolt.kafka.parallelism.hint=1
+
+#Feature: HDFS Bolt
+##Feature Description: Writes telemetry messages into HDFS
+
+bolt.hdfs.name=HDFSBolt
+bolt.hdfs.enabled=false
+bolt.hdfs.num.tasks=1
+bolt.hdfs.parallelism.hint=1
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/05e188ba/opensoc-streaming/OpenSOC-Topologies/src/main/resources/OpenSOC_Configs/topologies/ise/metrics.conf
----------------------------------------------------------------------
diff --git a/opensoc-streaming/OpenSOC-Topologies/src/main/resources/OpenSOC_Configs/topologies/ise/metrics.conf b/opensoc-streaming/OpenSOC-Topologies/src/main/resources/OpenSOC_Configs/topologies/ise/metrics.conf
new file mode 100644
index 0000000..1daef3d
--- /dev/null
+++ b/opensoc-streaming/OpenSOC-Topologies/src/main/resources/OpenSOC_Configs/topologies/ise/metrics.conf
@@ -0,0 +1,26 @@
+#reporters
+com.opensoc.metrics.reporter.graphite=true
+com.opensoc.metrics.reporter.console=false
+com.opensoc.metrics.reporter.jmx=false
+
+#Graphite Addresses
+
+com.opensoc.metrics.graphite.address=localhost
+com.opensoc.metrics.graphite.port=2023
+
+#TelemetryParserBolt
+com.opensoc.metrics.TelemetryParserBolt.acks=true
+com.opensoc.metrics.TelemetryParserBolt.emits=true
+com.opensoc.metrics.TelemetryParserBolt.fails=true
+
+
+#GenericEnrichmentBolt
+com.opensoc.metrics.GenericEnrichmentBolt.acks=true
+com.opensoc.metrics.GenericEnrichmentBolt.emits=true
+com.opensoc.metrics.GenericEnrichmentBolt.fails=true
+
+
+#TelemetryIndexingBolt
+com.opensoc.metrics.TelemetryIndexingBolt.acks=true
+com.opensoc.metrics.TelemetryIndexingBolt.emits=true
+com.opensoc.metrics.TelemetryIndexingBolt.fails=true

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/05e188ba/opensoc-streaming/OpenSOC-Topologies/src/main/resources/OpenSOC_Configs/topologies/ise/topology.conf
----------------------------------------------------------------------
diff --git a/opensoc-streaming/OpenSOC-Topologies/src/main/resources/OpenSOC_Configs/topologies/ise/topology.conf b/opensoc-streaming/OpenSOC-Topologies/src/main/resources/OpenSOC_Configs/topologies/ise/topology.conf
new file mode 100644
index 0000000..7fbc9ff
--- /dev/null
+++ b/opensoc-streaming/OpenSOC-Topologies/src/main/resources/OpenSOC_Configs/topologies/ise/topology.conf
@@ -0,0 +1,86 @@
+include = ../../etc/env/environment_common.conf
+include = ../../etc/env/es_connection.conf
+include = ../../etc/env/hdfs_connection.conf
+include = ../../etc/env/mysql_connection.conf
+include = metrics.conf
+include = features_enabled.conf
+
+#Global Properties
+
+debug.mode=true
+local.mode=true
+num.workers=1
+
+#Standard 5-tuple fields
+
+source.ip=ip_src_addr
+source.port=ip_src_port
+dest.ip=ip_dst_addr
+dest.port=ip_dst_port
+protocol=protocol
+
+#Test Spout
+spout.test.parallelism.repeat=false
+
+#Kafka Spout
+spout.kafka.topic=ise_raw
+
+
+
+#Host Enrichment
+
+bolt.enrichment.host.MAX_CACHE_SIZE=10000
+bolt.enrichment.host.MAX_TIME_RETAIN=10
+bolt.enrichment.host.enrichment_tag=host
+
+
+#GeoEnrichment
+
+bolt.enrichment.geo.enrichment_tag=geo
+bolt.enrichment.geo.adapter.table=GEO
+bolt.enrichment.geo.MAX_CACHE_SIZE=10000
+bolt.enrichment.geo.MAX_TIME_RETAIN=10
+
+#WhoisEnrichment
+
+bolt.enrichment.whois.hbase.table.name=whois
+bolt.enrichment.whois.enrichment_tag=whois
+bolt.enrichment.whois.source=tld
+bolt.enrichment.whois.MAX_CACHE_SIZE=10000
+bolt.enrichment.whois.MAX_TIME_RETAIN=10
+
+#CIF Enrichment
+bolt.enrichment.cif.tablename=cif_table
+bolt.enrichment.cif.host=tld
+bolt.enrichment.cif.email=email
+bolt.enrichment.cif.MAX_CACHE_SIZE=10000
+bolt.enrichment.cif.MAX_TIME_RETAIN=10
+bolt.enrichment.cif.enrichment_tag=cif
+
+
+#Indexing Bolt
+bolt.indexing.indexname=ise_index
+bolt.indexing.documentname=ise_doc
+bolt.indexing.bulk=200
+
+#Alerts Indexing Bolt
+bolt.alerts.indexing.indexname=alert
+bolt.alerts.indexing.documentname=ise_alert
+bolt.alerts.indexing.bulk=1
+
+#Error Indexing Bolt
+bolt.error.indexing.indexname=error
+bolt.error.indexing.documentname=ise_error
+bolt.error.indexing.bulk=1
+
+#HDFS Bolt
+bolt.hdfs.batch.size=5000
+bolt.hdfs.field.delimiter=|
+bolt.hdfs.file.rotation.size.in.mb=5
+bolt.hdfs.file.system.url=hdfs://nn1:8020
+bolt.hdfs.wip.file.path=/ise/wip
+bolt.hdfs.finished.file.path=/ise/rotated
+bolt.hdfs.compression.codec.class=org.apache.hadoop.io.compress.SnappyCodec
+
+#Kafka Bolt
+bolt.kafka.topic=ise_enriched
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/05e188ba/opensoc-streaming/OpenSOC-Topologies/src/main/resources/OpenSOC_Configs/topologies/ise/topology_identifier.conf
----------------------------------------------------------------------
diff --git a/opensoc-streaming/OpenSOC-Topologies/src/main/resources/OpenSOC_Configs/topologies/ise/topology_identifier.conf b/opensoc-streaming/OpenSOC-Topologies/src/main/resources/OpenSOC_Configs/topologies/ise/topology_identifier.conf
new file mode 100644
index 0000000..c500e9f
--- /dev/null
+++ b/opensoc-streaming/OpenSOC-Topologies/src/main/resources/OpenSOC_Configs/topologies/ise/topology_identifier.conf
@@ -0,0 +1,4 @@
+#Each topology must have a unique identifier.  This setting is required
+
+topology.id=ise
+instance.id=I001
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/05e188ba/opensoc-streaming/OpenSOC-Topologies/src/main/resources/OpenSOC_Configs/topologies/lancope/alerts.xml
----------------------------------------------------------------------
diff --git a/opensoc-streaming/OpenSOC-Topologies/src/main/resources/OpenSOC_Configs/topologies/lancope/alerts.xml b/opensoc-streaming/OpenSOC-Topologies/src/main/resources/OpenSOC_Configs/topologies/lancope/alerts.xml
new file mode 100644
index 0000000..368f1c0
--- /dev/null
+++ b/opensoc-streaming/OpenSOC-Topologies/src/main/resources/OpenSOC_Configs/topologies/lancope/alerts.xml
@@ -0,0 +1,11 @@
+<?xml version="1.0" encoding="ISO-8859-1" ?>
+<rule-definitions>
+	<rule>
+		<pattern>.*message.*</pattern>
+		<alert>{"type":"alert","priority":5, "title":"Lancope Alert", "body":
+			"Alert triggered by Lancope"}
+		</alert>
+	</rule>
+</rule-definitions>
+
+

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/05e188ba/opensoc-streaming/OpenSOC-Topologies/src/main/resources/OpenSOC_Configs/topologies/lancope/features_enabled.conf
----------------------------------------------------------------------
diff --git a/opensoc-streaming/OpenSOC-Topologies/src/main/resources/OpenSOC_Configs/topologies/lancope/features_enabled.conf b/opensoc-streaming/OpenSOC-Topologies/src/main/resources/OpenSOC_Configs/topologies/lancope/features_enabled.conf
new file mode 100644
index 0000000..765dde3
--- /dev/null
+++ b/opensoc-streaming/OpenSOC-Topologies/src/main/resources/OpenSOC_Configs/topologies/lancope/features_enabled.conf
@@ -0,0 +1,105 @@
+#Enable and disable features for each topology
+
+#Feature: Test spout 
+##Feature Description: Reads telemetry from file and ingests it into topology.  Used for testing or bulk loading the topology
+
+spout.test.name=TestSpout
+spout.test.enabled=true
+spout.test.num.tasks=1
+spout.test.parallelism.hint=1
+
+#Feature: Kafka spout
+##Feature Description: Acts as a Kafka consumer.  Takes messages from a Kafka topic and ingests them into a topology
+
+spout.kafka.name=KafkaSpout
+spout.kafka.enabled=false
+spout.kafka.num.tasks=1
+spout.kafka.parallelism.hint=1
+
+#Feature: Parser Bolt
+##Feature Description: Parses telemetry from its native format into a native JSON
+
+parser.bolt.name=ParserBolt
+bolt.parser.enabled=true
+bolt.parser.num.tasks=1
+bolt.parser.parallelism.hint=1
+
+#Feature: Host Enrichment
+##Feature Description: Appends information about known hosts to a telemetry message
+
+bolt.enrichment.host.name=HostEnrichment
+bolt.enrichment.host.enabled=false
+bolt.enrichment.host.num.tasks=1
+bolt.enrichment.host.parallelism.hint=1
+
+#Feature: Geo Enrichment
+##Feature Description: Appends geo information about known non-local IPs to a telemetry message
+
+bolt.enrichment.geo.name=GeoEnrichment 
+bolt.enrichment.geo.enabled=true
+bolt.enrichment.geo.num.tasks=1
+bolt.enrichment.geo.parallelism.hint=1
+
+#Feature: Whois Enrichment
+##Feature Description: Appends whois information about known domains to a telemetry message
+
+bolt.enrichment.whois.name=WhoisEnrichment
+bolt.enrichment.whois.enabled=false
+bolt.enrichment.whois.num.tasks=1
+bolt.enrichment.whois.parallelism.hint=1
+
+#Feature: CIF Enrichment
+##Feature Description: Appends information from CIF threat intelligence feeds to a telemetry message
+
+bolt.enrichment.cif.name=SIFBolt
+bolt.enrichment.cif.enabled=false
+bolt.enrichment.cif.num.tasks=1
+bolt.enrichment.cif.parallelism.hint=1
+
+#Feature: Rules-Based Alerts
+##Feature Description: Tags messages with rules-based alerts
+
+bolt.alerts.name=Alerts
+bolt.alerts.enabled=true
+bolt.alerts.num.tasks=1
+bolt.alerts.parallelism.hint=1
+
+#Feature: Indexer
+##Feature Description: Indexes telemetry messages in ElasticSearch or Solr
+
+bolt.indexing.name=IndexBolt
+bolt.indexing.enabled=true
+bolt.indexing.num.tasks=1
+bolt.indexing.parallelism.hint=1
+
+#Feature: Alerts Indexer
+##Feature Description: Indexes alert messages in ElasticSearch or Solr
+
+bolt.alerts.indexing.name=AlertIndexBolt
+bolt.alerts.indexing.enabled=true
+bolt.alerts.indexing.num.tasks=1
+bolt.alerts.indexing.parallelism.hint=1
+
+#Feature: Error Indexer
+##Feature Description: Indexes error messages in ElasticSearch or Solr
+
+bolt.error.indexing.name=ErrorIndexBolt
+bolt.error.indexing.enabled=true
+bolt.error.indexing.num.tasks=1
+bolt.error.indexing.parallelism.hint=1
+
+#Feature: Kafka Bolt
+##Feature Description: Writes telemetry messages back into a Kafka topic
+
+bolt.kafka.name=KafkaBolt
+bolt.kafka.enabled=true
+bolt.kafka.num.tasks=1
+bolt.kafka.parallelism.hint=1
+
+#Feature: HDFS Bolt
+##Feature Description: Writes telemetry messages into HDFS
+
+bolt.hdfs.name=HDFSBolt
+bolt.hdfs.enabled=false
+bolt.hdfs.num.tasks=1
+bolt.hdfs.parallelism.hint=1
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/05e188ba/opensoc-streaming/OpenSOC-Topologies/src/main/resources/OpenSOC_Configs/topologies/lancope/metrics.conf
----------------------------------------------------------------------
diff --git a/opensoc-streaming/OpenSOC-Topologies/src/main/resources/OpenSOC_Configs/topologies/lancope/metrics.conf b/opensoc-streaming/OpenSOC-Topologies/src/main/resources/OpenSOC_Configs/topologies/lancope/metrics.conf
new file mode 100644
index 0000000..1daef3d
--- /dev/null
+++ b/opensoc-streaming/OpenSOC-Topologies/src/main/resources/OpenSOC_Configs/topologies/lancope/metrics.conf
@@ -0,0 +1,26 @@
+#reporters
+com.opensoc.metrics.reporter.graphite=true
+com.opensoc.metrics.reporter.console=false
+com.opensoc.metrics.reporter.jmx=false
+
+#Graphite Addresses
+
+com.opensoc.metrics.graphite.address=localhost
+com.opensoc.metrics.graphite.port=2023
+
+#TelemetryParserBolt
+com.opensoc.metrics.TelemetryParserBolt.acks=true
+com.opensoc.metrics.TelemetryParserBolt.emits=true
+com.opensoc.metrics.TelemetryParserBolt.fails=true
+
+
+#GenericEnrichmentBolt
+com.opensoc.metrics.GenericEnrichmentBolt.acks=true
+com.opensoc.metrics.GenericEnrichmentBolt.emits=true
+com.opensoc.metrics.GenericEnrichmentBolt.fails=true
+
+
+#TelemetryIndexingBolt
+com.opensoc.metrics.TelemetryIndexingBolt.acks=true
+com.opensoc.metrics.TelemetryIndexingBolt.emits=true
+com.opensoc.metrics.TelemetryIndexingBolt.fails=true

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/05e188ba/opensoc-streaming/OpenSOC-Topologies/src/main/resources/OpenSOC_Configs/topologies/lancope/topology.conf
----------------------------------------------------------------------
diff --git a/opensoc-streaming/OpenSOC-Topologies/src/main/resources/OpenSOC_Configs/topologies/lancope/topology.conf b/opensoc-streaming/OpenSOC-Topologies/src/main/resources/OpenSOC_Configs/topologies/lancope/topology.conf
new file mode 100644
index 0000000..8c50580
--- /dev/null
+++ b/opensoc-streaming/OpenSOC-Topologies/src/main/resources/OpenSOC_Configs/topologies/lancope/topology.conf
@@ -0,0 +1,86 @@
+include = ../../etc/env/environment_common.conf
+include = ../../etc/env/es_connection.conf
+include = ../../etc/env/hdfs_connection.conf
+include = ../../etc/env/mysql_connection.conf
+include = metrics.conf
+include = features_enabled.conf
+
+#Global Properties
+
+debug.mode=true
+local.mode=true
+num.workers=1
+
+#Standard 5-tuple fields
+
+source.ip=ip_src_addr
+source.port=ip_src_port
+dest.ip=ip_dst_addr
+dest.port=ip_dst_port
+protocol=protocol
+
+#Test Spout
+spout.test.parallelism.repeat=false
+
+#Kafka Spout
+spout.kafka.topic=lancope_raw
+
+
+
+#Host Enrichment
+
+bolt.enrichment.host.MAX_CACHE_SIZE=10000
+bolt.enrichment.host.MAX_TIME_RETAIN=10
+bolt.enrichment.host.enrichment_tag=host
+
+
+#GeoEnrichment
+
+bolt.enrichment.geo.enrichment_tag=geo
+bolt.enrichment.geo.adapter.table=GEO
+bolt.enrichment.geo.MAX_CACHE_SIZE=10000
+bolt.enrichment.geo.MAX_TIME_RETAIN=10
+
+#WhoisEnrichment
+
+bolt.enrichment.whois.hbase.table.name=whois
+bolt.enrichment.whois.enrichment_tag=whois
+bolt.enrichment.whois.source=tld
+bolt.enrichment.whois.MAX_CACHE_SIZE=10000
+bolt.enrichment.whois.MAX_TIME_RETAIN=10
+
+#CIF Enrichment
+bolt.enrichment.cif.tablename=cif_table
+bolt.enrichment.cif.host=tld
+bolt.enrichment.cif.email=email
+bolt.enrichment.cif.MAX_CACHE_SIZE=10000
+bolt.enrichment.cif.MAX_TIME_RETAIN=10
+bolt.enrichment.cif.enrichment_tag=cif
+
+
+#Indexing Bolt
+bolt.indexing.indexname=lancope_index
+bolt.indexing.documentname=lancope_doc
+bolt.indexing.bulk=200
+
+#Alerts Indexing Bolt
+bolt.alerts.indexing.indexname=alert
+bolt.alerts.indexing.documentname=lancope_alert
+bolt.alerts.indexing.bulk=1
+
+#Error Indexing Bolt
+bolt.error.indexing.indexname=error
+bolt.error.indexing.documentname=lancope_error
+bolt.error.indexing.bulk=1
+
+#HDFS Bolt
+bolt.hdfs.batch.size=5000
+bolt.hdfs.field.delimiter=|
+bolt.hdfs.file.rotation.size.in.mb=5
+bolt.hdfs.file.system.url=hdfs://nn1:8020
+bolt.hdfs.wip.file.path=/lancope/wip
+bolt.hdfs.finished.file.path=/lancope/rotated
+bolt.hdfs.compression.codec.class=org.apache.hadoop.io.compress.SnappyCodec
+
+#Kafka Bolt
+bolt.kafka.topic=lancope_enriched
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/05e188ba/opensoc-streaming/OpenSOC-Topologies/src/main/resources/OpenSOC_Configs/topologies/lancope/topology_identifier.conf
----------------------------------------------------------------------
diff --git a/opensoc-streaming/OpenSOC-Topologies/src/main/resources/OpenSOC_Configs/topologies/lancope/topology_identifier.conf b/opensoc-streaming/OpenSOC-Topologies/src/main/resources/OpenSOC_Configs/topologies/lancope/topology_identifier.conf
new file mode 100644
index 0000000..a68084e
--- /dev/null
+++ b/opensoc-streaming/OpenSOC-Topologies/src/main/resources/OpenSOC_Configs/topologies/lancope/topology_identifier.conf
@@ -0,0 +1,4 @@
+#Each topology must have a unique identifier.  This setting is required
+
+topology.id=lancope
+instance.id=L001
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/05e188ba/opensoc-streaming/OpenSOC-Topologies/src/main/resources/OpenSOC_Configs/topologies/pcap/features_enabled.conf
----------------------------------------------------------------------
diff --git a/opensoc-streaming/OpenSOC-Topologies/src/main/resources/OpenSOC_Configs/topologies/pcap/features_enabled.conf b/opensoc-streaming/OpenSOC-Topologies/src/main/resources/OpenSOC_Configs/topologies/pcap/features_enabled.conf
new file mode 100644
index 0000000..a79d7ee
--- /dev/null
+++ b/opensoc-streaming/OpenSOC-Topologies/src/main/resources/OpenSOC_Configs/topologies/pcap/features_enabled.conf
@@ -0,0 +1,110 @@
+#Enable and disable features for each topology
+
+#Feature: Test spout 
+##Feature Description: Reads telemetry from file and ingests it into topology.  Used for testing or bulk loading the topology
+
+spout.test.name=TestSpout
+spout.test.enabled=true
+spout.test.num.tasks=1
+spout.test.parallelism.hint=1
+
+#Feature: Kafka spout
+##Feature Description: Acts as a Kafka consumer.  Takes messages from a Kafka topic and ingests them into a topology
+
+spout.kafka.name=KafkaSpout
+spout.kafka.enabled=false
+spout.kafka.num.tasks=1
+spout.kafka.parallelism.hint=1
+
+#Feature: Parser Bolt
+##Feature Description: Parses telemetry from its native format into a native JSON
+
+parser.bolt.name=ParserBolt
+bolt.parser.enabled=true
+bolt.parser.num.tasks=1
+bolt.parser.parallelism.hint=1
+
+#Feature: Host Enrichment
+##Feature Description: Appends information about known hosts to a telemetry message
+
+bolt.enrichment.host.name=HostEnrichment
+bolt.enrichment.host.enabled=false
+bolt.enrichment.host.num.tasks=1
+bolt.enrichment.host.parallelism.hint=1
+
+#Feature: Geo Enrichment
+##Feature Description: Appends geo information about known non-local IPs to a telemetry message
+
+bolt.enrichment.geo.name=GeoEnrichment 
+bolt.enrichment.geo.enabled=false
+bolt.enrichment.geo.num.tasks=1
+bolt.enrichment.geo.parallelism.hint=1
+
+#Feature: Whois Enrichment
+##Feature Description: Appends whois information about known domains to a telemetry message
+
+bolt.enrichment.whois.name=WhoisEnrichment
+bolt.enrichment.whois.enabled=false
+bolt.enrichment.whois.num.tasks=1
+bolt.enrichment.whois.parallelism.hint=1
+
+#Feature: CIF Enrichment
+##Feature Description: Appends information from CIF threat intelligence feeds to a telemetry message
+
+bolt.enrichment.cif.name=SIFBolt
+bolt.enrichment.cif.enabled=false
+bolt.enrichment.cif.num.tasks=1
+bolt.enrichment.cif.parallelism.hint=1
+
+#Feature: Rules-Based Alerts
+##Feature Description: Tags messages with rules-based alerts
+
+bolt.alerts.name=Alerts
+bolt.alerts.enabled=false
+bolt.alerts.num.tasks=1
+bolt.alerts.parallelism.hint=1
+
+#Feature: Indexer
+##Feature Description: Indexes telemetry messages in ElasticSearch or Solr
+
+bolt.indexing.name=IndexBolt
+bolt.indexing.enabled=true
+bolt.indexing.num.tasks=1
+bolt.indexing.parallelism.hint=1
+
+#Feature: Alerts Indexer
+##Feature Description: Indexes alert messages in ElasticSearch or Solr
+
+bolt.alerts.indexing.name=AlertIndexBolt
+bolt.alerts.indexing.enabled=false
+bolt.alerts.indexing.num.tasks=1
+bolt.alerts.indexing.parallelism.hint=1
+
+#Feature: Error Indexer
+##Feature Description: Indexes error messages in ElasticSearch or Solr
+
+bolt.error.indexing.name=ErrorIndexBolt
+bolt.error.indexing.enabled=true
+bolt.error.indexing.num.tasks=1
+bolt.error.indexing.parallelism.hint=1
+
+#Feature: Kafka Bolt
+##Feature Description: Writes telemetry messages back into a Kafka topic
+
+bolt.kafka.name=KafkaBolt
+bolt.kafka.enabled=false
+bolt.kafka.num.tasks=1
+bolt.kafka.parallelism.hint=1
+
+#Feature: HDFS Bolt
+##Feature Description: Writes telemetry messages into HDFS
+
+bolt.hdfs.name=HDFSBolt
+bolt.hdfs.enabled=true
+bolt.hdfs.num.tasks=1
+bolt.hdfs.parallelism.hint=1
+
+bolt.hbase.name=HBaseBolt
+bolt.hbase.enabled=true
+bolt.hbase.num.tasks=1
+bolt.hbase.parallelism.hint=1
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/05e188ba/opensoc-streaming/OpenSOC-Topologies/src/main/resources/OpenSOC_Configs/topologies/pcap/metrics.conf
----------------------------------------------------------------------
diff --git a/opensoc-streaming/OpenSOC-Topologies/src/main/resources/OpenSOC_Configs/topologies/pcap/metrics.conf b/opensoc-streaming/OpenSOC-Topologies/src/main/resources/OpenSOC_Configs/topologies/pcap/metrics.conf
new file mode 100644
index 0000000..1daef3d
--- /dev/null
+++ b/opensoc-streaming/OpenSOC-Topologies/src/main/resources/OpenSOC_Configs/topologies/pcap/metrics.conf
@@ -0,0 +1,26 @@
+#reporters
+com.opensoc.metrics.reporter.graphite=true
+com.opensoc.metrics.reporter.console=false
+com.opensoc.metrics.reporter.jmx=false
+
+#Graphite Addresses
+
+com.opensoc.metrics.graphite.address=localhost
+com.opensoc.metrics.graphite.port=2023
+
+#TelemetryParserBolt
+com.opensoc.metrics.TelemetryParserBolt.acks=true
+com.opensoc.metrics.TelemetryParserBolt.emits=true
+com.opensoc.metrics.TelemetryParserBolt.fails=true
+
+
+#GenericEnrichmentBolt
+com.opensoc.metrics.GenericEnrichmentBolt.acks=true
+com.opensoc.metrics.GenericEnrichmentBolt.emits=true
+com.opensoc.metrics.GenericEnrichmentBolt.fails=true
+
+
+#TelemetryIndexingBolt
+com.opensoc.metrics.TelemetryIndexingBolt.acks=true
+com.opensoc.metrics.TelemetryIndexingBolt.emits=true
+com.opensoc.metrics.TelemetryIndexingBolt.fails=true

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/05e188ba/opensoc-streaming/OpenSOC-Topologies/src/main/resources/OpenSOC_Configs/topologies/pcap/topology.conf
----------------------------------------------------------------------
diff --git a/opensoc-streaming/OpenSOC-Topologies/src/main/resources/OpenSOC_Configs/topologies/pcap/topology.conf b/opensoc-streaming/OpenSOC-Topologies/src/main/resources/OpenSOC_Configs/topologies/pcap/topology.conf
new file mode 100644
index 0000000..bd5bc59
--- /dev/null
+++ b/opensoc-streaming/OpenSOC-Topologies/src/main/resources/OpenSOC_Configs/topologies/pcap/topology.conf
@@ -0,0 +1,112 @@
+include = ../../etc/env/environment_common.conf
+include = ../../etc/env/es_connection.conf
+include = ../../etc/env/hdfs_connection.conf
+include = ../../etc/env/mysql_connection.conf
+include = metrics.conf
+include = features_enabled.conf
+
+#Global Properties
+
+debug.mode=true
+local.mode=true
+num.workers=1
+
+#Standard 5-tuple fields
+
+source.ip=ip_src_addr
+source.port=ip_src_port
+dest.ip=ip_dst_addr
+dest.port=ip_dst_port
+protocol=protocol
+
+#Kafka Spout
+spout.kafka.buffer.size.bytes=1024000
+spout.kafka.consumer.id=pcap.kafka
+spout.kafka.fetch.size.bytes=1024
+spout.kafka.forcefromstart=false
+spout.kafka.socket.timeout.ms=600000
+spout.kafka.start.offset.time=-1
+spout.kafka.zk.root=/storm/topology/pcap/kafka
+spout.kafka.topic=pcap
+
+#Parser Bolt
+bolt.parser.enabled=true
+bolt.parser.num.of.key.chars.to.use.for.shuffle.grouping=6
+bolt.parser.ts.precision=MICRO
+
+#Test Spout
+spout.test.parallelism.repeat=false
+
+#Kafka Spout
+spout.kafka.topic=pcap_raw
+
+
+
+#Host Enrichment
+
+bolt.enrichment.host.MAX_CACHE_SIZE=10000
+bolt.enrichment.host.MAX_TIME_RETAIN=10
+bolt.enrichment.host.enrichment_tag=host
+
+
+#GeoEnrichment
+
+bolt.enrichment.geo.enrichment_tag=geo
+bolt.enrichment.geo.adapter.table=GEO
+bolt.enrichment.geo.MAX_CACHE_SIZE=10000
+bolt.enrichment.geo.MAX_TIME_RETAIN=10
+
+#WhoisEnrichment
+
+bolt.enrichment.whois.hbase.table.name=whois
+bolt.enrichment.whois.enrichment_tag=whois
+bolt.enrichment.whois.source=tld
+bolt.enrichment.whois.MAX_CACHE_SIZE=10000
+bolt.enrichment.whois.MAX_TIME_RETAIN=10
+
+#CIF Enrichment
+bolt.enrichment.cif.tablename=cif_table
+bolt.enrichment.cif.host=tld
+bolt.enrichment.cif.email=email
+bolt.enrichment.cif.MAX_CACHE_SIZE=10000
+bolt.enrichment.cif.MAX_TIME_RETAIN=10
+bolt.enrichment.cif.enrichment_tag=cif
+
+
+#Indexing Bolt
+bolt.indexing.indexname=pcap_index_test
+bolt.indexing.documentname=pcap_doc
+bolt.indexing.bulk=1
+
+#Alerts Indexing Bolt
+bolt.alerts.indexing.indexname=pcap_alert_test
+bolt.alerts.indexing.documentname=pcap_alert
+bolt.alerts.indexing.bulk=1
+
+#Error Indexing Bolt
+bolt.error.indexing.indexname=pcap_error_test
+bolt.error.indexing.documentname=pcap_error
+bolt.error.indexing.bulk=1
+
+#HDFS Bolt
+bolt.hdfs.batch.size=5000
+bolt.hdfs.field.delimiter=|
+bolt.hdfs.file.rotation.size.in.mb=5
+bolt.hdfs.file.system.url=hdfs://nn1:8020
+bolt.hdfs.wip.file.path=/pcap/wip
+bolt.hdfs.finished.file.path=/pcap/rotated
+bolt.hdfs.compression.codec.class=org.apache.hadoop.io.compress.SnappyCodec
+
+#Kafka Bolt
+bolt.kafka.topic=pcap_enriched
+
+#HBase Bolt
+bolt.hbase.table.name=pcap_test
+## Define the hbase table columns in the form <cf1>:<cq11>,<cq12>,<cq13>|<cf2>:<cq21>,<cq22>|.......
+bolt.hbase.table.fields=t:pcap
+bolt.hbase.table.key.tuple.field.name=pcap_id
+bolt.hbase.table.timestamp.tuple.field.name=timestamp
+bolt.hbase.enable.batching=false
+bolt.hbase.write.buffer.size.in.bytes=2000000
+bolt.hbase.durability=SKIP_WAL
+bolt.hbase.partitioner.region.info.refresh.interval.mins=60

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/05e188ba/opensoc-streaming/OpenSOC-Topologies/src/main/resources/OpenSOC_Configs/topologies/pcap/topology_identifier.conf
----------------------------------------------------------------------
diff --git a/opensoc-streaming/OpenSOC-Topologies/src/main/resources/OpenSOC_Configs/topologies/pcap/topology_identifier.conf b/opensoc-streaming/OpenSOC-Topologies/src/main/resources/OpenSOC_Configs/topologies/pcap/topology_identifier.conf
new file mode 100644
index 0000000..aad3257
--- /dev/null
+++ b/opensoc-streaming/OpenSOC-Topologies/src/main/resources/OpenSOC_Configs/topologies/pcap/topology_identifier.conf
@@ -0,0 +1,4 @@
+#Each topology must have a unique identifier.  This setting is required
+
+topology.id=pcap
+instance.id=P001
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/05e188ba/opensoc-streaming/OpenSOC-Topologies/src/main/resources/OpenSOC_Configs/topologies/sourcefire/alerts.xml
----------------------------------------------------------------------
diff --git a/opensoc-streaming/OpenSOC-Topologies/src/main/resources/OpenSOC_Configs/topologies/sourcefire/alerts.xml b/opensoc-streaming/OpenSOC-Topologies/src/main/resources/OpenSOC_Configs/topologies/sourcefire/alerts.xml
new file mode 100644
index 0000000..9286b10
--- /dev/null
+++ b/opensoc-streaming/OpenSOC-Topologies/src/main/resources/OpenSOC_Configs/topologies/sourcefire/alerts.xml
@@ -0,0 +1,11 @@
+<?xml version="1.0" encoding="ISO-8859-1" ?>
+<rule-definitions>
+	<rule>
+		<pattern>.*message.*</pattern>
+		<alert>{"type":"alert","priority":5, "title":"Sourcefire Alert", "body":
+			"Alert triggered by sourcefire"}
+		</alert>
+	</rule>
+</rule-definitions>
+
+

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/05e188ba/opensoc-streaming/OpenSOC-Topologies/src/main/resources/OpenSOC_Configs/topologies/sourcefire/features_enabled.conf
----------------------------------------------------------------------
diff --git a/opensoc-streaming/OpenSOC-Topologies/src/main/resources/OpenSOC_Configs/topologies/sourcefire/features_enabled.conf b/opensoc-streaming/OpenSOC-Topologies/src/main/resources/OpenSOC_Configs/topologies/sourcefire/features_enabled.conf
new file mode 100644
index 0000000..f6c9ab8
--- /dev/null
+++ b/opensoc-streaming/OpenSOC-Topologies/src/main/resources/OpenSOC_Configs/topologies/sourcefire/features_enabled.conf
@@ -0,0 +1,105 @@
+#Enable and disable features for each topology
+
+#Feature: Test spout 
+##Feature Description: Reads telemetry from file and ingests it into topology.  Used for testing or bulk loading the topology
+
+spout.test.name=TestSpout
+spout.test.enabled=true
+spout.test.num.tasks=1
+spout.test.parallelism.hint=1
+
+#Feature: Kafka spout
+##Feature Description: Acts as a Kafka consumer.  Takes messages from a Kafka topic and ingests them into a topology
+
+spout.kafka.name=KafkaSpout
+spout.kafka.enabled=false
+spout.kafka.num.tasks=1
+spout.kafka.parallelism.hint=1
+
+#Feature: Parser Bolt
+##Feature Description: Parses telemetry from its native format into a native JSON
+
+parser.bolt.name=ParserBolt
+bolt.parser.enabled=true
+bolt.parser.num.tasks=1
+bolt.parser.parallelism.hint=1
+
+#Feature: Host Enrichment
+##Feature Description: Appends information about known hosts to a telemetry message
+
+bolt.enrichment.host.name=HostEnrichment
+bolt.enrichment.host.enabled=true
+bolt.enrichment.host.num.tasks=1
+bolt.enrichment.host.parallelism.hint=1
+
+#Feature: Geo Enrichment
+##Feature Description: Appends geo information about known non-local IPs to a telemetry message
+
+bolt.enrichment.geo.name=GeoEnrichment 
+bolt.enrichment.geo.enabled=true
+bolt.enrichment.geo.num.tasks=1
+bolt.enrichment.geo.parallelism.hint=1
+
+#Feature: Whois Enrichment
+##Feature Description: Appends whois information about known domains to a telemetry message
+
+bolt.enrichment.whois.name=WhoisEnrichment
+bolt.enrichment.whois.enabled=false
+bolt.enrichment.whois.num.tasks=1
+bolt.enrichment.whois.parallelism.hint=1
+
+#Feature: CIF Enrichment
+##Feature Description: Appends information from CIF threat intelligence feeds to a telemetry message
+
+bolt.enrichment.cif.name=SIFBolt
+bolt.enrichment.cif.enabled=false
+bolt.enrichment.cif.num.tasks=1
+bolt.enrichment.cif.parallelism.hint=1
+
+#Feature: Rules-Based Alerts
+##Feature Description: Tags messages with rules-based alerts
+
+bolt.alerts.name=Alerts
+bolt.alerts.enabled=true
+bolt.alerts.num.tasks=1
+bolt.alerts.parallelism.hint=1
+
+#Feature: Indexer
+##Feature Description: Indexes telemetry messages in ElasticSearch or Solr
+
+bolt.indexing.name=IndexBolt
+bolt.indexing.enabled=true
+bolt.indexing.num.tasks=1
+bolt.indexing.parallelism.hint=1
+
+#Feature: Alerts Indexer
+##Feature Description: Indexes alert messages in ElasticSearch or Solr
+
+bolt.alerts.indexing.name=AlertIndexBolt
+bolt.alerts.indexing.enabled=true
+bolt.alerts.indexing.num.tasks=1
+bolt.alerts.indexing.parallelism.hint=1
+
+#Feature: Error Indexer
+##Feature Description: Indexes error messages in ElasticSearch or Solr
+
+bolt.error.indexing.name=ErrorIndexBolt
+bolt.error.indexing.enabled=true
+bolt.error.indexing.num.tasks=1
+bolt.error.indexing.parallelism.hint=1
+
+#Feature: Kafka Bolt
+##Feature Description: Writes telemetry messages back into a Kafka topic
+
+bolt.kafka.name=KafkaBolt
+bolt.kafka.enabled=false
+bolt.kafka.num.tasks=1
+bolt.kafka.parallelism.hint=1
+
+#Feature: HDFS Bolt
+##Feature Description: Writes telemetry messages into HDFS
+
+bolt.hdfs.name=HDFSBolt
+bolt.hdfs.enabled=false
+bolt.hdfs.num.tasks=1
+bolt.hdfs.parallelism.hint=1
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/05e188ba/opensoc-streaming/OpenSOC-Topologies/src/main/resources/OpenSOC_Configs/topologies/sourcefire/metrics.conf
----------------------------------------------------------------------
diff --git a/opensoc-streaming/OpenSOC-Topologies/src/main/resources/OpenSOC_Configs/topologies/sourcefire/metrics.conf b/opensoc-streaming/OpenSOC-Topologies/src/main/resources/OpenSOC_Configs/topologies/sourcefire/metrics.conf
new file mode 100644
index 0000000..1daef3d
--- /dev/null
+++ b/opensoc-streaming/OpenSOC-Topologies/src/main/resources/OpenSOC_Configs/topologies/sourcefire/metrics.conf
@@ -0,0 +1,26 @@
+#reporters
+com.opensoc.metrics.reporter.graphite=true
+com.opensoc.metrics.reporter.console=false
+com.opensoc.metrics.reporter.jmx=false
+
+#Graphite Addresses
+
+com.opensoc.metrics.graphite.address=localhost
+com.opensoc.metrics.graphite.port=2023
+
+#TelemetryParserBolt
+com.opensoc.metrics.TelemetryParserBolt.acks=true
+com.opensoc.metrics.TelemetryParserBolt.emits=true
+com.opensoc.metrics.TelemetryParserBolt.fails=true
+
+
+#GenericEnrichmentBolt
+com.opensoc.metrics.GenericEnrichmentBolt.acks=true
+com.opensoc.metrics.GenericEnrichmentBolt.emits=true
+com.opensoc.metrics.GenericEnrichmentBolt.fails=true
+
+
+#TelemetryIndexingBolt
+com.opensoc.metrics.TelemetryIndexingBolt.acks=true
+com.opensoc.metrics.TelemetryIndexingBolt.emits=true
+com.opensoc.metrics.TelemetryIndexingBolt.fails=true