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

[44/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-MessageParsers/src/main/java/com/opensoc/parsing/parsers/AbstractParser.java
----------------------------------------------------------------------
diff --git a/opensoc-streaming/OpenSOC-MessageParsers/src/main/java/com/opensoc/parsing/parsers/AbstractParser.java b/opensoc-streaming/OpenSOC-MessageParsers/src/main/java/com/opensoc/parsing/parsers/AbstractParser.java
new file mode 100644
index 0000000..e5fa29e
--- /dev/null
+++ b/opensoc-streaming/OpenSOC-MessageParsers/src/main/java/com/opensoc/parsing/parsers/AbstractParser.java
@@ -0,0 +1,42 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package com.opensoc.parsing.parsers;
+
+import java.io.Serializable;
+
+import org.json.simple.JSONObject;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.opensoc.parser.interfaces.MessageParser;
+import com.opensoc.parsing.AbstractParserBolt;
+
+@SuppressWarnings("serial")
+public abstract class AbstractParser implements MessageParser, Serializable {
+
+	protected static final Logger _LOG = LoggerFactory
+			.getLogger(AbstractParserBolt.class);
+
+	public void initializeParser() {
+		_LOG.debug("Initializing adapter...");
+
+	}
+
+	abstract public JSONObject parse(byte[] raw_message);
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/05e188ba/opensoc-streaming/OpenSOC-MessageParsers/src/main/java/com/opensoc/parsing/parsers/BasicBroParser.java
----------------------------------------------------------------------
diff --git a/opensoc-streaming/OpenSOC-MessageParsers/src/main/java/com/opensoc/parsing/parsers/BasicBroParser.java b/opensoc-streaming/OpenSOC-MessageParsers/src/main/java/com/opensoc/parsing/parsers/BasicBroParser.java
new file mode 100644
index 0000000..bdb4c52
--- /dev/null
+++ b/opensoc-streaming/OpenSOC-MessageParsers/src/main/java/com/opensoc/parsing/parsers/BasicBroParser.java
@@ -0,0 +1,114 @@
+/*
+ * 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.parsing.parsers;
+
+import org.json.simple.JSONObject;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings("serial")
+public class BasicBroParser extends AbstractParser {
+
+	protected static final Logger _LOG = LoggerFactory
+			.getLogger(BasicBroParser.class);
+	JSONCleaner cleaner = new JSONCleaner();
+
+	@SuppressWarnings("unchecked")
+	public JSONObject parse(byte[] msg) {
+
+		_LOG.trace("[OpenSOC] Starting to parse incoming message");
+		
+		String raw_message = null;
+
+		try {
+
+			raw_message = new String(msg, "UTF-8");
+			_LOG.trace("[OpenSOC] Received message: " + raw_message);
+
+			JSONObject cleaned_message = cleaner.Clean(raw_message);
+			_LOG.debug("[OpenSOC] Cleaned message: " + raw_message);
+			
+			if(cleaned_message == null || cleaned_message.isEmpty())
+				throw new Exception("Unable to clean message: " + raw_message);
+
+			String key = cleaned_message.keySet().iterator().next().toString();
+			
+			if(key == null)
+				throw new Exception("Unable to retrieve key for message: " + raw_message);
+
+			JSONObject payload = (JSONObject) cleaned_message.get(key);
+			
+			if(payload == null)
+				throw new Exception("Unable to retrieve payload for message: " + raw_message);
+
+			if (payload.containsKey("id.orig_h")) {
+				String source_ip = payload.remove("id.orig_h").toString();
+				payload.put("ip_src_addr", source_ip);
+				_LOG.trace("[OpenSOC] Added ip_src_addr to: " + payload);
+			}
+			if (payload.containsKey("id.resp_h")) {
+				String source_ip = payload.remove("id.resp_h").toString();
+				payload.put("ip_dst_addr", source_ip);
+				_LOG.trace("[OpenSOC] Added ip_dst_addr to: " + payload);
+			}
+			if (payload.containsKey("id.orig_p")) {
+				String source_port = payload.remove("id.orig_p")
+						.toString();
+				payload.put("ip_src_port", source_port);
+				_LOG.trace("[OpenSOC] Added ip_src_port to: " + payload);
+			}
+			if (payload.containsKey("id.resp_p")) {
+				String dest_port = payload.remove("id.resp_p").toString();
+				payload.put("ip_dst_port", dest_port);
+				_LOG.trace("[OpenSOC] Added ip_dst_port to: " + payload);
+			}
+			if (payload.containsKey("host")) {
+
+				String host = payload.get("host").toString().trim();
+				String[] parts = host.split("\\.");
+				int length = parts.length;
+				payload.put("tld", parts[length - 2] + "."
+						+ parts[length - 1]);
+				_LOG.trace("[OpenSOC] Added tld to: " + payload);
+			}
+			if (payload.containsKey("query")) {
+				String host = payload.get("query").toString();
+				String[] parts = host.split("\\.");
+				int length = parts.length;
+				payload.put("tld", parts[length - 2] + "."
+						+ parts[length - 1]);
+				_LOG.trace("[OpenSOC] Added tld to: " + payload);
+			}
+
+			_LOG.trace("[OpenSOC] Inner message: " + payload);
+
+			payload.put("protocol", key);
+			_LOG.debug("[OpenSOC] Returning parsed message: " + payload);
+			
+			return payload;
+			
+		} catch (Exception e) {
+
+			_LOG.error("Unable to Parse Message: " + raw_message);
+			e.printStackTrace();
+			return null;
+		}
+		
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/05e188ba/opensoc-streaming/OpenSOC-MessageParsers/src/main/java/com/opensoc/parsing/parsers/BasicIseParser.java
----------------------------------------------------------------------
diff --git a/opensoc-streaming/OpenSOC-MessageParsers/src/main/java/com/opensoc/parsing/parsers/BasicIseParser.java b/opensoc-streaming/OpenSOC-MessageParsers/src/main/java/com/opensoc/parsing/parsers/BasicIseParser.java
new file mode 100644
index 0000000..f7a4b71
--- /dev/null
+++ b/opensoc-streaming/OpenSOC-MessageParsers/src/main/java/com/opensoc/parsing/parsers/BasicIseParser.java
@@ -0,0 +1,82 @@
+/*
+ * 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.parsing.parsers;
+
+import java.io.StringReader;
+
+import org.json.simple.JSONObject;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.esotericsoftware.minlog.Log;
+import com.opensoc.ise.parser.ISEParser;
+
+@SuppressWarnings("serial")
+public class BasicIseParser extends AbstractParser {
+
+	protected static final Logger _LOG = LoggerFactory
+			.getLogger(BasicIseParser.class);
+	static final transient ISEParser _parser = new ISEParser("header=");
+
+	@SuppressWarnings("unchecked")
+	public JSONObject parse(byte[] msg) {
+	
+		String raw_message = "";
+
+		try {
+
+			raw_message = new String(msg, "UTF-8");
+			_LOG.debug("Received message: " + raw_message);
+			
+			/*
+			 * Reinitialize Parser. It has the effect of calling the constructor again. 
+			 */
+			_parser.ReInit(new StringReader("header=" + raw_message.trim()));
+
+			JSONObject payload = _parser.parseObject();
+
+			String ip_src_addr = (String) payload.get("Device IP Address");
+			String ip_src_port = (String) payload.get("Device Port");
+			String ip_dst_addr = (String) payload.get("DestinationIPAddress");
+			String ip_dst_port = (String) payload.get("DestinationPort");
+			
+			/*
+			 * Standard Fields for OpenSoc.
+			 */
+
+			if(ip_src_addr != null)
+				payload.put("ip_src_addr", ip_src_addr);
+			if(ip_src_port != null)
+				payload.put("ip_src_port", ip_src_port);
+			if(ip_dst_addr != null)
+				payload.put("ip_dst_addr", ip_dst_addr);
+			if(ip_dst_port != null)
+				payload.put("ip_dst_port", ip_dst_port);
+
+			JSONObject message = new JSONObject();
+			//message.put("message", payload);
+
+			return payload;
+
+		} catch (Exception e) {
+			Log.error(e.toString());
+			e.printStackTrace();
+		}
+		return null;
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/05e188ba/opensoc-streaming/OpenSOC-MessageParsers/src/main/java/com/opensoc/parsing/parsers/BasicLancopeParser.java
----------------------------------------------------------------------
diff --git a/opensoc-streaming/OpenSOC-MessageParsers/src/main/java/com/opensoc/parsing/parsers/BasicLancopeParser.java b/opensoc-streaming/OpenSOC-MessageParsers/src/main/java/com/opensoc/parsing/parsers/BasicLancopeParser.java
new file mode 100644
index 0000000..23f6c62
--- /dev/null
+++ b/opensoc-streaming/OpenSOC-MessageParsers/src/main/java/com/opensoc/parsing/parsers/BasicLancopeParser.java
@@ -0,0 +1,72 @@
+/*
+ * 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.parsing.parsers;
+
+import java.text.SimpleDateFormat;
+import java.util.Date;
+
+import org.json.simple.JSONObject;
+import org.json.simple.JSONValue;
+
+@SuppressWarnings("serial")
+public class BasicLancopeParser extends AbstractParser {
+	// Sample Lancope Message
+	// {"message":"<131>Jul 17 15:59:01 smc-01 StealthWatch[12365]: 2014-07-17T15:58:30Z 10.40.10.254 0.0.0.0 Minor High Concern Index The host's concern index has either exceeded the CI threshold or rapidly increased. Observed 36.55M points. Policy maximum allows up to 20M points.","@version":"1","@timestamp":"2014-07-17T15:56:05.992Z","type":"syslog","host":"10.122.196.201"}
+
+	@SuppressWarnings("unchecked")
+	@Override
+	public JSONObject parse(byte[] msg) {
+
+		JSONObject payload = null;
+
+		try {
+			
+			String raw_message = new String(msg, "UTF-8");
+			
+			payload = (JSONObject) JSONValue.parse(raw_message);
+			
+			
+
+			String message = payload.get("message").toString();
+			String[] parts = message.split(" ");
+			payload.put("ip_src_addr", parts[6]);
+			payload.put("ip_dst_addr", parts[7]);
+
+			String fixed_date = parts[5].replace('T', ' ');
+			fixed_date = fixed_date.replace('Z', ' ').trim();
+
+			SimpleDateFormat formatter = new SimpleDateFormat(
+					"yyyy-MM-dd HH:mm:ss");
+
+			Date date;
+
+			date = formatter.parse(fixed_date);
+			payload.put("timestamp", date.getTime());
+
+			payload.remove("@timestamp");
+			payload.remove("message");
+			payload.put("original_string", message);
+
+			return payload;
+		} catch (Exception e) {
+
+			_LOG.error("Unable to parse message: " + payload.toJSONString());
+			return null;
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/05e188ba/opensoc-streaming/OpenSOC-MessageParsers/src/main/java/com/opensoc/parsing/parsers/BasicSourcefireParser.java
----------------------------------------------------------------------
diff --git a/opensoc-streaming/OpenSOC-MessageParsers/src/main/java/com/opensoc/parsing/parsers/BasicSourcefireParser.java b/opensoc-streaming/OpenSOC-MessageParsers/src/main/java/com/opensoc/parsing/parsers/BasicSourcefireParser.java
new file mode 100644
index 0000000..b8eaf21
--- /dev/null
+++ b/opensoc-streaming/OpenSOC-MessageParsers/src/main/java/com/opensoc/parsing/parsers/BasicSourcefireParser.java
@@ -0,0 +1,91 @@
+/*
+ * 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.parsing.parsers;
+
+import java.util.regex.Pattern;
+
+import org.json.simple.JSONObject;
+
+@SuppressWarnings("serial")
+public class BasicSourcefireParser extends AbstractParser {
+
+	public static final String hostkey = "host";
+	String domain_name_regex = "([^\\.]+)\\.([a-z]{2}|[a-z]{3}|([a-z]{2}\\.[a-z]{2}))$";
+	Pattern pattern = Pattern.compile(domain_name_regex);
+
+	@SuppressWarnings({ "unchecked", "unused" })
+	public JSONObject parse(byte[] msg) {
+
+		JSONObject payload = new JSONObject();
+		String toParse = "";
+
+		try {
+
+			toParse = new String(msg, "UTF-8");
+			_LOG.debug("Received message: " + toParse);
+
+			String tmp = toParse.substring(toParse.lastIndexOf("{"));
+			payload.put("key", tmp);
+
+			String protocol = tmp.substring(tmp.indexOf("{") + 1,
+					tmp.indexOf("}")).toLowerCase();
+			String source = tmp.substring(tmp.indexOf("}") + 1,
+					tmp.indexOf("->")).trim();
+			String dest = tmp.substring(tmp.indexOf("->") + 2, tmp.length())
+					.trim();
+
+			payload.put("protocol", protocol);
+
+			String source_ip = "";
+			String dest_ip = "";
+
+			if (source.contains(":")) {
+				String parts[] = source.split(":");
+				payload.put("ip_src_addr", parts[0]);
+				payload.put("ip_src_port", parts[1]);
+				source_ip = parts[0];
+			} else {
+				payload.put("ip_src_addr", source);
+				source_ip = source;
+
+			}
+
+			if (dest.contains(":")) {
+				String parts[] = dest.split(":");
+				payload.put("ip_dst_addr", parts[0]);
+				payload.put("ip_dst_port", parts[1]);
+				dest_ip = parts[0];
+			} else {
+				payload.put("ip_dst_addr", dest);
+				dest_ip = dest;
+			}
+
+			payload.put("timestamp", System.currentTimeMillis());
+			payload.put("original_string",
+					toParse.substring(0, toParse.indexOf("{")));
+
+			return payload;
+		} catch (Exception e) {
+			e.printStackTrace();
+			_LOG.error("Failed to parse: " + toParse);
+			return null;
+		}
+	}
+
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/05e188ba/opensoc-streaming/OpenSOC-MessageParsers/src/main/java/com/opensoc/parsing/parsers/GrokSourcefireParser.java
----------------------------------------------------------------------
diff --git a/opensoc-streaming/OpenSOC-MessageParsers/src/main/java/com/opensoc/parsing/parsers/GrokSourcefireParser.java b/opensoc-streaming/OpenSOC-MessageParsers/src/main/java/com/opensoc/parsing/parsers/GrokSourcefireParser.java
new file mode 100644
index 0000000..c75bfd5
--- /dev/null
+++ b/opensoc-streaming/OpenSOC-MessageParsers/src/main/java/com/opensoc/parsing/parsers/GrokSourcefireParser.java
@@ -0,0 +1,74 @@
+package com.opensoc.parsing.parsers;
+
+import java.net.URL;
+
+import oi.thekraken.grok.api.Match;
+import oi.thekraken.grok.api.Grok;
+import oi.thekraken.grok.api.exception.GrokException;
+
+import org.json.simple.JSONObject;
+
+
+public class GrokSourcefireParser extends AbstractParser{
+	
+	/**
+	 * 
+	 */
+	private static final long serialVersionUID = 1L;
+	Grok grok;
+	
+	public GrokSourcefireParser() throws GrokException
+	{
+		URL pattern_url = getClass().getClassLoader().getResource(
+				"pattarns/sourcefire");
+		grok = Grok.create(pattern_url.getFile());
+		grok.compile("%{SOURCEFIRE}");
+	}
+
+	public GrokSourcefireParser(String filepath) throws GrokException
+	{
+
+		grok = Grok.create(filepath);
+		grok.compile("%{SOURCEFIRE}");
+	}
+	
+	public GrokSourcefireParser(String filepath, String pattern) throws GrokException
+	{
+
+		grok = Grok.create(filepath);
+		grok.compile("%{"+pattern+"}");
+	}
+	
+	@Override
+	public JSONObject parse(byte[] raw_message) {
+		JSONObject payload = new JSONObject();
+		String toParse = "";
+		JSONObject toReturn;
+		
+
+		try {
+
+			toParse = new String(raw_message, "UTF-8");
+			Match gm = grok.match(toParse);
+			gm.captures();
+			
+			toReturn = new JSONObject();
+			
+			toReturn.putAll(gm.toMap());
+			toReturn.remove("SOURCEFIRE");
+			String proto = toReturn.get("protocol").toString();
+			proto = proto.replace("{", "");
+			proto = proto.replace("}", "");
+			toReturn.put("protocol", proto);
+			return toReturn;
+			
+		}
+		catch(Exception e)
+		{
+			e.printStackTrace();
+			return null;
+		}
+		
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/05e188ba/opensoc-streaming/OpenSOC-MessageParsers/src/main/java/com/opensoc/parsing/parsers/JSONCleaner.java
----------------------------------------------------------------------
diff --git a/opensoc-streaming/OpenSOC-MessageParsers/src/main/java/com/opensoc/parsing/parsers/JSONCleaner.java b/opensoc-streaming/OpenSOC-MessageParsers/src/main/java/com/opensoc/parsing/parsers/JSONCleaner.java
new file mode 100644
index 0000000..8ce04dc
--- /dev/null
+++ b/opensoc-streaming/OpenSOC-MessageParsers/src/main/java/com/opensoc/parsing/parsers/JSONCleaner.java
@@ -0,0 +1,78 @@
+package com.opensoc.parsing.parsers;
+
+import java.io.Serializable;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.Map;
+
+import org.json.simple.JSONObject;
+import org.json.simple.parser.JSONParser;
+import org.json.simple.parser.ParseException;
+
+/**
+ * @author kiran
+ *
+ */
+public class JSONCleaner implements Serializable {
+
+	/**
+	 * 
+	 */
+	private static final long serialVersionUID = 1L;
+
+
+	/**
+	 * @param jsonString
+	 * @return
+	 * @throws ParseException
+	 * Takes a json String as input and removes any Special Chars (^ a-z A-Z 0-9) in the keys
+	 */
+	@SuppressWarnings({"unchecked","rawtypes"})
+	public JSONObject Clean(String jsonString) throws ParseException
+	{
+		JSONParser parser = new JSONParser();
+		
+		
+		Map json = (Map) parser.parse(jsonString);
+		JSONObject output = new JSONObject();
+	    Iterator iter = json.entrySet().iterator();
+
+		 while(iter.hasNext()){
+		      Map.Entry entry = (Map.Entry)iter.next();
+		      
+		      String key = ((String)entry.getKey()).replaceAll("[^\\._a-zA-Z0-9]+","");
+		      output.put(key, entry.getValue());
+		    }
+
+		return output;
+	}
+	
+	
+	@SuppressWarnings({ "unchecked", "rawtypes", "unused" })
+	public static void main(String args[])
+	{
+		String jsonText = "{\"first_1\": 123, \"second\": [4, 5, 6], \"third\": 789}";
+		JSONCleaner cleaner = new JSONCleaner();
+		try {
+			//cleaner.Clean(jsonText);
+			Map obj=new HashMap();
+			  obj.put("name","foo");
+			  obj.put("num",new Integer(100));
+			  obj.put("balance",new Double(1000.21));
+			  obj.put("is_vip",new Boolean(true));
+			  obj.put("nickname",null);
+			Map obj1 = new HashMap();
+			obj1.put("sourcefile", obj);
+			
+			JSONObject json = new JSONObject(obj1);
+			System.out.println(json);
+			  
+			  
+			  
+			  System.out.print(jsonText);
+		} catch (Exception e) {
+			e.printStackTrace();
+		}
+	}
+	
+}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/05e188ba/opensoc-streaming/OpenSOC-MessageParsers/src/main/java/com/opensoc/parsing/parsers/PcapParser.java
----------------------------------------------------------------------
diff --git a/opensoc-streaming/OpenSOC-MessageParsers/src/main/java/com/opensoc/parsing/parsers/PcapParser.java b/opensoc-streaming/OpenSOC-MessageParsers/src/main/java/com/opensoc/parsing/parsers/PcapParser.java
new file mode 100644
index 0000000..5740977
--- /dev/null
+++ b/opensoc-streaming/OpenSOC-MessageParsers/src/main/java/com/opensoc/parsing/parsers/PcapParser.java
@@ -0,0 +1,185 @@
+package com.opensoc.parsing.parsers;
+
+import java.io.EOFException;
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.commons.io.FileUtils;
+import org.apache.log4j.Logger;
+import org.krakenapps.pcap.decoder.ethernet.EthernetDecoder;
+import org.krakenapps.pcap.decoder.ethernet.EthernetType;
+import org.krakenapps.pcap.decoder.ip.IpDecoder;
+import org.krakenapps.pcap.decoder.ip.Ipv4Packet;
+import org.krakenapps.pcap.decoder.tcp.TcpPacket;
+import org.krakenapps.pcap.decoder.udp.UdpPacket;
+import org.krakenapps.pcap.file.GlobalHeader;
+import org.krakenapps.pcap.packet.PacketHeader;
+import org.krakenapps.pcap.packet.PcapPacket;
+import org.krakenapps.pcap.util.Buffer;
+
+import com.opensoc.pcap.Constants;
+import com.opensoc.pcap.OpenSocEthernetDecoder;
+import com.opensoc.pcap.PacketInfo;
+import com.opensoc.pcap.PcapByteInputStream;
+
+/**
+ * The Class PcapParser.
+ * 
+ * @author sheetal
+ * @version $Revision: 1.0 $
+ */
+public final class PcapParser {
+
+  /** The Constant LOG. */
+  private static final Logger LOG = Logger.getLogger(PcapParser.class);
+
+  /** The ETHERNET_DECODER. */
+  private static final EthernetDecoder ETHERNET_DECODER = new OpenSocEthernetDecoder();
+
+  /** The ip decoder. */
+  private static final IpDecoder IP_DECODER = new IpDecoder();
+
+  // /** The tcp decoder. */
+  // private static final TcpDecoder TCP_DECODER = new TcpDecoder(new
+  // TcpPortProtocolMapper());
+  //
+  // /** The udp decoder. */
+  // private static final UdpDecoder UDP_DECODER = new UdpDecoder(new
+  // UdpPortProtocolMapper());
+
+  static {
+    // IP_DECODER.register(InternetProtocol.TCP, TCP_DECODER);
+    // IP_DECODER.register(InternetProtocol.UDP, UDP_DECODER);
+    ETHERNET_DECODER.register(EthernetType.IPV4, IP_DECODER);
+  }
+
+  /**
+   * Instantiates a new pcap parser.
+   */
+  private PcapParser() { // $codepro.audit.disable emptyMethod
+
+  }
+
+  /**
+   * Parses the.
+   * 
+   * @param pcap
+   *          the pcap
+   * @return the list * @throws IOException Signals that an I/O exception has
+   *         occurred. * @throws IOException * @throws IOException * @throws
+   *         IOException
+   * @throws IOException
+   *           Signals that an I/O exception has occurred.
+   */
+  public static List<PacketInfo> parse(byte[] pcap) throws IOException {
+    List<PacketInfo> packetInfoList = new ArrayList<PacketInfo>();
+
+    PcapByteInputStream pcapByteInputStream = new PcapByteInputStream(pcap);
+
+    GlobalHeader globalHeader = pcapByteInputStream.getGlobalHeader();
+    while (true) {
+      try
+
+      {
+        PcapPacket packet = pcapByteInputStream.getPacket();
+        // int packetCounter = 0;
+        // PacketHeader packetHeader = null;
+        // Ipv4Packet ipv4Packet = null;
+        TcpPacket tcpPacket = null;
+        UdpPacket udpPacket = null;
+        // Buffer packetDataBuffer = null;
+        int sourcePort = 0;
+        int destinationPort = 0;
+
+        // LOG.trace("Got packet # " + ++packetCounter);
+
+        // LOG.trace(packet.getPacketData());
+        ETHERNET_DECODER.decode(packet);
+
+        PacketHeader packetHeader = packet.getPacketHeader();
+        Ipv4Packet ipv4Packet = Ipv4Packet.parse(packet.getPacketData());
+
+        if (ipv4Packet.getProtocol() == Constants.PROTOCOL_TCP) {
+          tcpPacket = TcpPacket.parse(ipv4Packet);
+
+        }
+
+        if (ipv4Packet.getProtocol() == Constants.PROTOCOL_UDP) {
+
+          Buffer packetDataBuffer = ipv4Packet.getData();
+          sourcePort = packetDataBuffer.getUnsignedShort();
+          destinationPort = packetDataBuffer.getUnsignedShort();
+
+          udpPacket = new UdpPacket(ipv4Packet, sourcePort, destinationPort);
+
+          udpPacket.setLength(packetDataBuffer.getUnsignedShort());
+          udpPacket.setChecksum(packetDataBuffer.getUnsignedShort());
+          packetDataBuffer.discardReadBytes();
+          udpPacket.setData(packetDataBuffer);
+        }
+
+        packetInfoList.add(new PacketInfo(globalHeader, packetHeader, packet,
+            ipv4Packet, tcpPacket, udpPacket));
+      } catch (NegativeArraySizeException ignored) {
+        LOG.debug("Ignorable exception while parsing packet.", ignored);
+      } catch (EOFException eof) { // $codepro.audit.disable logExceptions
+        // Ignore exception and break
+        break;
+      }
+    }
+    return packetInfoList;
+  }
+
+  /**
+   * The main method.
+   * 
+   * @param args
+   *          the arguments
+   * @throws IOException
+   *           Signals that an I/O exception has occurred.
+   * @throws InterruptedException
+   *           the interrupted exception
+   */
+  public static void main(String[] args) throws IOException,
+      InterruptedException {
+
+    double totalIterations = 1000000;
+    double parallelism = 64;
+    double targetEvents = 1000000;
+
+    File fin = new File("/Users/sheetal/Downloads/bad_packets/bad_packet_1405988125427.pcap");
+    File fout = new File(fin.getAbsolutePath() + ".parsed");
+    byte[] pcapBytes = FileUtils.readFileToByteArray(fin);
+    long startTime = System.currentTimeMillis();
+    for (int i = 0; i < totalIterations; i++) {
+      List<PacketInfo> list = parse(pcapBytes);
+
+      for (PacketInfo packetInfo : list) {
+        System.out.println(packetInfo.getJsonIndexDoc());
+      }
+    }
+    long endTime = System.currentTimeMillis();
+
+    System.out.println("Time taken to process " + totalIterations + " events :"
+        + (endTime - startTime) + " milliseconds");
+
+    System.out
+        .println("With parallelism of "
+            + parallelism
+            + " estimated time to process "
+            + targetEvents
+            + " events: "
+            + (((((endTime - startTime) / totalIterations) * targetEvents) / parallelism) / 1000)
+            + " seconds");
+    System.out.println("With parallelism of " + parallelism
+        + " estimated # of events per second: "
+        + ((parallelism * 1000 * totalIterations) / (endTime - startTime))
+        + " events");
+    System.out.println("Expected Parallelism to process " + targetEvents
+        + " events in a second: "
+        + (targetEvents / ((1000 * totalIterations) / (endTime - startTime))));
+  }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/05e188ba/opensoc-streaming/OpenSOC-MessageParsers/src/main/resources/patters/sourcefire
----------------------------------------------------------------------
diff --git a/opensoc-streaming/OpenSOC-MessageParsers/src/main/resources/patters/sourcefire b/opensoc-streaming/OpenSOC-MessageParsers/src/main/resources/patters/sourcefire
new file mode 100644
index 0000000..672f684
--- /dev/null
+++ b/opensoc-streaming/OpenSOC-MessageParsers/src/main/resources/patters/sourcefire
@@ -0,0 +1,30 @@
+POSINT \b(?:[1-9][0-9]*)\b
+NONNEGINT \b(?:[0-9]+)\b
+WORD \b\w+\b
+NOTSPACE \S+
+SPACE \s*
+DATA .*?
+GREEDYDATA .*
+QUOTEDSTRING (?>(?<!\\)(?>"(?>\\.|[^\\"]+)+"|""|(?>'(?>\\.|[^\\']+)+')|''|(?>`(?>\\.|[^\\`]+)+`)|``))
+UUID [A-Fa-f0-9]{8}-(?:[A-Fa-f0-9]{4}-){3}[A-Fa-f0-9]{12}
+
+# Networking
+MAC (?:%{CISCOMAC}|%{WINDOWSMAC}|%{COMMONMAC})
+CISCOMAC (?:(?:[A-Fa-f0-9]{4}\.){2}[A-Fa-f0-9]{4})
+WINDOWSMAC (?:(?:[A-Fa-f0-9]{2}-){5}[A-Fa-f0-9]{2})
+COMMONMAC (?:(?:[A-Fa-f0-9]{2}:){5}[A-Fa-f0-9]{2})
+IPV6 ((([0-9A-Fa-f]{1,4}:){7}([0-9A-Fa-f]{1,4}|:))|(([0-9A-Fa-f]{1,4}:){6}(:[0-9A-Fa-f]{1,4}|((25[0-5]|2[0-4]\d|1\d\d|[1-9]?\d)(\.(25[0-5]|2[0-4]\d|1\d\d|[1-9]?\d)){3})|:))|(([0-9A-Fa-f]{1,4}:){5}(((:[0-9A-Fa-f]{1,4}){1,2})|:((25[0-5]|2[0-4]\d|1\d\d|[1-9]?\d)(\.(25[0-5]|2[0-4]\d|1\d\d|[1-9]?\d)){3})|:))|(([0-9A-Fa-f]{1,4}:){4}(((:[0-9A-Fa-f]{1,4}){1,3})|((:[0-9A-Fa-f]{1,4})?:((25[0-5]|2[0-4]\d|1\d\d|[1-9]?\d)(\.(25[0-5]|2[0-4]\d|1\d\d|[1-9]?\d)){3}))|:))|(([0-9A-Fa-f]{1,4}:){3}(((:[0-9A-Fa-f]{1,4}){1,4})|((:[0-9A-Fa-f]{1,4}){0,2}:((25[0-5]|2[0-4]\d|1\d\d|[1-9]?\d)(\.(25[0-5]|2[0-4]\d|1\d\d|[1-9]?\d)){3}))|:))|(([0-9A-Fa-f]{1,4}:){2}(((:[0-9A-Fa-f]{1,4}){1,5})|((:[0-9A-Fa-f]{1,4}){0,3}:((25[0-5]|2[0-4]\d|1\d\d|[1-9]?\d)(\.(25[0-5]|2[0-4]\d|1\d\d|[1-9]?\d)){3}))|:))|(([0-9A-Fa-f]{1,4}:){1}(((:[0-9A-Fa-f]{1,4}){1,6})|((:[0-9A-Fa-f]{1,4}){0,4}:((25[0-5]|2[0-4]\d|1\d\d|[1-9]?\d)(\.(25[0-5]|2[0-4]\d|1\d\d|[1-9]?\d)){3}))|:))|(:(((:[0-9A-Fa-f]{1,4}){1,7})|((:[0-9A-Fa-f]{1,4}){0,5}:((25[0-5
 ]|2[0-4]\d|1\d\d|[1-9]?\d)(\.(25[0-5]|2[0-4]\d|1\d\d|[1-9]?\d)){3}))|:)))(%.+)?
+IPV4 (?<![0-9])(?:(?:25[0-5]|2[0-4][0-9]|[0-1]?[0-9]{1,2})[.](?:25[0-5]|2[0-4][0-9]|[0-1]?[0-9]{1,2})[.](?:25[0-5]|2[0-4][0-9]|[0-1]?[0-9]{1,2})[.](?:25[0-5]|2[0-4][0-9]|[0-1]?[0-9]{1,2}))(?![0-9])
+IP (?:%{IPV6}|%{IPV4})
+HOSTNAME \b(?:[0-9A-Za-z][0-9A-Za-z-]{0,62})(?:\.(?:[0-9A-Za-z][0-9A-Za-z-]{0,62}))*(\.?|\b)
+HOST %{HOSTNAME}
+IPORHOST (?:%{HOSTNAME}|%{IP})
+HOSTPORT %{IPORHOST}:%{POSINT}
+
+#Sourcefire Logs
+protocol \{[a-zA-Z0-9]+\}
+ip_src_addr (?<![0-9])(?:(?:25[0-5]|2[0-4][0-9]|[0-1]?[0-9]{1,2})[.](?:25[0-5]|2[0-4][0-9]|[0-1]?[0-9]{1,2})[.](?:25[0-5]|2[0-4][0-9]|[0-1]?[0-9]{1,2})[.](?:25[0-5]|2[0-4][0-9]|[0-1]?[0-9]{1,2}))(?![0-9])
+ip_dst_addr (?<![0-9])(?:(?:25[0-5]|2[0-4][0-9]|[0-1]?[0-9]{1,2})[.](?:25[0-5]|2[0-4][0-9]|[0-1]?[0-9]{1,2})[.](?:25[0-5]|2[0-4][0-9]|[0-1]?[0-9]{1,2})[.](?:25[0-5]|2[0-4][0-9]|[0-1]?[0-9]{1,2}))(?![0-9])
+ip_src_port [0-9]+
+ip_dst_port [0-9]+
+SOURCEFIRE %{GREEDYDATA}%{protocol}\s%{ip_src_addr}\:%{ip_src_port}\s->\s%{ip_dst_addr}\:%{ip_dst_port}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/05e188ba/opensoc-streaming/OpenSOC-MessageParsers/src/test/java/com/opensoc/parsing/test/BasicIseParserTest.java
----------------------------------------------------------------------
diff --git a/opensoc-streaming/OpenSOC-MessageParsers/src/test/java/com/opensoc/parsing/test/BasicIseParserTest.java b/opensoc-streaming/OpenSOC-MessageParsers/src/test/java/com/opensoc/parsing/test/BasicIseParserTest.java
new file mode 100644
index 0000000..5d58cc5
--- /dev/null
+++ b/opensoc-streaming/OpenSOC-MessageParsers/src/test/java/com/opensoc/parsing/test/BasicIseParserTest.java
@@ -0,0 +1,210 @@
+/*
+ * 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.parsing.test;
+
+import java.io.BufferedReader;
+import java.io.FileReader;
+import java.io.IOException;
+import java.net.URL;
+
+import junit.framework.TestCase;
+
+import org.json.simple.JSONObject;
+import org.json.simple.parser.ParseException;
+
+import com.fasterxml.jackson.databind.JsonNode;
+import com.github.fge.jackson.JsonLoader;
+import com.github.fge.jsonschema.core.report.ProcessingReport;
+import com.github.fge.jsonschema.main.JsonSchemaFactory;
+import com.github.fge.jsonschema.main.JsonValidator;
+import com.opensoc.parsing.parsers.BasicIseParser;
+
+/**
+ * <ul>
+ * <li>Title: Basic ISE Parser</li>
+ * <li>Description: Junit Test Case for BasicISE Parser</li>
+ * <li>Created: AUG 25, 2014</li>
+ * </ul>
+ * 
+ * @version $Revision: 1.1 $
+ */
+public class BasicIseParserTest extends TestCase {
+	private static String rawMessage = "";
+
+	private static BasicIseParser iseParser = null;
+	private static String schema_string;
+
+	/**
+	 * Constructs a new <code>BasicIseParserTest</code> instance.
+	 * 
+	 * @param name
+	 */
+
+	public BasicIseParserTest(String name) {
+		super(name);
+	}
+
+	/**
+	 * 
+	 * @throws java.lang.Exception
+	 */
+	protected static void setUpBeforeClass() throws Exception {
+		setRawMessage("Aug  6 17:26:31 10.34.84.145 Aug  7 00:45:43 stage-pdp01 CISE_Profiler 0000024855 1 0 2014-08-07 00:45:43.741 -07:00 0000288542 80002 INFO  Profiler: Profiler EndPoint profiling event occurred, ConfigVersionId=113, EndpointCertainityMetric=10, EndpointIPAddress=10.56.111.14, EndpointMacAddress=3C:97:0E:C3:F8:F1, EndpointMatchedPolicy=Nortel-Device, EndpointNADAddress=10.56.72.127, EndpointOUI=Wistron InfoComm(Kunshan)Co.\\,Ltd., EndpointPolicy=Nortel-Device, EndpointProperty=StaticAssignment=false\\,PostureApplicable=Yes\\,PolicyVersion=402\\,IdentityGroupID=0c1d9270-68a6-11e1-bc72-0050568e013c\\,Total Certainty Factor=10\\,BYODRegistration=Unknown\\,FeedService=false\\,EndPointPolicyID=49054ed0-68a6-11e1-bc72-0050568e013c\\,FirstCollection=1407397543718\\,MatchedPolicyID=49054ed0-68a6-11e1-bc72-0050568e013c\\,TimeToProfile=19\\,StaticGroupAssignment=false\\,NmapSubnetScanID=0\\,DeviceRegistrationStatus=NotRegistered\\,PortalUser=, EndpointSourceEvent=SNMPQuery Prob
 e, EndpointIdentityGroup=Profiled, ProfilerServer=stage-pdp01.cisco.com,");
+
+	}
+
+	/**
+	 * 
+	 * @throws java.lang.Exception
+	 */
+	protected static void tearDownAfterClass() throws Exception {
+		setRawMessage("");
+	}
+
+	/*
+	 * (non-Javadoc)
+	 * 
+	 * @see junit.framework.TestCase#setUp()
+	 */
+
+	protected void setUp() throws Exception {
+		super.setUp();
+		assertNotNull(getRawMessage());
+		BasicIseParserTest.setIseParser(new BasicIseParser());
+		
+		URL schema_url = getClass().getClassLoader().getResource(
+				"TestSchemas/IseSchema.json");
+		
+		 schema_string = readSchemaFromFile(schema_url);
+	}
+
+	/*
+	 * (non-Javadoc)
+	 * 
+	 * @see junit.framework.TestCase#tearDown()
+	 */
+
+	protected void tearDown() throws Exception {
+		super.tearDown();
+	}
+
+	/**
+	 * Test method for
+	 * {@link com.opensoc.parsing.parsers.BasicIseParser#parse(byte[])}.
+	 * 
+	 * @throws IOException
+	 * @throws Exception
+	 */
+	public void testParse() throws ParseException, IOException, Exception {
+		// JSONObject parsed = iseParser.parse(getRawMessage().getBytes());
+		// assertNotNull(parsed);
+
+		URL log_url = getClass().getClassLoader().getResource("IseSample.log");
+
+		BufferedReader br = new BufferedReader(new FileReader(log_url.getFile()));
+		String line = "";
+		while ((line = br.readLine()) != null) {
+			System.out.println(line);
+			JSONObject parsed = iseParser.parse(line.getBytes());
+			System.out.println(parsed);
+			assertEquals(true, validateJsonData(schema_string, parsed.toString()));
+
+		}
+		br.close();
+
+	}
+
+	/**
+	 * Returns the rawMessage.
+	 * 
+	 * @return the rawMessage.
+	 */
+
+	public static String getRawMessage() {
+		return rawMessage;
+	}
+
+	/**
+	 * Sets the rawMessage.
+	 * 
+	 * @param rawMessage
+	 *            the rawMessage.
+	 */
+
+	public static void setRawMessage(String rawMessage) {
+
+		BasicIseParserTest.rawMessage = rawMessage;
+	}
+
+	/**
+	 * Returns the iseParser.
+	 * 
+	 * @return the iseParser.
+	 */
+
+	public BasicIseParser getIseParser() {
+		return iseParser;
+	}
+
+	/**
+	 * Sets the iseParser.
+	 * 
+	 * @param iseParser
+	 *            the iseParser.
+	 */
+
+	public static void setIseParser(BasicIseParser iseParser) {
+
+		BasicIseParserTest.iseParser = iseParser;
+	}
+
+	private boolean validateJsonData(final String jsonSchema, final String jsonData)
+			throws Exception {
+
+		final JsonNode d = JsonLoader.fromString(jsonData);
+		final JsonNode s = JsonLoader.fromString(jsonSchema);
+
+		final JsonSchemaFactory factory = JsonSchemaFactory.byDefault();
+		JsonValidator v = factory.getValidator();
+
+		ProcessingReport report = v.validate(s, d);
+		System.out.println(report);
+		
+		return report.toString().contains("success");
+
+	}
+
+	private String readSchemaFromFile(URL schema_url) throws Exception {
+		BufferedReader br = new BufferedReader(new FileReader(
+				schema_url.getFile()));
+		String line;
+		StringBuilder sb = new StringBuilder();
+		while ((line = br.readLine()) != null) {
+			System.out.println(line);
+			sb.append(line);
+		}
+		br.close();
+
+		String schema_string = sb.toString().replaceAll("\n", "");
+		schema_string = schema_string.replaceAll(" ", "");
+
+		System.out.println("Read in schema: " + schema_string);
+
+		return schema_string;
+
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/05e188ba/opensoc-streaming/OpenSOC-MessageParsers/src/test/java/com/opensoc/parsing/test/BasicLancopeParserTest.java
----------------------------------------------------------------------
diff --git a/opensoc-streaming/OpenSOC-MessageParsers/src/test/java/com/opensoc/parsing/test/BasicLancopeParserTest.java b/opensoc-streaming/OpenSOC-MessageParsers/src/test/java/com/opensoc/parsing/test/BasicLancopeParserTest.java
new file mode 100644
index 0000000..cfdf6e1
--- /dev/null
+++ b/opensoc-streaming/OpenSOC-MessageParsers/src/test/java/com/opensoc/parsing/test/BasicLancopeParserTest.java
@@ -0,0 +1,156 @@
+  /*
+ * 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.parsing.test;
+
+import java.util.Iterator;
+import java.util.Map;
+
+import junit.framework.TestCase;
+
+import org.json.simple.JSONObject;
+import org.json.simple.parser.JSONParser;
+import org.json.simple.parser.ParseException;
+
+import com.opensoc.parsing.parsers.BasicLancopeParser;
+
+ /**
+ * <ul>
+ * <li>Title: Junit for LancopeParserTest</li>
+ * <li>Description: </li>
+ * <li>Created: Aug 25, 2014</li>
+ * </ul>
+ * @version $Revision: 1.1 $
+ */
+public class BasicLancopeParserTest extends TestCase {
+
+    private  static String rawMessage = "";
+    private static BasicLancopeParser lancopeParser=null;   
+
+    /**
+     * Constructs a new <code>BasicLancopeParserTest</code> instance.
+     * @param name
+     */
+
+    public BasicLancopeParserTest(String name) {
+        super(name);
+    }
+
+    /**
+     
+     * @throws java.lang.Exception
+     */
+    protected static void setUpBeforeClass() throws Exception {
+        setRawMessage("{\"message\":\"<131>Jul 17 15:59:01 smc-01 StealthWatch[12365]: 2014-07-17T15:58:30Z 10.40.10.254 0.0.0.0 Minor High Concern Index The host's concern index has either exceeded the CI threshold or rapidly increased. Observed 36.55M points. Policy maximum allows up to 20M points.\",\"@version\":\"1\",\"@timestamp\":\"2014-07-17T15:56:05.992Z\",\"type\":\"syslog\",\"host\":\"10.122.196.201\"}");        
+    }
+
+    /**
+     
+     * @throws java.lang.Exception
+     */
+    protected static void tearDownAfterClass() throws Exception {
+    }
+
+    /* 
+     * (non-Javadoc)
+     * @see junit.framework.TestCase#setUp()
+     */
+
+    protected void setUp() throws Exception {
+        super.setUp();
+        setRawMessage("{\"message\":\"<131>Jul 17 15:59:01 smc-01 StealthWatch[12365]: 2014-07-17T15:58:30Z 10.40.10.254 0.0.0.0 Minor High Concern Index The host's concern index has either exceeded the CI threshold or rapidly increased. Observed 36.55M points. Policy maximum allows up to 20M points.\",\"@version\":\"1\",\"@timestamp\":\"2014-07-17T15:56:05.992Z\",\"type\":\"syslog\",\"host\":\"10.122.196.201\"}");        
+        assertNotNull(getRawMessage());
+        BasicLancopeParserTest.setLancopeParser(new BasicLancopeParser());        
+    }
+
+    /* 
+     * (non-Javadoc)
+     * @see junit.framework.TestCase#tearDown()
+     */
+
+    protected void tearDown() throws Exception {
+        super.tearDown();
+    }
+
+    /**
+     * Test method for {@link com.opensoc.parsing.parsers.BasicLancopeParser#parse(byte[])}.
+     */
+    public void testParse() {
+        byte messages[] = getRawMessage().getBytes();
+        assertNotNull(messages);        
+        JSONObject parsed = lancopeParser.parse(getRawMessage().getBytes());
+        assertNotNull(parsed);
+        
+        System.out.println(parsed);
+        JSONParser parser = new JSONParser();
+        
+        Map json=null;
+        try {
+            json = (Map) parser.parse(parsed.toJSONString());
+        } catch (ParseException e) {
+            e.printStackTrace();
+        }
+        Iterator iter = json.entrySet().iterator();
+            
+
+        while (iter.hasNext()) {
+            Map.Entry entry = (Map.Entry) iter.next();
+            String key = (String) entry.getKey();
+            assertNotNull((String) json.get("original_string").toString());
+            
+            assertNotNull((String)json.get("ip_src_addr").toString());
+            assertNotNull((String)json.get("ip_dst_addr").toString());            
+        }        
+    }
+    
+    /**
+     * Returns the rawMessage.
+     * @return the rawMessage.
+     */
+    
+    public static String getRawMessage() {
+        return BasicLancopeParserTest.rawMessage;
+    }
+
+    /**
+     * Sets the rawMessage.
+     * @param rawMessage the rawMessage.
+     */
+    
+    public static void setRawMessage(String rawMessage) {
+    
+        BasicLancopeParserTest.rawMessage = rawMessage;
+    }
+    /**
+     * Returns the lancopeParser.
+     * @return the lancopeParser.
+     */
+    
+    public static BasicLancopeParser getLancopeParser() {
+        return lancopeParser;
+    }
+
+    /**
+     * Sets the lancopeParser.
+     * @param lancopeParser the lancopeParser.
+     */
+    
+    public static void setLancopeParser(BasicLancopeParser lancopeParser) {
+    
+        BasicLancopeParserTest.lancopeParser = lancopeParser;
+    }
+}
+

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/05e188ba/opensoc-streaming/OpenSOC-MessageParsers/src/test/java/com/opensoc/parsing/test/BasicSourcefireParserTest.java
----------------------------------------------------------------------
diff --git a/opensoc-streaming/OpenSOC-MessageParsers/src/test/java/com/opensoc/parsing/test/BasicSourcefireParserTest.java b/opensoc-streaming/OpenSOC-MessageParsers/src/test/java/com/opensoc/parsing/test/BasicSourcefireParserTest.java
new file mode 100644
index 0000000..4faeb39
--- /dev/null
+++ b/opensoc-streaming/OpenSOC-MessageParsers/src/test/java/com/opensoc/parsing/test/BasicSourcefireParserTest.java
@@ -0,0 +1,107 @@
+/**
+ * 
+ */
+package com.opensoc.parsing.test;
+
+
+
+import java.util.Iterator;
+import java.util.Map;
+
+import junit.framework.TestCase;
+
+import org.json.simple.JSONObject;
+import org.json.simple.parser.JSONParser;
+import org.json.simple.parser.ParseException;
+
+import com.opensoc.parsing.parsers.BasicSourcefireParser;
+
+/**
+ * <ul>
+ * <li>Title: Test For SourceFireParser</li>
+ * <li>Description: </li>
+ * <li>Created: July 8, 2014</li>
+ * </ul>
+ * @version $Revision: 1.0 $
+ */
+public class BasicSourcefireParserTest extends TestCase
+	{
+
+	private  static String sourceFireString = "";
+	private BasicSourcefireParser sourceFireParser=null;
+
+
+
+	/**
+	 * @throws java.lang.Exception
+	 */
+	public static void setUpBeforeClass() throws Exception {
+	}
+
+	/**
+	 * @throws java.lang.Exception
+	 */
+	public static void tearDownAfterClass() throws Exception {
+		setSourceFireString("");
+	}
+
+	/**
+	 * @throws java.lang.Exception
+	 */
+	public void setUp() throws Exception {
+		setSourceFireString("SFIMS: [Primary Detection Engine (a7213248-6423-11e3-8537-fac6a92b7d9d)][MTD Access Control] Connection Type: Start, User: Unknown, Client: Unknown, Application Protocol: Unknown, Web App: Unknown, Firewall Rule Name: MTD Access Control, Firewall Rule Action: Allow, Firewall Rule Reasons: Unknown, URL Category: Unknown, URL_Reputation: Risk unknown, URL: Unknown, Interface Ingress: s1p1, Interface Egress: N/A, Security Zone Ingress: Unknown, Security Zone Egress: N/A, Security Intelligence Matching IP: None, Security Intelligence Category: None, {TCP} 72.163.0.129:60517 -> 10.1.128.236:443");		assertNotNull(getSourceFireString());
+		sourceFireParser = new BasicSourcefireParser();		
+	}
+
+	/**
+	 * 	
+	 * 	
+	 * @throws java.lang.Exception
+	 */
+	public void tearDown() throws Exception {
+		sourceFireParser = null;
+	}
+
+	/**
+	 * Test method for {@link com.opensoc.parsing.parsers.BasicSourcefireParser#parse(java.lang.String)}.
+	 */
+	@SuppressWarnings({ "rawtypes", "unused" })
+	public void testParse() {
+		JSONObject parsed = sourceFireParser.parse(getSourceFireString().getBytes());
+		assertNotNull(parsed);
+		
+		System.out.println(parsed);
+		JSONParser parser = new JSONParser();
+
+		Map json=null;
+		try {
+			json = (Map) parser.parse(parsed.toJSONString());
+		} catch (ParseException e) {
+			e.printStackTrace();
+		}
+		Iterator iter = json.entrySet().iterator();
+			
+
+		while (iter.hasNext()) {
+			Map.Entry entry = (Map.Entry) iter.next();
+			String key = (String) entry.getKey();
+            String value = (String) json.get("original_string").toString();
+			assertNotNull(value);
+		}
+	}
+
+	/**
+	 * Returns SourceFire Input String
+	 */
+	public static String getSourceFireString() {
+		return sourceFireString;
+	}
+
+		
+	/**
+	 * Sets SourceFire Input String
+	 */	
+	public static void setSourceFireString(String sourceFireString) {
+		BasicSourcefireParserTest.sourceFireString = sourceFireString;
+	}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/05e188ba/opensoc-streaming/OpenSOC-MessageParsers/src/test/java/com/opensoc/parsing/test/BroParserTest.java
----------------------------------------------------------------------
diff --git a/opensoc-streaming/OpenSOC-MessageParsers/src/test/java/com/opensoc/parsing/test/BroParserTest.java b/opensoc-streaming/OpenSOC-MessageParsers/src/test/java/com/opensoc/parsing/test/BroParserTest.java
new file mode 100644
index 0000000..6742011
--- /dev/null
+++ b/opensoc-streaming/OpenSOC-MessageParsers/src/test/java/com/opensoc/parsing/test/BroParserTest.java
@@ -0,0 +1,122 @@
+package com.opensoc.parsing.test;
+
+
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+import org.json.simple.JSONObject;
+import org.json.simple.parser.JSONParser;
+import org.json.simple.parser.ParseException;
+
+import junit.framework.TestCase;
+
+import com.opensoc.parsing.parsers.BasicBroParser;
+
+/**
+ * <ul>
+ * <li>Title: Test For BroParser</li>
+ * <li>Description: </li>
+ * <li>Created: July 8, 2014</li>
+ * </ul>
+ * @version $Revision: 1.0 $
+ */
+public class BroParserTest extends TestCase {
+	
+	private static String broJsonString="";
+	private static BasicBroParser broParser=null;
+	
+    /**
+     * Constructs a new <code>BroParserTest</code> instance.
+     */
+    public BroParserTest() {
+        super();
+    }	
+
+
+	/**
+	 * @throws java.lang.Exception
+	 */
+	public static void setUpBeforeClass() throws Exception {
+		
+	}
+
+	/**
+	 * @throws java.lang.Exception
+	 */
+	public static void tearDownAfterClass() throws Exception {
+		setBroJsonString("");
+	}
+
+	/**
+	 * @throws java.lang.Exception
+	 */
+	public void setUp() throws Exception {
+	    setBroJsonString("{\"http\":{\"ts\":1402307733473,\"uid\":\"CTo78A11g7CYbbOHvj\",\"id.orig_h\":\"192.249.113.37\",\"id.orig_p\":58808,\"id.resp_h\":\"72.163.4.161\",\"id.resp_p\":80,\"trans_depth\":1,\"method\":\"GET\",\"host\":\"www.cisco.com\",\"uri\":\"/\",\"user_agent\":\"curl/7.22.0 (x86_64-pc-linux-gnu) libcurl/7.22.0 OpenSSL/1.0.1 zlib/1.2.3.4 libidn/1.23 librtmp/2.3\",\"request_body_len\":0,\"response_body_len\":25523,\"status_code\":200,\"status_msg\":\"OK\",\"tags\":[],\"resp_fuids\":[\"FJDyMC15lxUn5ngPfd\"],\"resp_mime_types\":[\"text/html\"]}}");	    
+		assertNotNull(getBroJsonString());
+		BroParserTest.setBroParser(new BasicBroParser());		
+	}
+	
+	/**
+	 * @throws ParseException
+	 * Tests for Parse Method
+	 * Parses Static json String and checks if any spl chars are present in parsed string.
+	 */
+	@SuppressWarnings({ "unused", "rawtypes" })
+	public void testParse() throws ParseException {
+
+
+		BasicBroParser broparser = new BasicBroParser();
+		assertNotNull(getBroJsonString());
+		JSONObject cleanJson = broparser.parse(getBroJsonString().getBytes());
+        assertNotNull(cleanJson);		
+		System.out.println(cleanJson);
+
+
+		Pattern p = Pattern.compile("[^\\._a-z0-9 ]", Pattern.CASE_INSENSITIVE);
+
+		JSONParser parser = new JSONParser();
+
+		Map json = (Map) cleanJson;
+		Map output = new HashMap();
+		Iterator iter = json.entrySet().iterator();
+
+		while (iter.hasNext()) {
+			Map.Entry entry = (Map.Entry) iter.next();
+			String key = (String) entry.getKey();
+
+			Matcher m = p.matcher(key);
+			boolean b = m.find();
+			// Test False
+			assertFalse(b);
+		}
+
+	}
+    /**
+     * Returns the instance of BroParser
+     */
+	public static BasicBroParser getBroParser() {
+		return broParser;
+	}
+    /**
+     * Sets the instance of BroParser
+     */
+	public static void setBroParser(BasicBroParser broParser) {
+		BroParserTest.broParser = broParser;
+	}
+    /**
+     * Return BroPaser JSON String
+     */
+	public static String getBroJsonString() {
+		return BroParserTest.broJsonString;
+	}
+
+    /**
+     * Sets BroPaser JSON String
+     */
+	public static void setBroJsonString(String broJsonString) {
+		BroParserTest.broJsonString = broJsonString;
+	}	
+}