You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@metron.apache.org by rm...@apache.org on 2016/04/26 16:46:04 UTC

[16/51] [partial] incubator-metron git commit: METRON-113 Project Reorganization (merrimanr) closes apache/incubator-metron#88

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/0117987e/metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/asa/GrokAsaParser.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/asa/GrokAsaParser.java b/metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/asa/GrokAsaParser.java
new file mode 100644
index 0000000..0f8a862
--- /dev/null
+++ b/metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/asa/GrokAsaParser.java
@@ -0,0 +1,280 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.metron.parsers.asa;
+
+import oi.thekraken.grok.api.Grok;
+import oi.thekraken.grok.api.Match;
+import oi.thekraken.grok.api.exception.GrokException;
+import org.apache.commons.io.IOUtils;
+import org.apache.metron.parsers.BasicParser;
+import org.json.simple.JSONObject;
+
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.text.ParseException;
+import java.text.SimpleDateFormat;
+import java.util.*;
+
+public class GrokAsaParser extends BasicParser {
+
+	private static final long serialVersionUID = 945353287115350798L;
+	private transient  Grok  grok;
+	Map<String, String> patternMap;
+	private transient  Map<String, Grok> grokMap;
+	private transient  InputStream pattern_url;
+
+	public static final String PREFIX = "stream2file";
+	public static final String SUFFIX = ".tmp";
+
+	public static File stream2file(InputStream in) throws IOException {
+		final File tempFile = File.createTempFile(PREFIX, SUFFIX);
+		tempFile.deleteOnExit();
+		try (FileOutputStream out = new FileOutputStream(tempFile)) {
+			IOUtils.copy(in, out);
+		}
+		return tempFile;
+	}
+
+	public GrokAsaParser() throws Exception {
+		// pattern_url = Resources.getResource("patterns/asa");
+
+		pattern_url = getClass().getClassLoader().getResourceAsStream(
+						"src/main/patterns/asa");
+
+		File file = stream2file(pattern_url);
+		grok = Grok.create(file.getPath());
+
+		patternMap = getPatternMap();
+		grokMap = getGrokMap();
+
+		grok.compile("%{CISCO_TAGGED_SYSLOG}");
+	}
+
+	public GrokAsaParser(String filepath) throws Exception {
+
+		grok = Grok.create(filepath);
+		// grok.getNamedRegexCollection().put("ciscotag","CISCOFW302013_302014_302015_302016");
+		grok.compile("%{CISCO_TAGGED_SYSLOG}");
+
+	}
+
+	public GrokAsaParser(String filepath, String pattern) throws Exception {
+
+		grok = Grok.create(filepath);
+		grok.compile("%{" + pattern + "}");
+	}
+
+	private Map<String, Object> getMap(String pattern, String text)
+			throws GrokException {
+
+		Grok g = grokMap.get(pattern);
+		if (g != null) {
+			Match gm = g.match(text);
+			gm.captures();
+			return gm.toMap();
+		} else {
+			return new HashMap<String, Object>();
+		}
+
+	}
+
+	private Map<String, Grok> getGrokMap() throws GrokException, IOException {
+		Map<String, Grok> map = new HashMap<String, Grok>();
+
+		for (Map.Entry<String, String> entry : patternMap.entrySet()) {
+			File file = stream2file(pattern_url);
+			Grok grok = Grok.create(file.getPath());
+			grok.compile("%{" + entry.getValue() + "}");
+
+			map.put(entry.getValue(), grok);
+
+		}
+
+		return map;
+	}
+
+	private Map<String, String> getPatternMap() {
+		Map<String, String> map = new HashMap<String, String>();
+
+		map.put("ASA-2-106001", "CISCOFW106001");
+		map.put("ASA-2-106006", "CISCOFW106006_106007_106010");
+		map.put("ASA-2-106007", "CISCOFW106006_106007_106010");
+		map.put("ASA-2-106010", "CISCOFW106006_106007_106010");
+		map.put("ASA-3-106014", "CISCOFW106014");
+		map.put("ASA-6-106015", "CISCOFW106015");
+		map.put("ASA-1-106021", "CISCOFW106021");
+		map.put("ASA-4-106023", "CISCOFW106023");
+		map.put("ASA-5-106100", "CISCOFW106100");
+		map.put("ASA-6-110002", "CISCOFW110002");
+		map.put("ASA-6-302010", "CISCOFW302010");
+		map.put("ASA-6-302013", "CISCOFW302013_302014_302015_302016");
+		map.put("ASA-6-302014", "CISCOFW302013_302014_302015_302016");
+		map.put("ASA-6-302015", "CISCOFW302013_302014_302015_302016");
+		map.put("ASA-6-302016", "CISCOFW302013_302014_302015_302016");
+		map.put("ASA-6-302020", "CISCOFW302020_302021");
+		map.put("ASA-6-302021", "CISCOFW302020_302021");
+		map.put("ASA-6-305011", "CISCOFW305011");
+		map.put("ASA-3-313001", "CISCOFW313001_313004_313008");
+		map.put("ASA-3-313004", "CISCOFW313001_313004_313008");
+		map.put("ASA-3-313008", "CISCOFW313001_313004_313008");
+		map.put("ASA-4-313005", "CISCOFW313005");
+		map.put("ASA-4-402117", "CISCOFW402117");
+		map.put("ASA-4-402119", "CISCOFW402119");
+		map.put("ASA-4-419001", "CISCOFW419001");
+		map.put("ASA-4-419002", "CISCOFW419002");
+		map.put("ASA-4-500004", "CISCOFW500004");
+		map.put("ASA-6-602303", "CISCOFW602303_602304");
+		map.put("ASA-6-602304", "CISCOFW602303_602304");
+		map.put("ASA-7-710001", "CISCOFW710001_710002_710003_710005_710006");
+		map.put("ASA-7-710002", "CISCOFW710001_710002_710003_710005_710006");
+		map.put("ASA-7-710003", "CISCOFW710001_710002_710003_710005_710006");
+		map.put("ASA-7-710005", "CISCOFW710001_710002_710003_710005_710006");
+		map.put("ASA-7-710006", "CISCOFW710001_710002_710003_710005_710006");
+		map.put("ASA-6-713172", "CISCOFW713172");
+		map.put("ASA-4-733100", "CISCOFW733100");
+		map.put("ASA-6-305012", "CISCOFW305012");
+		map.put("ASA-7-609001", "CISCOFW609001");
+		map.put("ASA-7-609002", "CISCOFW609002");
+
+		return map;
+	}
+
+	public static Long convertToEpoch(String m, String d, String ts,
+			boolean adjust_timezone) throws ParseException {
+		d = d.trim();
+
+		if (d.length() <= 2)
+			d = "0" + d;
+
+		Date date = new SimpleDateFormat("MMM", Locale.ENGLISH).parse(m);
+		Calendar cal = Calendar.getInstance();
+		cal.setTime(date);
+		String month = String.valueOf(cal.get(Calendar.MONTH));
+		int year = Calendar.getInstance().get(Calendar.YEAR);
+
+		if (month.length() <= 2)
+			month = "0" + month;
+
+		String coglomerated_ts = year + "-" + month + "-" + d + " " + ts;
+
+		System.out.println(coglomerated_ts);
+
+		SimpleDateFormat sdf = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss");
+
+		if (adjust_timezone)
+			sdf.setTimeZone(TimeZone.getTimeZone("GMT"));
+
+		date = sdf.parse(coglomerated_ts);
+		long timeInMillisSinceEpoch = date.getTime();
+
+		return timeInMillisSinceEpoch;
+	}
+	
+	@Override
+	public void init() {
+		// pattern_url = Resources.getResource("patterns/asa");
+
+				pattern_url = getClass().getClassLoader().getResourceAsStream(
+								"src/main/patterns/asa");
+
+				File file = null;
+				try {
+					file = stream2file(pattern_url);
+				} catch (IOException e) {
+					// TODO Auto-generated catch block
+					e.printStackTrace();
+				}
+				try {
+					grok = Grok.create(file.getPath());
+				} catch (GrokException e) {
+					// TODO Auto-generated catch block
+					e.printStackTrace();
+				}
+
+				patternMap = getPatternMap();
+				try {
+					grokMap = getGrokMap();
+				} catch (GrokException | IOException e1) {
+					// TODO Auto-generated catch block
+					e1.printStackTrace();
+				}
+
+				try {
+					grok.compile("%{CISCO_TAGGED_SYSLOG}");
+				} catch (GrokException e) {
+					// TODO Auto-generated catch block
+					e.printStackTrace();
+				}
+	}
+
+	@Override
+	public List<JSONObject> parse(byte[] raw_message) {
+
+		String toParse = "";
+		JSONObject toReturn;
+		List<JSONObject> messages = new ArrayList<>();
+		try {
+
+			toParse = new String(raw_message, "UTF-8");
+
+			System.out.println("Received message: " + toParse);
+
+			Match gm = grok.match(toParse);
+			gm.captures();
+
+			toReturn = new JSONObject();
+
+			toReturn.putAll(gm.toMap());
+
+			String str = toReturn.get("ciscotag").toString();
+			String pattern = patternMap.get(str);
+
+			Map<String, Object> response = getMap(pattern, toParse);
+
+			toReturn.putAll(response);
+
+			//System.out.println("*******I MAPPED: " + toReturn);
+			long timestamp = convertToEpoch(toReturn.get("MONTH").toString(), toReturn
+											.get("MONTHDAY").toString(),
+							toReturn.get("TIME").toString(),
+							true);
+			toReturn.put("timestamp", timestamp);
+			
+			toReturn.remove("MONTHDAY");
+			toReturn.remove("TIME");
+			toReturn.remove("MINUTE");
+			toReturn.remove("HOUR");
+			toReturn.remove("YEAR");
+			toReturn.remove("SECOND");
+			
+			toReturn.put("ip_src_addr", toReturn.remove("IPORHOST"));
+			toReturn.put("original_string", toParse);
+			messages.add(toReturn);
+			return messages;
+
+		} catch (Exception e) {
+			e.printStackTrace();
+			return null;
+		}
+
+	}
+
+	
+}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/0117987e/metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/bolt/ParserBolt.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/bolt/ParserBolt.java b/metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/bolt/ParserBolt.java
new file mode 100644
index 0000000..e29f900
--- /dev/null
+++ b/metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/bolt/ParserBolt.java
@@ -0,0 +1,89 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.metron.parsers.bolt;
+
+import backtype.storm.task.OutputCollector;
+import backtype.storm.task.TopologyContext;
+import backtype.storm.topology.OutputFieldsDeclarer;
+import backtype.storm.tuple.Tuple;
+import org.apache.metron.common.Constants;
+import org.apache.metron.common.bolt.ConfiguredBolt;
+import org.apache.metron.parsers.filters.GenericMessageFilter;
+import org.apache.metron.common.utils.ErrorUtils;
+import org.apache.metron.parsers.interfaces.MessageFilter;
+import org.apache.metron.parsers.interfaces.MessageParser;
+import org.apache.metron.common.interfaces.MessageWriter;
+import org.json.simple.JSONObject;
+
+import java.util.List;
+import java.util.Map;
+
+public class ParserBolt extends ConfiguredBolt {
+
+  private OutputCollector collector;
+  private MessageParser<JSONObject> parser;
+  private MessageFilter<JSONObject> filter = new GenericMessageFilter();
+  private MessageWriter<JSONObject> writer;
+  private String sensorType;
+
+  public ParserBolt(String zookeeperUrl, String sensorType, MessageParser<JSONObject> parser, MessageWriter<JSONObject> writer) {
+    super(zookeeperUrl);
+    this.parser = parser;
+    this.sensorType = sensorType;
+    this.writer = writer;
+  }
+
+  public ParserBolt withMessageFilter(MessageFilter<JSONObject> filter) {
+    this.filter = filter;
+    return this;
+  }
+
+  @SuppressWarnings("unchecked")
+  @Override
+  public void prepare(Map stormConf, TopologyContext context, OutputCollector collector) {
+    super.prepare(stormConf, context, collector);
+    this.collector = collector;
+    parser.init();
+    writer.init();
+  }
+
+  @SuppressWarnings("unchecked")
+  @Override
+  public void execute(Tuple tuple) {
+    byte[] originalMessage = tuple.getBinary(0);
+    try {
+      List<JSONObject> messages = parser.parse(originalMessage);
+      for(JSONObject message: messages) {
+        if (parser.validate(message)) {
+          if (filter != null && filter.emitTuple(message)) {
+            message.put(Constants.SENSOR_TYPE, sensorType);
+            writer.write(sensorType, configurations, tuple, message);
+          }
+        }
+      }
+      collector.ack(tuple);
+    } catch (Throwable ex) {
+      ErrorUtils.handleError(collector, ex, Constants.ERROR_STREAM);
+    }
+  }
+
+  @Override
+  public void declareOutputFields(OutputFieldsDeclarer declarer) {
+
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/0117987e/metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/bro/BasicBroParser.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/bro/BasicBroParser.java b/metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/bro/BasicBroParser.java
new file mode 100644
index 0000000..71eb64f
--- /dev/null
+++ b/metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/bro/BasicBroParser.java
@@ -0,0 +1,159 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.metron.parsers.bro;
+
+import org.apache.metron.parsers.BasicParser;
+import org.json.simple.JSONArray;
+import org.json.simple.JSONObject;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.List;
+
+@SuppressWarnings("serial")
+public class BasicBroParser extends BasicParser {
+
+    protected static final Logger _LOG = LoggerFactory
+            .getLogger(BasicBroParser.class);
+    private JSONCleaner cleaner = new JSONCleaner();
+
+    @Override
+    public void init() {
+
+    }
+
+    @SuppressWarnings("unchecked")
+    public List<JSONObject> parse(byte[] msg) {
+
+        _LOG.trace("[Metron] Starting to parse incoming message");
+
+        String rawMessage = null;
+        List<JSONObject> messages = new ArrayList<>();
+        try {
+            rawMessage = new String(msg, "UTF-8");
+            _LOG.trace("[Metron] Received message: " + rawMessage);
+
+            JSONObject cleanedMessage = cleaner.clean(rawMessage);
+            _LOG.debug("[Metron] Cleaned message: " + cleanedMessage);
+
+            if (cleanedMessage == null || cleanedMessage.isEmpty()) {
+                throw new Exception("Unable to clean message: " + rawMessage);
+            }
+
+            String key;
+            JSONObject payload;
+            if (cleanedMessage.containsKey("type")) {
+                key = cleanedMessage.get("type").toString();
+                payload = cleanedMessage;
+            } else {
+                key = cleanedMessage.keySet().iterator().next().toString();
+
+                if (key == null) {
+                    throw new Exception("Unable to retrieve key for message: "
+                            + rawMessage);
+                }
+
+                payload = (JSONObject) cleanedMessage.get(key);
+            }
+
+            if (payload == null) {
+                throw new Exception("Unable to retrieve payload for message: "
+                    + rawMessage);
+            }
+
+            String originalString = key.toUpperCase() + " |";
+            for (Object k : payload.keySet()) {
+                String value = payload.get(k).toString();
+                originalString += " " + k.toString() + ":" + value;
+            }
+            payload.put("original_string", originalString);
+
+            replaceKey(payload, "timestamp", new String[]{ "ts" });
+
+            long timestamp = 0L;
+            if (payload.containsKey("timestamp")) {
+                try {
+                    String broTimestamp = payload.get("timestamp").toString();
+                    String convertedTimestamp = broTimestamp.replace(".","");
+                    convertedTimestamp = convertedTimestamp.substring(0,13);
+                    timestamp = Long.parseLong(convertedTimestamp);
+                    payload.put("timestamp", timestamp);
+                    payload.put("bro_timestamp",broTimestamp);
+                    _LOG.trace(String.format("[Metron] new bro record - timestamp : %s", payload.get("timestamp")));
+                } catch (NumberFormatException nfe) {
+                    _LOG.error(String.format("[Metron] timestamp is invalid: %s", payload.get("timestamp")));
+                    payload.put("timestamp", 0);
+                }
+            }
+
+            boolean ipSrcReplaced = replaceKey(payload, "ip_src_addr", new String[]{"source_ip", "id.orig_h"});
+            if (!ipSrcReplaced) {
+                replaceKeyArray(payload, "ip_src_addr", new String[]{ "tx_hosts" });
+            }
+
+            boolean ipDstReplaced = replaceKey(payload, "ip_dst_addr", new String[]{"dest_ip", "id.resp_h"});
+            if (!ipDstReplaced) {
+                replaceKeyArray(payload, "ip_dst_addr", new String[]{ "rx_hosts" });
+            }
+
+            replaceKey(payload, "ip_src_port", new String[]{"source_port", "id.orig_p"});
+            replaceKey(payload, "ip_dst_port", new String[]{"dest_port", "id.resp_p"});
+
+            payload.put("protocol", key);
+            _LOG.debug("[Metron] Returning parsed message: " + payload);
+            messages.add(payload);
+            return messages;
+
+        } catch (Exception e) {
+
+            _LOG.error("Unable to Parse Message: " + rawMessage);
+            e.printStackTrace();
+            return null;
+        }
+
+    }
+
+    private boolean replaceKey(JSONObject payload, String toKey, String[] fromKeys) {
+        for (String fromKey : fromKeys) {
+            if (payload.containsKey(fromKey)) {
+                Object value = payload.remove(fromKey);
+                payload.put(toKey, value);
+                _LOG.trace(String.format("[Metron] Added %s to %s", toKey, payload));
+                return true;
+            }
+        }
+        return false;
+    }
+
+    private boolean replaceKeyArray(JSONObject payload, String toKey, String[] fromKeys) {
+        for (String fromKey : fromKeys) {
+            if (payload.containsKey(fromKey)) {
+                JSONArray value = (JSONArray) payload.remove(fromKey);
+                if (value != null && !value.isEmpty()) {
+                    payload.put(toKey, value.get(0));
+                    _LOG.trace(String.format("[Metron] Added %s to %s", toKey, payload));
+                    return true;
+                }
+            }
+        }
+        return false;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/0117987e/metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/bro/JSONCleaner.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/bro/JSONCleaner.java b/metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/bro/JSONCleaner.java
new file mode 100644
index 0000000..41d97f4
--- /dev/null
+++ b/metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/bro/JSONCleaner.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 org.apache.metron.parsers.bro;
+
+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;
+
+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/0117987e/metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/filters/BroMessageFilter.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/filters/BroMessageFilter.java b/metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/filters/BroMessageFilter.java
new file mode 100644
index 0000000..d026d08
--- /dev/null
+++ b/metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/filters/BroMessageFilter.java
@@ -0,0 +1,62 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.metron.parsers.filters;
+
+import org.apache.commons.configuration.Configuration;
+import org.apache.metron.parsers.interfaces.MessageFilter;
+import org.json.simple.JSONObject;
+
+import java.io.Serializable;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+public class BroMessageFilter implements MessageFilter<JSONObject>,
+				Serializable {
+
+	/**
+	 * Filter protocols based on whitelists and blacklists
+	 */
+	
+	private static final long serialVersionUID = -3824683649114625033L;
+	private String _key;
+	private final Set<String> _known_protocols;
+
+	 /**
+	 * @param  conf  Commons configuration for reading properties files
+	 * @param  key Key in a JSON mesage where the protocol field is located
+	 */
+	
+	@SuppressWarnings({ "unchecked", "rawtypes" })
+	public BroMessageFilter(Configuration conf, String key) {
+		_key = key;
+		_known_protocols = new HashSet<>();
+		List known_protocols = conf.getList("source.known.protocols");
+		_known_protocols.addAll(known_protocols);
+	}
+
+	 /**
+	 * @param  message  JSON representation of a message with a protocol field
+	 * @return      False if message if filtered and True if message is not filtered
+	 */
+	
+	public boolean emitTuple(JSONObject message) {
+		String protocol = (String) message.get(_key);
+		return _known_protocols.contains(protocol);
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/0117987e/metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/filters/GenericMessageFilter.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/filters/GenericMessageFilter.java b/metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/filters/GenericMessageFilter.java
new file mode 100644
index 0000000..9defe32
--- /dev/null
+++ b/metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/filters/GenericMessageFilter.java
@@ -0,0 +1,34 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.metron.parsers.filters;
+
+import org.apache.metron.parsers.interfaces.MessageFilter;
+import org.json.simple.JSONObject;
+
+import java.io.Serializable;
+
+public class GenericMessageFilter implements MessageFilter<JSONObject>,
+				Serializable {
+
+	private static final long serialVersionUID = 3626397212398318852L;
+
+	public boolean emitTuple(JSONObject message) {
+		return true;
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/0117987e/metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/fireeye/BasicFireEyeParser.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/fireeye/BasicFireEyeParser.java b/metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/fireeye/BasicFireEyeParser.java
new file mode 100644
index 0000000..b90d2b7
--- /dev/null
+++ b/metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/fireeye/BasicFireEyeParser.java
@@ -0,0 +1,218 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.metron.parsers.fireeye;
+
+import com.google.common.base.Joiner;
+import com.google.common.collect.ArrayListMultimap;
+import com.google.common.collect.Multimap;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.metron.parsers.utils.ParserUtils;
+import org.apache.metron.parsers.BasicParser;
+import org.json.simple.JSONObject;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.text.ParseException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+public class BasicFireEyeParser extends BasicParser {
+
+	private static final long serialVersionUID = 6328907550159134550L;
+	protected static final Logger LOG = LoggerFactory
+					.getLogger(BasicFireEyeParser.class);
+
+
+	String tsRegex ="([a-zA-Z]{3})\\s+(\\d+)\\s+(\\d+\\:\\d+\\:\\d+)\\s+(\\d+\\.\\d+\\.\\d+\\.\\d+)";
+	
+	
+	Pattern tsPattern = Pattern.compile(tsRegex);
+	// private transient static MetronGrok grok;
+	// private transient static InputStream pattern_url;
+
+	public BasicFireEyeParser() throws Exception {
+		// pattern_url = getClass().getClassLoader().getResourceAsStream(
+		// "patterns/fireeye");
+		//
+		// File file = ParserUtils.stream2file(pattern_url);
+		// grok = MetronGrok.create(file.getPath());
+		//
+		// grok.compile("%{FIREEYE_BASE}");
+	}
+
+	@Override
+	public void init() {
+
+	}
+
+	@Override
+	public List<JSONObject> parse(byte[] raw_message) {
+		String toParse = "";
+		List<JSONObject> messages = new ArrayList<>();
+		try {
+
+			toParse = new String(raw_message, "UTF-8");
+
+			// String[] mTokens = toParse.split(" ");
+
+			String positveIntPattern = "<[1-9][0-9]*>";
+			Pattern p = Pattern.compile(positveIntPattern);
+			Matcher m = p.matcher(toParse);
+
+			String delimiter = "";
+
+			while (m.find()) {
+				delimiter = m.group();
+
+			}
+
+			if (!StringUtils.isBlank(delimiter)) {
+				String[] tokens = toParse.split(delimiter);
+
+				if (tokens.length > 1)
+					toParse = delimiter + tokens[1];
+
+			}
+
+			JSONObject toReturn = parseMessage(toParse);
+
+			toReturn.put("timestamp", getTimeStamp(toParse,delimiter));
+			messages.add(toReturn);
+			return messages;
+
+		} catch (Exception e) {
+			e.printStackTrace();
+			return null;
+		}
+
+	}
+
+	private long getTimeStamp(String toParse,String delimiter) throws ParseException {
+		
+		long ts = 0;
+		String month = null;
+		String day = null;
+		String time = null;
+		Matcher tsMatcher = tsPattern.matcher(toParse);
+		if (tsMatcher.find()) {
+			month = tsMatcher.group(1);
+			day = tsMatcher.group(2);
+			time = tsMatcher.group(3);
+	
+				} else {
+			LOG.warn("Unable to find timestamp in message: " + toParse);
+			ts = ParserUtils.convertToEpoch(month, day, time, true);
+		}
+
+			return ts;
+	
+	}
+
+	private JSONObject parseMessage(String toParse) {
+
+		// System.out.println("Received message: " + toParse);
+
+		// MetronMatch gm = grok.match(toParse);
+		// gm.captures();
+
+		JSONObject toReturn = new JSONObject();
+		//toParse = toParse.replaceAll("  ", " ");
+		String[] mTokens = toParse.split("\\s+");
+	 //mTokens = toParse.split(" ");
+
+		// toReturn.putAll(gm.toMap());
+
+		String id = mTokens[4];
+
+		// We are not parsing the fedata for multi part message as we cannot
+		// determine how we can split the message and how many multi part
+		// messages can there be.
+		// The message itself will be stored in the response.
+
+		String[] tokens = id.split("\\.");
+		if (tokens.length == 2) {
+
+			String[] array = Arrays.copyOfRange(mTokens, 1, mTokens.length - 1);
+			String syslog = Joiner.on(" ").join(array);
+
+			Multimap<String, String> multiMap = formatMain(syslog);
+
+			for (String key : multiMap.keySet()) {
+
+				String value = Joiner.on(",").join(multiMap.get(key));
+				toReturn.put(key, value.trim());
+			}
+
+		}
+
+		toReturn.put("original_string", toParse);
+
+		String ip_src_addr = (String) toReturn.get("dvc");
+		String ip_src_port = (String) toReturn.get("src_port");
+		String ip_dst_addr = (String) toReturn.get("dst_ip");
+		String ip_dst_port = (String) toReturn.get("dst_port");
+
+		if (ip_src_addr != null)
+			toReturn.put("ip_src_addr", ip_src_addr);
+		if (ip_src_port != null)
+			toReturn.put("ip_src_port", ip_src_port);
+		if (ip_dst_addr != null)
+			toReturn.put("ip_dst_addr", ip_dst_addr);
+		if (ip_dst_port != null)
+			toReturn.put("ip_dst_port", ip_dst_port);
+
+		System.out.println(toReturn);
+
+		return toReturn;
+	}
+
+	private Multimap<String, String> formatMain(String in) {
+		Multimap<String, String> multiMap = ArrayListMultimap.create();
+		String input = in.replaceAll("cn3", "dst_port")
+				.replaceAll("cs5", "cncHost").replaceAll("proto", "protocol")
+				.replaceAll("rt=", "timestamp=").replaceAll("cs1", "malware")
+				.replaceAll("dst=", "dst_ip=")
+				.replaceAll("shost", "src_hostname")
+				.replaceAll("dmac", "dst_mac").replaceAll("smac", "src_mac")
+				.replaceAll("spt", "src_port")
+				.replaceAll("\\bsrc\\b", "src_ip");
+		String[] tokens = input.split("\\|");
+
+		if (tokens.length > 0) {
+			String message = tokens[tokens.length - 1];
+
+			String pattern = "([\\w\\d]+)=([^=]*)(?=\\s*\\w+=|\\s*$) ";
+			Pattern p = Pattern.compile(pattern);
+			Matcher m = p.matcher(message);
+
+			while (m.find()) {
+				String[] str = m.group().split("=");
+				multiMap.put(str[0], str[1]);
+
+			}
+
+		}
+		return multiMap;
+	}
+
+	
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/0117987e/metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/interfaces/MessageFilter.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/interfaces/MessageFilter.java b/metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/interfaces/MessageFilter.java
new file mode 100644
index 0000000..2e5ab29
--- /dev/null
+++ b/metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/interfaces/MessageFilter.java
@@ -0,0 +1,24 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.metron.parsers.interfaces;
+
+public interface MessageFilter<T> {
+
+	boolean emitTuple(T message);
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/0117987e/metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/interfaces/MessageParser.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/interfaces/MessageParser.java b/metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/interfaces/MessageParser.java
new file mode 100644
index 0000000..11efa53
--- /dev/null
+++ b/metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/interfaces/MessageParser.java
@@ -0,0 +1,28 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.metron.parsers.interfaces;
+
+import java.util.List;
+
+public interface MessageParser<T> {
+
+	void init();
+	List<T> parse(byte[] rawMessage);
+	boolean validate(T message);
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/0117987e/metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/ise/BasicIseParser.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/ise/BasicIseParser.java b/metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/ise/BasicIseParser.java
new file mode 100644
index 0000000..19b3ac6
--- /dev/null
+++ b/metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/ise/BasicIseParser.java
@@ -0,0 +1,95 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+
+package org.apache.metron.parsers.ise;
+
+import com.esotericsoftware.minlog.Log;
+import org.apache.metron.parsers.BasicParser;
+import org.json.simple.JSONObject;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.StringReader;
+import java.util.ArrayList;
+import java.util.List;
+
+@SuppressWarnings("serial")
+public class BasicIseParser extends BasicParser {
+
+	private static final Logger _LOG = LoggerFactory
+			.getLogger(BasicIseParser.class);
+	static final transient ISEParser _parser = new ISEParser("header=");
+
+	@Override
+	public void init() {
+
+	}
+
+	@SuppressWarnings("unchecked")
+	@Override
+	public List<JSONObject> parse(byte[] msg) {
+	
+		String raw_message = "";
+		List<JSONObject> messages = new ArrayList<>();
+		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 Metron.
+			 */
+
+			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);
+			messages.add(payload);
+			return messages;
+
+		} catch (Exception e) {
+			Log.error(e.toString());
+			e.printStackTrace();
+		}
+		return null;
+	}
+
+	@Override
+	public boolean validate(JSONObject message) {
+		return true;
+	}
+
+	
+}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/0117987e/metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/ise/ISEParser.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/ise/ISEParser.java b/metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/ise/ISEParser.java
new file mode 100644
index 0000000..0f54261
--- /dev/null
+++ b/metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/ise/ISEParser.java
@@ -0,0 +1,660 @@
+/**
+ * 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.
+ */
+/* Generated By:JavaCC: Do not edit this line. ISEParser.java */
+package org.apache.metron.parsers.ise;
+import java.io.*;
+import java.util.*;
+import org.json.simple.*;
+
+/**
+* Basic ISE data parser generated by JavaCC. 
+*/
+public class ISEParser implements Serializable, ISEParserConstants {
+ // private boolean nativeNumbers = false;
+
+	private static final long serialVersionUID = -2531656825360044979L;
+
+	public ISEParser()
+	  { //do nothing
+	  }
+
+  public ISEParser(String input)
+  {
+    this (new StringReader(input));
+  }
+
+  /**
+	* Parses a ISE String into a JSON object {@code Map}.
+	*/
+  public JSONObject parseObject() throws ParseException
+  {
+    JSONObject toReturn = object();
+    if (!ensureEOF()) throw new IllegalStateException("Expected EOF, but still had content to parse");
+    return toReturn;
+  }
+
+  @SuppressWarnings("unused")
+final public boolean ensureEOF() throws ParseException {
+    switch (jj_nt.kind) {
+    case COMMA:
+      jj_consume_token(COMMA);
+      break;
+    default:
+      jj_la1[0] = jj_gen;
+      ;
+    }
+    jj_consume_token(0);
+    {if (true) return true;}
+    throw new Error("Missing return statement in function");
+  }
+
+  @SuppressWarnings({ "unchecked", "unused" })
+final public JSONObject innerMap() throws ParseException {
+  final JSONObject json = new JSONObject();
+  String key;
+  Object value;
+    key = objectKey();
+    jj_consume_token(EQUALS);
+    value = value();
+    json.put(key, value);
+    key = null;
+    value = null;
+    label_1:
+    while (true) {
+      switch (jj_nt.kind) {
+      case SLASH:
+        ;
+        break;
+      default:
+        jj_la1[1] = jj_gen;
+        break label_1;
+      }
+      jj_consume_token(SLASH);
+      jj_consume_token(COMMA);
+      key = objectKey();
+      jj_consume_token(EQUALS);
+      value = value();
+      json.put(key, value);
+      key = null;
+      value = null;
+    }
+    {if (true) return json;}
+    throw new Error("Missing return statement in function");
+  }
+
+  @SuppressWarnings({ "unused", "unchecked" })
+final public JSONObject object() throws ParseException {
+  final JSONObject json = new JSONObject();
+  String key;
+  Object value;
+    key = objectKey();
+    jj_consume_token(EQUALS);
+    value = value();
+    json.put(key, value);
+    key = null;
+    value = null;
+    label_2:
+    while (true) {
+      if (jj_2_1(2)) {
+        ;
+      } else {
+        break label_2;
+      }
+      jj_consume_token(COMMA);
+      key = objectKey();
+      jj_consume_token(EQUALS);
+      value = value();
+        json.put(key, value);
+        key = null;
+        value = null;
+    }
+    {if (true) return json;}
+    throw new Error("Missing return statement in function");
+  }
+
+  @SuppressWarnings("unused")
+final public String objectKey() throws ParseException {
+  String k;
+    k = string();
+    //  System.out.println("key == " + k);
+    {if (true) return k.trim();}
+    throw new Error("Missing return statement in function");
+  }
+
+  @SuppressWarnings({ "unused", "rawtypes" })
+final public Object value() throws ParseException {
+  Object x;
+  String eof = "EOF";
+  Map m = null;
+    if (jj_2_2(2147483647)) {
+      x = nullValue();
+    } else if (jj_2_3(2147483647)) {
+      x = innerMap();
+    } else {
+      switch (jj_nt.kind) {
+      case TAG:
+        x = tagString();
+        break;
+      default:
+        jj_la1[2] = jj_gen;
+        if (jj_2_4(2147483647)) {
+          x = blankValue();
+        } else if (jj_2_5(2147483647)) {
+          x = braced_string();
+        } else if (jj_2_6(2)) {
+          x = string();
+        } else {
+          jj_consume_token(-1);
+          throw new ParseException();
+        }
+      }
+    }
+    //  System.out.println("val == " + x);
+    //if (x instanceof Map) return "Map";
+    //return (String) x;
+    {if (true) return x;}
+    throw new Error("Missing return statement in function");
+  }
+
+  @SuppressWarnings("unused")
+final public String nullValue() throws ParseException {
+    {if (true) return null;}
+    throw new Error("Missing return statement in function");
+  }
+
+  @SuppressWarnings("unused")
+final public String tagString() throws ParseException {
+  String output = "(tag=0)";
+    jj_consume_token(TAG);
+    jj_consume_token(STRING_BODY);
+    {if (true) return output + token.image;}
+    throw new Error("Missing return statement in function");
+  }
+
+  @SuppressWarnings("unused")
+final public String blankValue() throws ParseException {
+    {if (true) return null;}
+    throw new Error("Missing return statement in function");
+  }
+
+  @SuppressWarnings("unused")
+final public String string() throws ParseException {
+  String s;
+    jj_consume_token(STRING_BODY);
+    {if (true) return token.image.trim();}
+    throw new Error("Missing return statement in function");
+  }
+
+  @SuppressWarnings("unused")
+final public String braced_string() throws ParseException {
+  String s;
+    jj_consume_token(BRACED_STRING);
+    //  System.out.println("braced == " + token.image);
+    s = token.image;
+    jj_consume_token(COMMA);
+    {if (true) return s.trim();}
+    throw new Error("Missing return statement in function");
+  }
+
+  private boolean jj_2_1(int xla) {
+    jj_la = xla; jj_lastpos = jj_scanpos = token;
+    try { return !jj_3_1(); }
+    catch(LookaheadSuccess ls) { return true; }
+    finally { jj_save(0, xla); }
+  }
+
+  private boolean jj_2_2(int xla) {
+    jj_la = xla; jj_lastpos = jj_scanpos = token;
+    try { return !jj_3_2(); }
+    catch(LookaheadSuccess ls) { return true; }
+    finally { jj_save(1, xla); }
+  }
+
+  private boolean jj_2_3(int xla) {
+    jj_la = xla; jj_lastpos = jj_scanpos = token;
+    try { return !jj_3_3(); }
+    catch(LookaheadSuccess ls) { return true; }
+    finally { jj_save(2, xla); }
+  }
+
+  private boolean jj_2_4(int xla) {
+    jj_la = xla; jj_lastpos = jj_scanpos = token;
+    try { return !jj_3_4(); }
+    catch(LookaheadSuccess ls) { return true; }
+    finally { jj_save(3, xla); }
+  }
+
+  private boolean jj_2_5(int xla) {
+    jj_la = xla; jj_lastpos = jj_scanpos = token;
+    try { return !jj_3_5(); }
+    catch(LookaheadSuccess ls) { return true; }
+    finally { jj_save(4, xla); }
+  }
+
+  private boolean jj_2_6(int xla) {
+    jj_la = xla; jj_lastpos = jj_scanpos = token;
+    try { return !jj_3_6(); }
+    catch(LookaheadSuccess ls) { return true; }
+    finally { jj_save(5, xla); }
+  }
+
+  private boolean jj_3_5() {
+    if (jj_3R_5()) return true;
+    return false;
+  }
+
+  private boolean jj_3_4() {
+    if (jj_scan_token(0)) return true;
+    return false;
+  }
+
+  private boolean jj_3R_5() {
+    if (jj_scan_token(BRACED_STRING)) return true;
+    if (jj_scan_token(COMMA)) return true;
+    return false;
+  }
+
+  private boolean jj_3_3() {
+    if (jj_3R_4()) return true;
+    return false;
+  }
+
+  private boolean jj_3R_4() {
+    if (jj_3R_3()) return true;
+    if (jj_scan_token(EQUALS)) return true;
+    if (jj_3R_7()) return true;
+    Token xsp;
+    while (true) {
+      xsp = jj_scanpos;
+      if (jj_3R_8()) { jj_scanpos = xsp; break; }
+    }
+    return false;
+  }
+
+  private boolean jj_3_2() {
+    if (jj_scan_token(COMMA)) return true;
+    return false;
+  }
+
+  private boolean jj_3_6() {
+    if (jj_3R_6()) return true;
+    return false;
+  }
+
+  private boolean jj_3_1() {
+    if (jj_scan_token(COMMA)) return true;
+    if (jj_3R_3()) return true;
+    return false;
+  }
+
+  private boolean jj_3R_13() {
+    if (jj_3R_5()) return true;
+    return false;
+  }
+
+  private boolean jj_3R_12() {
+    if (jj_3R_16()) return true;
+    return false;
+  }
+
+  private boolean jj_3R_11() {
+    if (jj_3R_15()) return true;
+    return false;
+  }
+
+  private boolean jj_3R_6() {
+    if (jj_scan_token(STRING_BODY)) return true;
+    return false;
+  }
+
+  private boolean jj_3R_10() {
+    if (jj_3R_4()) return true;
+    return false;
+  }
+
+  private boolean jj_3R_9() {
+    if (jj_3R_14()) return true;
+    return false;
+  }
+
+  private boolean jj_3R_7() {
+    Token xsp;
+    xsp = jj_scanpos;
+    if (jj_3R_9()) {
+    jj_scanpos = xsp;
+    if (jj_3R_10()) {
+    jj_scanpos = xsp;
+    if (jj_3R_11()) {
+    jj_scanpos = xsp;
+    if (jj_3R_12()) {
+    jj_scanpos = xsp;
+    if (jj_3R_13()) {
+    jj_scanpos = xsp;
+    if (jj_3_6()) return true;
+    }
+    }
+    }
+    }
+    }
+    return false;
+  }
+
+  private boolean jj_3R_16() {
+    return false;
+  }
+
+  private boolean jj_3R_15() {
+    if (jj_scan_token(TAG)) return true;
+    if (jj_scan_token(STRING_BODY)) return true;
+    return false;
+  }
+
+  private boolean jj_3R_3() {
+    if (jj_3R_6()) return true;
+    return false;
+  }
+
+  private boolean jj_3R_8() {
+    if (jj_scan_token(SLASH)) return true;
+    if (jj_scan_token(COMMA)) return true;
+    if (jj_3R_3()) return true;
+    if (jj_scan_token(EQUALS)) return true;
+    if (jj_3R_7()) return true;
+    return false;
+  }
+
+  private boolean jj_3R_14() {
+    return false;
+  }
+
+  /** Generated Token Manager. */
+  public ISEParserTokenManager token_source;
+  JavaCharStream jj_input_stream;
+  /** Current token. */
+  public Token token;
+  /** Next token. */
+  public Token jj_nt;
+  private Token jj_scanpos, jj_lastpos;
+  private int jj_la;
+  private int jj_gen;
+  final private int[] jj_la1 = new int[3];
+  static private int[] jj_la1_0;
+  static {
+      jj_la1_init_0();
+   }
+   private static void jj_la1_init_0() {
+      jj_la1_0 = new int[] {0x20,0x80,0x100,};
+   }
+  final private JJCalls[] jj_2_rtns = new JJCalls[6];
+  private boolean jj_rescan = false;
+  private int jj_gc = 0;
+
+  /** Constructor with InputStream. */
+  public ISEParser(java.io.InputStream stream) {
+     this(stream, null);
+  }
+  /** Constructor with InputStream and supplied encoding */
+  public ISEParser(java.io.InputStream stream, String encoding) {
+    try { jj_input_stream = new JavaCharStream(stream, encoding, 1, 1); } catch(java.io.UnsupportedEncodingException e) { throw new RuntimeException(e); }
+    token_source = new ISEParserTokenManager(jj_input_stream);
+    token = new Token();
+    token.next = jj_nt = token_source.getNextToken();
+    jj_gen = 0;
+    for (int i = 0; i < 3; i++) jj_la1[i] = -1;
+    for (int i = 0; i < jj_2_rtns.length; i++) jj_2_rtns[i] = new JJCalls();
+  }
+
+  /** Reinitialise. */
+  public void ReInit(java.io.InputStream stream) {
+     ReInit(stream, null);
+  }
+  /** Reinitialise. */
+  public void ReInit(java.io.InputStream stream, String encoding) {
+    try { jj_input_stream.ReInit(stream, encoding, 1, 1); } catch(java.io.UnsupportedEncodingException e) { throw new RuntimeException(e); }
+    token_source.ReInit(jj_input_stream);
+    token = new Token();
+    token.next = jj_nt = token_source.getNextToken();
+    jj_gen = 0;
+    for (int i = 0; i < 3; i++) jj_la1[i] = -1;
+    for (int i = 0; i < jj_2_rtns.length; i++) jj_2_rtns[i] = new JJCalls();
+  }
+
+  /** Constructor. */
+  public ISEParser(java.io.Reader stream) {
+    jj_input_stream = new JavaCharStream(stream, 1, 1);
+    token_source = new ISEParserTokenManager(jj_input_stream);
+    token = new Token();
+    token.next = jj_nt = token_source.getNextToken();
+    jj_gen = 0;
+    for (int i = 0; i < 3; i++) jj_la1[i] = -1;
+    for (int i = 0; i < jj_2_rtns.length; i++) jj_2_rtns[i] = new JJCalls();
+  }
+
+  /** Reinitialise. */
+  public void ReInit(java.io.Reader stream) {
+    jj_input_stream.ReInit(stream, 1, 1);
+    token_source.ReInit(jj_input_stream);
+    token = new Token();
+    token.next = jj_nt = token_source.getNextToken();
+    jj_gen = 0;
+    for (int i = 0; i < 3; i++) jj_la1[i] = -1;
+    for (int i = 0; i < jj_2_rtns.length; i++) jj_2_rtns[i] = new JJCalls();
+  }
+
+  /** Constructor with generated Token Manager. */
+  public ISEParser(ISEParserTokenManager tm) {
+    token_source = tm;
+    token = new Token();
+    token.next = jj_nt = token_source.getNextToken();
+    jj_gen = 0;
+    for (int i = 0; i < 3; i++) jj_la1[i] = -1;
+    for (int i = 0; i < jj_2_rtns.length; i++) jj_2_rtns[i] = new JJCalls();
+  }
+
+  /** Reinitialise. */
+  public void ReInit(ISEParserTokenManager tm) {
+    token_source = tm;
+    token = new Token();
+    token.next = jj_nt = token_source.getNextToken();
+    jj_gen = 0;
+    for (int i = 0; i < 3; i++) jj_la1[i] = -1;
+    for (int i = 0; i < jj_2_rtns.length; i++) jj_2_rtns[i] = new JJCalls();
+  }
+
+  private Token jj_consume_token(int kind) throws ParseException {
+    Token oldToken = token;
+    if ((token = jj_nt).next != null) jj_nt = jj_nt.next;
+    else jj_nt = jj_nt.next = token_source.getNextToken();
+    if (token.kind == kind) {
+      jj_gen++;
+      if (++jj_gc > 100) {
+        jj_gc = 0;
+        for (int i = 0; i < jj_2_rtns.length; i++) {
+          JJCalls c = jj_2_rtns[i];
+          while (c != null) {
+            if (c.gen < jj_gen) c.first = null;
+            c = c.next;
+          }
+        }
+      }
+      return token;
+    }
+    jj_nt = token;
+    token = oldToken;
+    jj_kind = kind;
+    throw generateParseException();
+  }
+
+  static private final class LookaheadSuccess extends java.lang.Error {
+
+	private static final long serialVersionUID = -5724812746511794505L; }
+  final private LookaheadSuccess jj_ls = new LookaheadSuccess();
+  private boolean jj_scan_token(int kind) {
+    if (jj_scanpos == jj_lastpos) {
+      jj_la--;
+      if (jj_scanpos.next == null) {
+        jj_lastpos = jj_scanpos = jj_scanpos.next = token_source.getNextToken();
+      } else {
+        jj_lastpos = jj_scanpos = jj_scanpos.next;
+      }
+    } else {
+      jj_scanpos = jj_scanpos.next;
+    }
+    if (jj_rescan) {
+      int i = 0; Token tok = token;
+      while (tok != null && tok != jj_scanpos) { i++; tok = tok.next; }
+      if (tok != null) jj_add_error_token(kind, i);
+    }
+    if (jj_scanpos.kind != kind) return true;
+    if (jj_la == 0 && jj_scanpos == jj_lastpos) throw jj_ls;
+    return false;
+  }
+
+
+/** Get the next Token. */
+  final public Token getNextToken() {
+    if ((token = jj_nt).next != null) jj_nt = jj_nt.next;
+    else jj_nt = jj_nt.next = token_source.getNextToken();
+    jj_gen++;
+    return token;
+  }
+
+/** Get the specific Token. */
+  final public Token getToken(int index) {
+    Token t = token;
+    for (int i = 0; i < index; i++) {
+      if (t.next != null) t = t.next;
+      else t = t.next = token_source.getNextToken();
+    }
+    return t;
+  }
+
+  private java.util.List<int[]> jj_expentries = new java.util.ArrayList<int[]>();
+  private int[] jj_expentry;
+  private int jj_kind = -1;
+  private int[] jj_lasttokens = new int[100];
+  private int jj_endpos;
+
+  private void jj_add_error_token(int kind, int pos) {
+    if (pos >= 100) return;
+    if (pos == jj_endpos + 1) {
+      jj_lasttokens[jj_endpos++] = kind;
+    } else if (jj_endpos != 0) {
+      jj_expentry = new int[jj_endpos];
+      for (int i = 0; i < jj_endpos; i++) {
+        jj_expentry[i] = jj_lasttokens[i];
+      }
+      jj_entries_loop: for (java.util.Iterator<?> it = jj_expentries.iterator(); it.hasNext();) {
+        int[] oldentry = (int[])(it.next());
+        if (oldentry.length == jj_expentry.length) {
+          for (int i = 0; i < jj_expentry.length; i++) {
+            if (oldentry[i] != jj_expentry[i]) {
+              continue jj_entries_loop;
+            }
+          }
+          jj_expentries.add(jj_expentry);
+          break jj_entries_loop;
+        }
+      }
+      if (pos != 0) jj_lasttokens[(jj_endpos = pos) - 1] = kind;
+    }
+  }
+
+  /** Generate ParseException. */
+  public ParseException generateParseException() {
+    jj_expentries.clear();
+    boolean[] la1tokens = new boolean[11];
+    if (jj_kind >= 0) {
+      la1tokens[jj_kind] = true;
+      jj_kind = -1;
+    }
+    for (int i = 0; i < 3; i++) {
+      if (jj_la1[i] == jj_gen) {
+        for (int j = 0; j < 32; j++) {
+          if ((jj_la1_0[i] & (1<<j)) != 0) {
+            la1tokens[j] = true;
+          }
+        }
+      }
+    }
+    for (int i = 0; i < 11; i++) {
+      if (la1tokens[i]) {
+        jj_expentry = new int[1];
+        jj_expentry[0] = i;
+        jj_expentries.add(jj_expentry);
+      }
+    }
+    jj_endpos = 0;
+    jj_rescan_token();
+    jj_add_error_token(0, 0);
+    int[][] exptokseq = new int[jj_expentries.size()][];
+    for (int i = 0; i < jj_expentries.size(); i++) {
+      exptokseq[i] = jj_expentries.get(i);
+    }
+    return new ParseException(token, exptokseq, tokenImage);
+  }
+
+  /** Enable tracing. */
+  final public void enable_tracing() {
+  }
+
+  /** Disable tracing. */
+  final public void disable_tracing() {
+  }
+
+  private void jj_rescan_token() {
+    jj_rescan = true;
+    for (int i = 0; i < 6; i++) {
+    try {
+      JJCalls p = jj_2_rtns[i];
+      do {
+        if (p.gen > jj_gen) {
+          jj_la = p.arg; jj_lastpos = jj_scanpos = p.first;
+          switch (i) {
+            case 0: jj_3_1(); break;
+            case 1: jj_3_2(); break;
+            case 2: jj_3_3(); break;
+            case 3: jj_3_4(); break;
+            case 4: jj_3_5(); break;
+            case 5: jj_3_6(); break;
+          }
+        }
+        p = p.next;
+      } while (p != null);
+      } catch(LookaheadSuccess ls) { }
+    }
+    jj_rescan = false;
+  }
+
+  private void jj_save(int index, int xla) {
+    JJCalls p = jj_2_rtns[index];
+    while (p.gen > jj_gen) {
+      if (p.next == null) { p = p.next = new JJCalls(); break; }
+      p = p.next;
+    }
+    p.gen = jj_gen + xla - jj_la; p.first = token; p.arg = xla;
+  }
+
+  static final class JJCalls {
+    int gen;
+    Token first;
+    int arg;
+    JJCalls next;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/0117987e/metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/ise/ISEParser.jj
----------------------------------------------------------------------
diff --git a/metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/ise/ISEParser.jj b/metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/ise/ISEParser.jj
new file mode 100644
index 0000000..6071922
--- /dev/null
+++ b/metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/ise/ISEParser.jj
@@ -0,0 +1,12 @@
+options{  CHOICE_AMBIGUITY_CHECK = 3;  OTHER_AMBIGUITY_CHECK = 2;  //DEBUG_PARSER=true
+  //DEBUG_LOOKAHEAD=true
+  //DEBUG_TOKEN_MANAGER=true
+  ERROR_REPORTING = true;  JAVA_UNICODE_ESCAPE = true;  UNICODE_INPUT = true;  IGNORE_CASE = true;  SUPPORT_CLASS_VISIBILITY_PUBLIC = false;  FORCE_LA_CHECK = true;  CACHE_TOKENS = true;  SANITY_CHECK = true;  STATIC = false;  //KEEP_LINE_COLUMN=true;
+}PARSER_BEGIN(ISEParser)/** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements.  See the NOTICE file * distributed with this work for additional information * regarding copyright ownership.  The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License.  You may obtain a copy of the License at * *     http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */package org.apache.metron.ise.parser;import java.io.*;import java.util.*;import org.json.simple.*;/**
+* Basic ISE data parser generated by JavaCC.
+*/public class ISEParser implements Serializable{  private boolean nativeNumbers = false;  public ISEParser()  { //do nothing
+  }  public ISEParser(String input)  {    this (new StringReader(input));  }  /**
+	* Parses a ISE String into a JSON object {@code Map}.
+	*/  public JSONObject parseObject() throws ParseException  {    JSONObject toReturn = object();    if (!ensureEOF()) throw new IllegalStateException("Expected EOF, but still had content to parse");    return toReturn;  }}PARSER_END(ISEParser)// Ignore commentsSKIP :{  < C_SINGLE_COMMENT : "//" (~[ "\n", "\r", "\f" ])* < EOL >>| < C_MULTILINE_COMMENT : "/*" (~[ ])* "*/" >| < SH_SINGLE_COMMENT : "#" (~[ "\n", "\r", "\f" ])* < EOL >>  /*| < WHITESPACE :    " "  | "\t" >*/| < EOL :    "\n"  | "\r"  | "\f" >}// Common tokens
+TOKEN :{  < COMMA : "," >| < EQUALS : "=" >| < SLASH : "\\" >| < TAG : "(tag=0)" >}// Null token/*TOKEN :{  //< NULL : "null" >}*/// String tokens
+TOKEN :{  //< SYMBOL : ([ "a"-"z", "A"-"Z", "0", "1"-"9", " ", "\t" , ":" , "-" , "." ])+ >  < STRING_BODY :    (      (~[ "\"", "\r", "\n", "\f", "\t", "=", "," ])    |      (        "\\"        (          "r"        | "n"        | "f"        | "\\"        | "/"        | "\""        | "b"        | "t"        | ","        )      )    )+ >| < BRACED_STRING :    (      "{" (~[ "{", "}" ])+ "}"    ) >}boolean ensureEOF() :{}{  (< COMMA >)? < EOF >  {    return true;  }}JSONObject innerMap() :{  final JSONObject json = new JSONObject();  String key;  Object value;}{  key = objectKey() < EQUALS > value = value()  {    json.put(key, value);  }  {    key = null;    value = null;  }  (    < SLASH > < COMMA > key = objectKey() < EQUALS > value = value()    {      json.put(key, value);    }    {      key = null;      value = null;    }  )*  {    return json;  }}JSONObject object() :{  final JSONObject json = new JSONObject()
 ;  String key;  Object value;}{  key = objectKey() < EQUALS > value = value()  {    json.put(key, value);  }  {    key = null;    value = null;  }  (    (      LOOKAHEAD(2)      < COMMA > key = objectKey() < EQUALS > value = value()      {        json.put(key, value);      }      {        key = null;        value = null;      }    )*  | LOOKAHEAD(2)    < COMMA > < EOF >  )  // ensureEOF()  {    return json;  }}String objectKey() :{  String k;}{  (    k = string()  )  {    //  System.out.println("key == " + k);    return k.trim();  }}Object value() :{  Object x;  String eof = "EOF";  Map m = null;}{  (    LOOKAHEAD(< COMMA >)    x = nullValue()  | LOOKAHEAD(innerMap())    x = innerMap()  | x = tagString()  | LOOKAHEAD(< EOF >)    x = blankValue()  | LOOKAHEAD(braced_string())    x = braced_string()  | LOOKAHEAD(2)    x = string()  )  {    //  System.out.println("val == " + x);    //if (x instanceof Map) return "Map
 ";    //return (String) x;    return x;  }}String nullValue() :{}{  {    return null;  }}String tagString() :{  String output = "(tag=0)";}{  < TAG > < STRING_BODY >  {    return output + token.image;  }}String blankValue() :{}{  {    return null;  }}String string() :{  String s;}{  < STRING_BODY >  {    return token.image.trim();  }}String braced_string() :{  String s;}{  < BRACED_STRING >  {    //  System.out.println("braced == " + token.image);    s = token.image;  }  < COMMA >  {    return s.trim();  }}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/0117987e/metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/ise/ISEParserConstants.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/ise/ISEParserConstants.java b/metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/ise/ISEParserConstants.java
new file mode 100644
index 0000000..126d120
--- /dev/null
+++ b/metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/ise/ISEParserConstants.java
@@ -0,0 +1,69 @@
+/**
+ * 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.
+ */
+/* Generated By:JavaCC: Do not edit this line. ISEParserConstants.java */
+package org.apache.metron.parsers.ise;
+
+
+/**
+ * Token literal values and constants.
+ * Generated by org.javacc.parser.OtherFilesGen#start()
+ */
+interface ISEParserConstants {
+
+  /** End of File. */
+  int EOF = 0;
+  /** RegularExpression Id. */
+  int C_SINGLE_COMMENT = 1;
+  /** RegularExpression Id. */
+  int C_MULTILINE_COMMENT = 2;
+  /** RegularExpression Id. */
+  int SH_SINGLE_COMMENT = 3;
+  /** RegularExpression Id. */
+  int EOL = 4;
+  /** RegularExpression Id. */
+  int COMMA = 5;
+  /** RegularExpression Id. */
+  int EQUALS = 6;
+  /** RegularExpression Id. */
+  int SLASH = 7;
+  /** RegularExpression Id. */
+  int TAG = 8;
+  /** RegularExpression Id. */
+  int STRING_BODY = 9;
+  /** RegularExpression Id. */
+  int BRACED_STRING = 10;
+
+  /** Lexical state. */
+  int DEFAULT = 0;
+
+  /** Literal token values. */
+  String[] tokenImage = {
+    "<EOF>",
+    "<C_SINGLE_COMMENT>",
+    "<C_MULTILINE_COMMENT>",
+    "<SH_SINGLE_COMMENT>",
+    "<EOL>",
+    "\",\"",
+    "\"=\"",
+    "\"\\\\\"",
+    "\"(tag=0)\"",
+    "<STRING_BODY>",
+    "<BRACED_STRING>",
+  };
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/0117987e/metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/ise/ISEParserTokenManager.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/ise/ISEParserTokenManager.java b/metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/ise/ISEParserTokenManager.java
new file mode 100644
index 0000000..9bd5347
--- /dev/null
+++ b/metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/ise/ISEParserTokenManager.java
@@ -0,0 +1,676 @@
+/**
+ * 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.
+ */
+/* Generated By:JavaCC: Do not edit this line. ISEParserTokenManager.java */
+package org.apache.metron.parsers.ise;
+
+/** Token Manager. */
+class ISEParserTokenManager implements ISEParserConstants
+{
+
+  /** Debug output. */
+  public  java.io.PrintStream debugStream = System.out;
+  /** Set debug output. */
+  public  void setDebugStream(java.io.PrintStream ds) { debugStream = ds; }
+private final int jjStopStringLiteralDfa_0(int pos, long active0)
+{
+   switch (pos)
+   {
+      case 0:
+         if ((active0 & 0x100L) != 0L)
+         {
+            jjmatchedKind = 9;
+            return 18;
+         }
+         if ((active0 & 0x80L) != 0L)
+            return 6;
+         return -1;
+      case 1:
+         if ((active0 & 0x100L) != 0L)
+         {
+            jjmatchedKind = 9;
+            jjmatchedPos = 1;
+            return 18;
+         }
+         return -1;
+      case 2:
+         if ((active0 & 0x100L) != 0L)
+         {
+            jjmatchedKind = 9;
+            jjmatchedPos = 2;
+            return 18;
+         }
+         return -1;
+      case 3:
+         if ((active0 & 0x100L) != 0L)
+         {
+            jjmatchedKind = 9;
+            jjmatchedPos = 3;
+            return 18;
+         }
+         return -1;
+      case 4:
+         if ((active0 & 0x100L) != 0L)
+         {
+            if (jjmatchedPos < 3)
+            {
+               jjmatchedKind = 9;
+               jjmatchedPos = 3;
+            }
+            return -1;
+         }
+         return -1;
+      case 5:
+         if ((active0 & 0x100L) != 0L)
+         {
+            if (jjmatchedPos < 3)
+            {
+               jjmatchedKind = 9;
+               jjmatchedPos = 3;
+            }
+            return -1;
+         }
+         return -1;
+      default :
+         return -1;
+   }
+}
+private final int jjStartNfa_0(int pos, long active0)
+{
+   return jjMoveNfa_0(jjStopStringLiteralDfa_0(pos, active0), pos + 1);
+}
+private int jjStopAtPos(int pos, int kind)
+{
+   jjmatchedKind = kind;
+   jjmatchedPos = pos;
+   return pos + 1;
+}
+private int jjMoveStringLiteralDfa0_0()
+{
+   switch(curChar)
+   {
+      case 40:
+         return jjMoveStringLiteralDfa1_0(0x100L);
+      case 44:
+         return jjStopAtPos(0, 5);
+      case 61:
+         return jjStopAtPos(0, 6);
+      case 92:
+         return jjStartNfaWithStates_0(0, 7, 6);
+      default :
+         return jjMoveNfa_0(0, 0);
+   }
+}
+private int jjMoveStringLiteralDfa1_0(long active0)
+{
+   try { curChar = input_stream.readChar(); }
+   catch(java.io.IOException e) {
+      jjStopStringLiteralDfa_0(0, active0);
+      return 1;
+   }
+   switch(curChar)
+   {
+      case 84:
+      case 116:
+         return jjMoveStringLiteralDfa2_0(active0, 0x100L);
+      default :
+         break;
+   }
+   return jjStartNfa_0(0, active0);
+}
+private int jjMoveStringLiteralDfa2_0(long old0, long active0)
+{
+   if (((active0 &= old0)) == 0L)
+      return jjStartNfa_0(0, old0);
+   try { curChar = input_stream.readChar(); }
+   catch(java.io.IOException e) {
+      jjStopStringLiteralDfa_0(1, active0);
+      return 2;
+   }
+   switch(curChar)
+   {
+      case 65:
+      case 97:
+         return jjMoveStringLiteralDfa3_0(active0, 0x100L);
+      default :
+         break;
+   }
+   return jjStartNfa_0(1, active0);
+}
+private int jjMoveStringLiteralDfa3_0(long old0, long active0)
+{
+   if (((active0 &= old0)) == 0L)
+      return jjStartNfa_0(1, old0);
+   try { curChar = input_stream.readChar(); }
+   catch(java.io.IOException e) {
+      jjStopStringLiteralDfa_0(2, active0);
+      return 3;
+   }
+   switch(curChar)
+   {
+      case 71:
+      case 103:
+         return jjMoveStringLiteralDfa4_0(active0, 0x100L);
+      default :
+         break;
+   }
+   return jjStartNfa_0(2, active0);
+}
+private int jjMoveStringLiteralDfa4_0(long old0, long active0)
+{
+   if (((active0 &= old0)) == 0L)
+      return jjStartNfa_0(2, old0);
+   try { curChar = input_stream.readChar(); }
+   catch(java.io.IOException e) {
+      jjStopStringLiteralDfa_0(3, active0);
+      return 4;
+   }
+   switch(curChar)
+   {
+      case 61:
+         return jjMoveStringLiteralDfa5_0(active0, 0x100L);
+      default :
+         break;
+   }
+   return jjStartNfa_0(3, active0);
+}
+private int jjMoveStringLiteralDfa5_0(long old0, long active0)
+{
+   if (((active0 &= old0)) == 0L)
+      return jjStartNfa_0(3, old0);
+   try { curChar = input_stream.readChar(); }
+   catch(java.io.IOException e) {
+      jjStopStringLiteralDfa_0(4, active0);
+      return 5;
+   }
+   switch(curChar)
+   {
+      case 48:
+         return jjMoveStringLiteralDfa6_0(active0, 0x100L);
+      default :
+         break;
+   }
+   return jjStartNfa_0(4, active0);
+}
+private int jjMoveStringLiteralDfa6_0(long old0, long active0)
+{
+   if (((active0 &= old0)) == 0L)
+      return jjStartNfa_0(4, old0);
+   try { curChar = input_stream.readChar(); }
+   catch(java.io.IOException e) {
+      jjStopStringLiteralDfa_0(5, active0);
+      return 6;
+   }
+   switch(curChar)
+   {
+      case 41:
+         if ((active0 & 0x100L) != 0L)
+            return jjStopAtPos(6, 8);
+         break;
+      default :
+         break;
+   }
+   return jjStartNfa_0(5, active0);
+}
+private int jjStartNfaWithStates_0(int pos, int kind, int state)
+{
+   jjmatchedKind = kind;
+   jjmatchedPos = pos;
+   try { curChar = input_stream.readChar(); }
+   catch(java.io.IOException e) { return pos + 1; }
+   return jjMoveNfa_0(state, pos + 1);
+}
+static final long[] jjbitVec0 = {
+   0xfffffffffffffffeL, 0xffffffffffffffffL, 0xffffffffffffffffL, 0xffffffffffffffffL
+};
+static final long[] jjbitVec2 = {
+   0x0L, 0x0L, 0xffffffffffffffffL, 0xffffffffffffffffL
+};
+private int jjMoveNfa_0(int startState, int curPos)
+{
+   int startsAt = 0;
+   jjnewStateCnt = 18;
+   int i = 1;
+   jjstateSet[0] = startState;
+   int kind = 0x7fffffff;
+   for (;;)
+   {
+      if (++jjround == 0x7fffffff)
+         ReInitRounds();
+      if (curChar < 64)
+      {
+         long l = 1L << curChar;
+         do
+         {
+            switch(jjstateSet[--i])
+            {
+               case 18:
+               case 4:
+                  if ((0xdfffeffbffffc9ffL & l) == 0L)
+                     break;
+                  if (kind > 9)
+                     kind = 9;
+                  jjCheckNAddTwoStates(4, 5);
+                  break;
+               case 0:
+                  if ((0xdfffeffbffffc9ffL & l) != 0L)
+                  {
+                     if (kind > 9)
+                        kind = 9;
+                     jjCheckNAddTwoStates(4, 5);
+                  }
+                  else if ((0x3400L & l) != 0L)
+                  {
+                     if (kind > 4)
+                        kind = 4;
+                  }
+                  if (curChar == 47)
+                     jjAddStates(0, 1);
+                  else if (curChar == 35)
+                     jjCheckNAddTwoStates(1, 2);
+                  break;
+               case 6:
+                  if ((0xdfffeffbffffc9ffL & l) != 0L)
+                  {
+                     if (kind > 9)
+                        kind = 9;
+                     jjCheckNAddTwoStates(4, 5);
+                  }
+                  if ((0x900400000000L & l) != 0L)
+                  {
+                     if (kind > 9)
+                        kind = 9;
+                     jjCheckNAddTwoStates(4, 5);
+                  }
+                  break;
+               case 1:
+                  if ((0xffffffffffffcbffL & l) != 0L)
+                     jjCheckNAddTwoStates(1, 2);
+                  break;
+               case 2:
+                  if ((0x3400L & l) != 0L && kind > 3)
+                     kind = 3;
+                  break;
+               case 3:
+                  if ((0x3400L & l) != 0L && kind > 4)
+                     kind = 4;
+                  break;
+               case 8:
+                  jjAddStates(2, 3);
+                  break;
+               case 10:
+                  if (curChar == 47)
+                     jjAddStates(0, 1);
+                  break;
+               case 11:
+                  if (curChar == 47)
+                     jjCheckNAddTwoStates(12, 13);
+                  break;
+               case 12:
+                  if ((0xffffffffffffcbffL & l) != 0L)
+                     jjCheckNAddTwoStates(12, 13);
+                  break;
+               case 13:
+                  if ((0x3400L & l) != 0L && kind > 1)
+                     kind = 1;
+                  break;
+               case 14:
+                  if (curChar == 42)
+                     jjCheckNAddTwoStates(15, 17);
+                  break;
+               case 15:
+                  jjCheckNAddTwoStates(15, 17);
+                  break;
+               case 16:
+                  if (curChar == 47 && kind > 2)
+                     kind = 2;
+                  break;
+               case 17:
+                  if (curChar == 42)
+                     jjstateSet[jjnewStateCnt++] = 16;
+                  break;
+               default : break;
+            }
+         } while(i != startsAt);
+      }
+      else if (curChar < 128)
+      {
+         long l = 1L << (curChar & 077);
+         do
+         {
+            switch(jjstateSet[--i])
+            {
+               case 18:
+                  if (kind > 9)
+                     kind = 9;
+                  jjCheckNAddTwoStates(4, 5);
+                  if (curChar == 92)
+                     jjstateSet[jjnewStateCnt++] = 6;
+                  break;
+               case 0:
+                  if (kind > 9)
+                     kind = 9;
+                  jjCheckNAddTwoStates(4, 5);
+                  if (curChar == 123)
+                     jjCheckNAdd(8);
+                  else if (curChar == 92)
+                     jjstateSet[jjnewStateCnt++] = 6;
+                  break;
+               case 6:
+                  if (kind > 9)
+                     kind = 9;
+                  jjCheckNAddTwoStates(4, 5);
+                  if ((0x14404410144044L & l) != 0L)
+                  {
+                     if (kind > 9)
+                        kind = 9;
+                     jjCheckNAddTwoStates(4, 5);
+                  }
+                  if (curChar == 92)
+                     jjstateSet[jjnewStateCnt++] = 6;
+                  break;
+               case 1:
+                  jjAddStates(4, 5);
+                  break;
+               case 4:
+                  if (kind > 9)
+                     kind = 9;
+                  jjCheckNAddTwoStates(4, 5);
+                  break;
+               case 5:
+                  if (curChar == 92)
+                     jjstateSet[jjnewStateCnt++] = 6;
+                  break;
+               case 7:
+                  if (curChar == 123)
+                     jjCheckNAdd(8);
+                  break;
+               case 8:
+                  if ((0xd7ffffffffffffffL & l) != 0L)
+                     jjCheckNAddTwoStates(8, 9);
+                  break;
+               case 9:
+                  if (curChar == 125 && kind > 10)
+                     kind = 10;
+                  break;
+               case 12:
+                  jjAddStates(6, 7);
+                  break;
+               case 15:
+                  jjAddStates(8, 9);
+                  break;
+               default : break;
+            }
+         } while(i != startsAt);
+      }
+      else
+      {
+         int hiByte = (int)(curChar >> 8);
+         int i1 = hiByte >> 6;
+         long l1 = 1L << (hiByte & 077);
+         int i2 = (curChar & 0xff) >> 6;
+         long l2 = 1L << (curChar & 077);
+         do
+         {
+            switch(jjstateSet[--i])
+            {
+               case 18:
+               case 4:
+                  if (!jjCanMove_0(hiByte, i1, i2, l1, l2))
+                     break;
+                  if (kind > 9)
+                     kind = 9;
+                  jjCheckNAddTwoStates(4, 5);
+                  break;
+               case 0:
+                  if (!jjCanMove_0(hiByte, i1, i2, l1, l2))
+                     break;
+                  if (kind > 9)
+                     kind = 9;
+                  jjCheckNAddTwoStates(4, 5);
+                  break;
+               case 6:
+                  if (!jjCanMove_0(hiByte, i1, i2, l1, l2))
+                     break;
+                  if (kind > 9)
+                     kind = 9;
+                  jjCheckNAddTwoStates(4, 5);
+                  break;
+               case 1:
+                  if (jjCanMove_0(hiByte, i1, i2, l1, l2))
+                     jjAddStates(4, 5);
+                  break;
+               case 8:
+                  if (jjCanMove_0(hiByte, i1, i2, l1, l2))
+                     jjAddStates(2, 3);
+                  break;
+               case 12:
+                  if (jjCanMove_0(hiByte, i1, i2, l1, l2))
+                     jjAddStates(6, 7);
+                  break;
+               case 15:
+                  if (jjCanMove_0(hiByte, i1, i2, l1, l2))
+                     jjAddStates(8, 9);
+                  break;
+               default : break;
+            }
+         } while(i != startsAt);
+      }
+      if (kind != 0x7fffffff)
+      {
+         jjmatchedKind = kind;
+         jjmatchedPos = curPos;
+         kind = 0x7fffffff;
+      }
+      ++curPos;
+      if ((i = jjnewStateCnt) == (startsAt = 18 - (jjnewStateCnt = startsAt)))
+         return curPos;
+      try { curChar = input_stream.readChar(); }
+      catch(java.io.IOException e) { return curPos; }
+   }
+}
+static final int[] jjnextStates = {
+   11, 14, 8, 9, 1, 2, 12, 13, 15, 17, 
+};
+private static final boolean jjCanMove_0(int hiByte, int i1, int i2, long l1, long l2)
+{
+   switch(hiByte)
+   {
+      case 0:
+         return ((jjbitVec2[i2] & l2) != 0L);
+      default :
+         if ((jjbitVec0[i1] & l1) != 0L)
+            return true;
+         return false;
+   }
+}
+
+/** Token literal values. */
+public static final String[] jjstrLiteralImages = {
+"", null, null, null, null, "\54", "\75", "\134", null, null, null, };
+
+/** Lexer state names. */
+public static final String[] lexStateNames = {
+   "DEFAULT",
+};
+static final long[] jjtoToken = {
+   0x7e1L, 
+};
+static final long[] jjtoSkip = {
+   0x1eL, 
+};
+protected JavaCharStream input_stream;
+private final int[] jjrounds = new int[18];
+private final int[] jjstateSet = new int[36];
+protected char curChar;
+/** Constructor. */
+public ISEParserTokenManager(JavaCharStream stream){
+   if (JavaCharStream.staticFlag)
+      throw new Error("ERROR: Cannot use a static CharStream class with a non-static lexical analyzer.");
+   input_stream = stream;
+}
+
+/** Constructor. */
+public ISEParserTokenManager(JavaCharStream stream, int lexState){
+   this(stream);
+   SwitchTo(lexState);
+}
+
+/** Reinitialise parser. */
+public void ReInit(JavaCharStream stream)
+{
+   jjmatchedPos = jjnewStateCnt = 0;
+   curLexState = defaultLexState;
+   input_stream = stream;
+   ReInitRounds();
+}
+private void ReInitRounds()
+{
+   int i;
+   jjround = 0x80000001;
+   for (i = 18; i-- > 0;)
+      jjrounds[i] = 0x80000000;
+}
+
+/** Reinitialise parser. */
+public void ReInit(JavaCharStream stream, int lexState)
+{
+   ReInit(stream);
+   SwitchTo(lexState);
+}
+
+/** Switch to specified lex state. */
+public void SwitchTo(int lexState)
+{
+   if (lexState >= 1 || lexState < 0)
+      throw new TokenMgrError("Error: Ignoring invalid lexical state : " + lexState + ". State unchanged.", TokenMgrError.INVALID_LEXICAL_STATE);
+   else
+      curLexState = lexState;
+}
+
+protected Token jjFillToken()
+{
+   final Token t;
+   final String curTokenImage;
+   final int beginLine;
+   final int endLine;
+   final int beginColumn;
+   final int endColumn;
+   String im = jjstrLiteralImages[jjmatchedKind];
+   curTokenImage = (im == null) ? input_stream.GetImage() : im;
+   beginLine = input_stream.getBeginLine();
+   beginColumn = input_stream.getBeginColumn();
+   endLine = input_stream.getEndLine();
+   endColumn = input_stream.getEndColumn();
+   t = Token.newToken(jjmatchedKind, curTokenImage);
+
+   t.beginLine = beginLine;
+   t.endLine = endLine;
+   t.beginColumn = beginColumn;
+   t.endColumn = endColumn;
+
+   return t;
+}
+
+int curLexState = 0;
+int defaultLexState = 0;
+int jjnewStateCnt;
+int jjround;
+int jjmatchedPos;
+int jjmatchedKind;
+
+/** Get the next Token. */
+public Token getNextToken() 
+{
+  Token matchedToken;
+  int curPos = 0;
+
+  EOFLoop :
+  for (;;)
+  {
+   try
+   {
+      curChar = input_stream.BeginToken();
+   }
+   catch(java.io.IOException e)
+   {
+      jjmatchedKind = 0;
+      matchedToken = jjFillToken();
+      return matchedToken;
+   }
+
+   jjmatchedKind = 0x7fffffff;
+   jjmatchedPos = 0;
+   curPos = jjMoveStringLiteralDfa0_0();
+   if (jjmatchedKind != 0x7fffffff)
+   {
+      if (jjmatchedPos + 1 < curPos)
+         input_stream.backup(curPos - jjmatchedPos - 1);
+      if ((jjtoToken[jjmatchedKind >> 6] & (1L << (jjmatchedKind & 077))) != 0L)
+      {
+         matchedToken = jjFillToken();
+         return matchedToken;
+      }
+      else
+      {
+         continue EOFLoop;
+      }
+   }
+   int error_line = input_stream.getEndLine();
+   int error_column = input_stream.getEndColumn();
+   String error_after = null;
+   boolean EOFSeen = false;
+   try { input_stream.readChar(); input_stream.backup(1); }
+   catch (java.io.IOException e1) {
+      EOFSeen = true;
+      error_after = curPos <= 1 ? "" : input_stream.GetImage();
+      if (curChar == '\n' || curChar == '\r') {
+         error_line++;
+         error_column = 0;
+      }
+      else
+         error_column++;
+   }
+   if (!EOFSeen) {
+      input_stream.backup(1);
+      error_after = curPos <= 1 ? "" : input_stream.GetImage();
+   }
+   throw new TokenMgrError(EOFSeen, curLexState, error_line, error_column, error_after, curChar, TokenMgrError.LEXICAL_ERROR);
+  }
+}
+
+private void jjCheckNAdd(int state)
+{
+   if (jjrounds[state] != jjround)
+   {
+      jjstateSet[jjnewStateCnt++] = state;
+      jjrounds[state] = jjround;
+   }
+}
+private void jjAddStates(int start, int end)
+{
+   do {
+      jjstateSet[jjnewStateCnt++] = jjnextStates[start];
+   } while (start++ != end);
+}
+private void jjCheckNAddTwoStates(int state1, int state2)
+{
+   jjCheckNAdd(state1);
+   jjCheckNAdd(state2);
+}
+
+}