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/05/02 19:26:42 UTC

[1/3] incubator-metron git commit: METRON-122 Create generic unit test framework for testing grok statements (merrimanr) closes apache/incubator-metron#96

Repository: incubator-metron
Updated Branches:
  refs/heads/master 28c250d15 -> 645d8292b


http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/645d8292/metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/GrokParser.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/GrokParser.java b/metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/GrokParser.java
index 6ebc36b..39a88bb 100644
--- a/metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/GrokParser.java
+++ b/metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/GrokParser.java
@@ -17,6 +17,8 @@
  */
 package org.apache.metron.parsers;
 
+import com.google.common.base.Joiner;
+import com.google.common.base.Splitter;
 import oi.thekraken.grok.api.Grok;
 import oi.thekraken.grok.api.Match;
 import org.apache.hadoop.conf.Configuration;
@@ -43,25 +45,19 @@ public class GrokParser implements MessageParser<JSONObject>, Serializable {
 
   protected static final Logger LOG = LoggerFactory.getLogger(GrokParser.class);
 
-  private transient Grok grok;
-  private String grokHdfsPath;
-  private String patternLabel;
-  private String[] timeFields = new String[0];
-  private String timestampField;
-  private String dateFormat = "yyyy-MM-dd HH:mm:ss.S z";
-  private TimeZone timeZone = TimeZone.getTimeZone("UTC");
+  protected transient Grok grok;
+  protected String grokHdfsPath;
+  protected String patternLabel;
+  protected String[] timeFields = new String[0];
+  protected String timestampField;
+  protected String dateFormat = "yyyy-MM-dd HH:mm:ss.S z";
+  protected TimeZone timeZone = TimeZone.getTimeZone("UTC");
 
-  private String metronHdfsHome = "/apps/metron";
   public GrokParser(String grokHdfsPath, String patterLabel) {
     this.grokHdfsPath = grokHdfsPath;
     this.patternLabel = patterLabel;
   }
 
-  public GrokParser withMetronHDFSHome(String home) {
-    this.metronHdfsHome= home;
-    return this;
-  }
-
   public GrokParser withTimestampField(String timestampField) {
     this.timestampField = timestampField;
     return this;
@@ -86,7 +82,7 @@ public class GrokParser implements MessageParser<JSONObject>, Serializable {
     InputStream is = getClass().getResourceAsStream(streamName);
     if(is == null) {
       FileSystem fs = FileSystem.get(new Configuration());
-      Path path = new Path((metronHdfsHome != null && metronHdfsHome.length() > 0?metronHdfsHome + "/":"") + streamName);
+      Path path = new Path(streamName);
       if(fs.exists(path)) {
         return fs.open(path);
       }
@@ -134,7 +130,7 @@ public class GrokParser implements MessageParser<JSONObject>, Serializable {
         }
       }
       if (timestampField != null) {
-        message.put(Constants.Fields.TIMESTAMP.getName(), message.get(timestampField));
+        message.put(Constants.Fields.TIMESTAMP.getName(), formatTimestamp(message.get(timestampField)));
       }
       message.remove(patternLabel);
       messages.add(message);
@@ -164,4 +160,15 @@ public class GrokParser implements MessageParser<JSONObject>, Serializable {
     return date.getTime();
   }
 
+  protected long formatTimestamp(Object value) {
+    if (value == null) {
+      throw new RuntimeException(patternLabel + " pattern does not include field " + timestampField);
+    }
+    if (value instanceof Number) {
+      return ((Number) value).longValue();
+    } else {
+      return Long.parseLong(Joiner.on("").join(Splitter.on('.').split(value + "")));
+    }
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/645d8292/metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/bluecoat/BasicBluecoatParser.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/bluecoat/BasicBluecoatParser.java b/metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/bluecoat/BasicBluecoatParser.java
new file mode 100644
index 0000000..9d9ab6b
--- /dev/null
+++ b/metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/bluecoat/BasicBluecoatParser.java
@@ -0,0 +1,101 @@
+/**
+ * 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.bluecoat;
+
+import java.text.SimpleDateFormat;
+import java.util.ArrayList;
+import java.util.Calendar;
+import java.util.Date;
+import java.util.List;
+
+import org.apache.metron.parsers.BasicParser;
+import org.json.simple.JSONObject;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings("serial")
+public class BasicBluecoatParser extends BasicParser {
+
+	private static final Logger _LOG = LoggerFactory.getLogger(BasicBluecoatParser.class);
+	private SimpleDateFormat df = new SimpleDateFormat("MMM dd yyyy HH:mm:ss");
+	
+	@Override
+	public void init() {
+
+	}
+
+	@SuppressWarnings({ "unchecked", "unused" })
+	public List<JSONObject> parse(byte[] msg) {
+
+		String message = "";
+		List<JSONObject> messages = new ArrayList<>();
+		JSONObject payload = new JSONObject();
+		
+		try {
+			message = new String(msg, "UTF-8");
+			
+			
+			String[] parts = message.split("<|>|\\(|\\)| ");
+			payload.put("original_string", message);
+			payload.put("priority", parts[1]);
+			
+			int year = Calendar.getInstance().get(Calendar.YEAR);
+			Date date = df.parse(parts[2] + " " + parts[3] + " " + year + " "+ parts[4]);
+		    long epoch = date.getTime();
+		    	    
+			payload.put("timestamp", epoch);
+			payload.put("event_code", parts[6]);
+			
+			if(parts[6].equals("250017"))
+			{
+				payload.put("event_type", "authentication failure");
+				payload.put("eid", parts[12].substring(1, parts[12].length()-1));
+				payload.put("designated_host", parts[10].substring(0, parts[10].length()-1));
+				payload.put("realm", parts[15]);
+				payload.put("message", message.split("250017")[1]);
+			}
+			else if(parts[6].equals("250018"))
+			{
+				payload.put("event_type", "authentication failure");
+				payload.put("realm", parts[24].substring(1, parts[24].length()-1));
+				payload.put("message", message.split("250018")[1]);
+			}
+			else if(parts[6].equals("E0000"))
+			{
+				payload.put("event_type", "authentication");
+				payload.put("message", message.split("E0000")[1]);
+			}
+			else if(parts[6].equals("250001"))
+			{
+				payload.put("event_type", "authentication failure");
+				payload.put("designated_host", parts[11].substring(0, parts[11].length()-1));
+				payload.put("realm", parts[17].substring(1, parts[17].length()-1));
+				payload.put("message", message.split("250001")[1]);
+			}
+			
+			messages.add(payload);
+			return messages;
+		} catch (Exception e) {
+			e.printStackTrace();
+			_LOG.error("Failed to parse: " + message);
+			return null;
+		}
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/645d8292/metron-platform/metron-parsers/src/main/resources/patterns/squid
----------------------------------------------------------------------
diff --git a/metron-platform/metron-parsers/src/main/resources/patterns/squid b/metron-platform/metron-parsers/src/main/resources/patterns/squid
new file mode 100644
index 0000000..ba5bab8
--- /dev/null
+++ b/metron-platform/metron-parsers/src/main/resources/patterns/squid
@@ -0,0 +1,2 @@
+WDOM [^(?:http:\/\/|www\.|https:\/\/)]([^\/]+)
+SQUID_DELIMITED %{NUMBER:timestamp} %{SPACE:UNWANTED}  %{INT:elapsed} %{IPV4:ip_src_addr} %{WORD:action}/%{NUMBER:code} %{NUMBER:bytes} %{WORD:method} http:\/\/\www.%{WDOM:url}\/ - %{WORD:UNWANTED}\/%{IPV4:ip_dst_addr} %{WORD:UNWANTED}\/%{WORD:UNWANTED}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/645d8292/metron-platform/metron-parsers/src/test/java/org/apache/metron/parsers/GrokParserTest.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-parsers/src/test/java/org/apache/metron/parsers/GrokParserTest.java b/metron-platform/metron-parsers/src/test/java/org/apache/metron/parsers/GrokParserTest.java
index ec80396..7abcc87 100644
--- a/metron-platform/metron-parsers/src/test/java/org/apache/metron/parsers/GrokParserTest.java
+++ b/metron-platform/metron-parsers/src/test/java/org/apache/metron/parsers/GrokParserTest.java
@@ -20,68 +20,43 @@ package org.apache.metron.parsers;
 import com.google.common.collect.MapDifference;
 import com.google.common.collect.Maps;
 import junit.framework.Assert;
-import org.adrianwalker.multilinestring.Multiline;
 import org.json.simple.JSONObject;
 import org.json.simple.parser.JSONParser;
 import org.json.simple.parser.ParseException;
 import org.junit.Before;
 import org.junit.Test;
+
 import java.io.IOException;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 
-public class GrokParserTest {
-
-  public String expectedRaw = "2016-01-28 15:29:48.512|2016-01-28 15:29:48.512|   0.000|   0.000|  6|                          216.21.170.221|   80|                               10.0.2.15|39468|      AS|       0|       0|       0|22efa001|00000000|000|000|       1|      44|       0|       0|    0|idle";
-
-  /**
-   * {
-   * "roct":0,
-   * "end_reason":"idle",
-   * "ip_dst_addr":"10.0.2.15",
-   * "iflags":"AS",
-   * "rpkt":0,
-   * "original_string":"2016-01-28 15:29:48.512|2016-01-28 15:29:48.512|   0.000|   0.000|  6|                          216.21.170.221|   80|                               10.0.2.15|39468|      AS|       0|       0|       0|22efa001|00000000|000|000|       1|      44|       0|       0|    0|idle",
-   * "tag":0,
-   * "risn":0,
-   * "ip_dst_port":39468,
-   * "ruflags":0,
-   * "app":0,
-   * "protocol":6
-   * ,"isn":"22efa001",
-   * "uflags":0,"duration":"0.000",
-   * "oct":44,
-   * "ip_src_port":80,
-   * "end_time":"2016-01-28 15:29:48.512",
-   * "riflags":0,"start_time":"2016-01-28 15:29:48.512",
-   * "rtt":"0.000",
-   * "rtag":0,
-   * "pkt":1,
-   * "ip_src_addr":"216.21.170.221"
-   * }
-   */
-  @Multiline
-  private String expectedParsedString;
+public abstract class GrokParserTest {
 
   private JSONObject expectedParsed;
 
   @Before
   public void parseJSON() throws ParseException {
     JSONParser jsonParser = new JSONParser();
-    expectedParsed = (JSONObject) jsonParser.parse(expectedParsedString);
+    expectedParsed = (JSONObject) jsonParser.parse(getExpectedParsedString());
   }
 
   @Test
   public void test() throws IOException, ParseException {
-    String metronHdfsHome = "../metron-parsers/src/main/";
-    String grokHdfsPath = "/patterns/yaf";
-    String patternLabel = "YAF_DELIMITED";
-    GrokParser grokParser = new GrokParser(grokHdfsPath, patternLabel);
-    grokParser.withMetronHDFSHome(metronHdfsHome);
+    String metronHdfsHome = "";
+    GrokParser grokParser = new GrokParser(getGrokPath(), getGrokPatternLabel());
+    String[] timeFields = getTimeFields();
+    if (timeFields != null) {
+      grokParser.withTimeFields(getTimeFields());
+    }
+    String dateFormat = getDateFormat();
+    if (dateFormat != null) {
+      grokParser.withDateFormat(getDateFormat());
+    }
+    grokParser.withTimestampField(getTimestampField());
     grokParser.init();
-    byte[] rawMessage = expectedRaw.getBytes();
+    byte[] rawMessage = getRawMessage().getBytes();
     List<JSONObject> parsedList = grokParser.parse(rawMessage);
     Assert.assertEquals(1, parsedList.size());
     compare(expectedParsed, parsedList.get(0));
@@ -111,4 +86,12 @@ public class GrokParserTest {
     if (actualDifferences.size() > 0) Assert.fail("Expected and Actual JSON values don't match: " + actualDifferences);
     return true;
   }
+
+  public abstract String getRawMessage();
+  public abstract String getExpectedParsedString();
+  public abstract String getGrokPath();
+  public abstract String getGrokPatternLabel();
+  public abstract String[] getTimeFields();
+  public abstract String getDateFormat();
+  public abstract String getTimestampField();
 }

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/645d8292/metron-platform/metron-parsers/src/test/java/org/apache/metron/parsers/SampleGrokParserTest.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-parsers/src/test/java/org/apache/metron/parsers/SampleGrokParserTest.java b/metron-platform/metron-parsers/src/test/java/org/apache/metron/parsers/SampleGrokParserTest.java
new file mode 100644
index 0000000..6f489a4
--- /dev/null
+++ b/metron-platform/metron-parsers/src/test/java/org/apache/metron/parsers/SampleGrokParserTest.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 org.apache.metron.parsers;
+
+import org.adrianwalker.multilinestring.Multiline;
+
+public class SampleGrokParserTest extends GrokParserTest {
+
+  /**
+   * {
+   * "roct":0,
+   * "end_reason":"idle",
+   * "ip_dst_addr":"10.0.2.15",
+   * "iflags":"AS",
+   * "rpkt":0,
+   * "original_string":"1453994987000|2016-01-28 15:29:48|   0.000|   0.000|  6|                          216.21.170.221|   80|                               10.0.2.15|39468|      AS|       0|       0|       0|22efa001|00000000|000|000|       1|      44|       0|       0|    0|idle",
+   * "tag":0,
+   * "risn":0,
+   * "ip_dst_port":39468,
+   * "ruflags":0,
+   * "app":0,
+   * "protocol":6
+   * ,"isn":"22efa001",
+   * "uflags":0,"duration":"0.000",
+   * "oct":44,
+   * "ip_src_port":80,
+   * "end_time":1453994988000,
+   * "start_time":1453994987000
+   * "timestamp":1453994987000,
+   * "riflags":0,
+   * "rtt":"0.000",
+   * "rtag":0,
+   * "pkt":1,
+   * "ip_src_addr":"216.21.170.221"
+   * }
+   */
+  @Multiline
+  public String expectedParsedString;
+
+  public String getExpectedParsedString() {
+    return expectedParsedString;
+  }
+
+  public String getRawMessage() {
+    return "1453994987000|2016-01-28 15:29:48|   0.000|   0.000|  6|                          216.21.170.221|   80|                               10.0.2.15|39468|      AS|       0|       0|       0|22efa001|00000000|000|000|       1|      44|       0|       0|    0|idle";
+  }
+
+  public String getGrokPath() {
+    return "../metron-integration-test/src/main/resources/sample/patterns/test";
+  }
+
+  public String getGrokPatternLabel() {
+    return "YAF_DELIMITED";
+  }
+
+  public String[] getTimeFields() {
+    return new String[]{"end_time"};
+  }
+
+  public String getDateFormat() {
+    return "yyyy-MM-dd HH:mm:ss";
+  }
+
+  public String getTimestampField() {
+    return "start_time";
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/645d8292/metron-platform/metron-parsers/src/test/java/org/apache/metron/parsers/SquidParserTest.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-parsers/src/test/java/org/apache/metron/parsers/SquidParserTest.java b/metron-platform/metron-parsers/src/test/java/org/apache/metron/parsers/SquidParserTest.java
new file mode 100644
index 0000000..1218595
--- /dev/null
+++ b/metron-platform/metron-parsers/src/test/java/org/apache/metron/parsers/SquidParserTest.java
@@ -0,0 +1,75 @@
+/**
+ * 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;
+
+import org.adrianwalker.multilinestring.Multiline;
+
+public class SquidParserTest extends GrokParserTest {
+
+  @Override
+  public String getRawMessage() {
+    return "1461576382.642    161 127.0.0.1 TCP_MISS/200 103701 GET http://www.cnn.com/ - DIRECT/199.27.79.73 text/html";
+  }
+
+  /**
+   * {
+   *   "elapsed":161,
+   *   "code":200,
+   *   "original_string":"1461576382.642    161 127.0.0.1 TCP_MISS/200 103701 GET http://www.cnn.com/ - DIRECT/199.27.79.73 text/html",
+   *   "method":"GET",
+   *   "bytes":103701,
+   *   "action":"TCP_MISS",
+   *   "ip_src_addr":"127.0.0.1",
+   *   "ip_dst_addr":"199.27.79.73",
+   *   "url":"cnn.com",
+   *   "timestamp":1461576382642
+   * }
+   */
+  @Multiline
+  public String expectedParsedString;
+
+  @Override
+  public String getExpectedParsedString() {
+    return expectedParsedString;
+  }
+
+  @Override
+  public String getGrokPath() {
+    return "../metron-parsers/src/main/resources/patterns/squid";
+  }
+
+  @Override
+  public String getGrokPatternLabel() {
+    return "SQUID_DELIMITED";
+  }
+
+  @Override
+  public String[] getTimeFields() {
+    return new String[0];
+  }
+
+  @Override
+  public String getDateFormat() {
+    return null;
+  }
+
+  @Override
+  public String getTimestampField() {
+    return "timestamp";
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/645d8292/metron-platform/metron-parsers/src/test/java/org/apache/metron/parsers/YafParserTest.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-parsers/src/test/java/org/apache/metron/parsers/YafParserTest.java b/metron-platform/metron-parsers/src/test/java/org/apache/metron/parsers/YafParserTest.java
new file mode 100644
index 0000000..9e887f5
--- /dev/null
+++ b/metron-platform/metron-parsers/src/test/java/org/apache/metron/parsers/YafParserTest.java
@@ -0,0 +1,90 @@
+/**
+ * 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;
+
+import org.adrianwalker.multilinestring.Multiline;
+
+public class YafParserTest extends GrokParserTest {
+
+  @Override
+  public String getRawMessage() {
+    return "2016-01-28 15:29:48.512|2016-01-28 15:29:48.512|   0.000|   0.000|  6|                          216.21.170.221|   80|                               10.0.2.15|39468|      AS|       0|       0|       0|22efa001|00000000|000|000|       1|      44|       0|       0|    0|idle";
+  }
+
+  @Override
+  public String getGrokPath() {
+    return "../metron-parsers/src/main/resources/patterns/yaf";
+  }
+
+  /**
+   * {
+   "iflags": "AS",
+   "uflags": 0,
+   "isn": "22efa001",
+   "ip_dst_addr": "10.0.2.15",
+   "ip_dst_port": 39468,
+   "duration": "0.000",
+   "rpkt": 0,
+   "original_string": "2016-01-28 15:29:48.512|2016-01-28 15:29:48.512|   0.000|   0.000|  6|                          216.21.170.221|   80|                               10.0.2.15|39468|      AS|       0|       0|       0|22efa001|00000000|000|000|       1|      44|       0|       0|    0|idle",
+   "pkt": 1,
+   "ruflags": 0,
+   "roct": 0,
+   "ip_src_addr": "216.21.170.221",
+   "tag": 0,
+   "rtag": 0,
+   "ip_src_port": 80,
+   "timestamp": 1453994988512,
+   "app": 0,
+   "oct": 44,
+   "end_reason": "idle",
+   "risn": 0,
+   "end_time": 1453994988512,
+   "start_time": 1453994988512,
+   "riflags": 0,
+   "rtt": "0.000",
+   "protocol": 6
+   }
+   */
+  @Multiline
+  public String expectedParsedString;
+
+  @Override
+  public String getExpectedParsedString() {
+    return expectedParsedString;
+  }
+
+  @Override
+  public String getGrokPatternLabel() {
+    return "YAF_DELIMITED";
+  }
+
+  @Override
+  public String[] getTimeFields() {
+    return new String[]{"start_time", "end_time"};
+  }
+
+  @Override
+  public String getDateFormat() {
+    return "yyyy-MM-dd HH:mm:ss.S";
+  }
+
+  @Override
+  public String getTimestampField() {
+    return "start_time";
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/645d8292/metron-platform/metron-parsers/src/test/java/org/apache/metron/parsers/bluecoat/BasicBluecoatParserTest.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-parsers/src/test/java/org/apache/metron/parsers/bluecoat/BasicBluecoatParserTest.java b/metron-platform/metron-parsers/src/test/java/org/apache/metron/parsers/bluecoat/BasicBluecoatParserTest.java
new file mode 100644
index 0000000..836a0dd
--- /dev/null
+++ b/metron-platform/metron-parsers/src/test/java/org/apache/metron/parsers/bluecoat/BasicBluecoatParserTest.java
@@ -0,0 +1,100 @@
+/**
+ * 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.bluecoat;
+
+import static org.junit.Assert.assertEquals;
+
+import java.util.List;
+
+import org.json.simple.JSONObject;
+import org.junit.Test;
+
+public class BasicBluecoatParserTest {
+
+	private BasicBluecoatParser bbp = new BasicBluecoatParser();
+
+	public BasicBluecoatParserTest() throws Exception {
+		super();
+
+	}
+
+	@Test
+	public void test250017() {
+		String testString = "<29>Apr 15 16:46:33 ProxySG: 250017 Authentication failed from 10.118.29.228: user 'UZL193' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113";
+
+		List<JSONObject> result = bbp.parse(testString.getBytes());
+
+		JSONObject jo = result.get(0);
+
+		assertEquals(jo.get("event_type"), "authentication failure");
+		assertEquals(jo.get("event_code"), "250017");
+		assertEquals(jo.get("realm"), "AD_ldap");
+		assertEquals(jo.get("priority"), "29");
+		assertEquals(jo.get("designated_host"), "10.118.29.228");
+
+		System.out.println(result);
+	}
+
+	@Test
+	public void test250018() {
+		String testString = "<29>Apr 14 20:31:07 ProxySG: 250018 LDAP: invalid credentials: reason: '80090308: LdapErr: DSID-0C0903AA, comment: AcceptSecurityContext error, data 52e, v1772' dn: 'CN=FJL928,OU=Developers,OU=All Users,DC=cof,DC=ds,DC=capitalone,DC=com' realm: 'AD_ldap'(2425130) NORMAL_EVENT realm_ldap.cpp 2833";
+
+		List<JSONObject> result = bbp.parse(testString.getBytes());
+		JSONObject jo = result.get(0);
+
+		assertEquals(jo.get("event_type"), "authentication failure");
+		assertEquals(jo.get("event_code"), "250018");
+		assertEquals(jo.get("realm"), "AD_ldap");
+		assertEquals(jo.get("priority"), "29");
+
+		System.out.println(result);
+	}
+
+	@Test
+	public void testE0000() {
+		String testString = "<29>Apr 15 18:01:46 ProxySG: E0000 Access Log FTP (main): 230 Login successful.(0) NORMAL_EVENT alog_ftp_client.cpp 1743";
+
+		List<JSONObject> result = bbp.parse(testString.getBytes());
+		
+		JSONObject jo = result.get(0);
+
+		assertEquals(jo.get("event_type"), "authentication");
+		assertEquals(jo.get("event_code"), "E0000");
+		assertEquals(jo.get("priority"), "29");
+
+		System.out.println(result);
+	}
+
+	@Test
+	public void test250001() {
+		String testString = "<29>Apr 14 20:31:07 ProxySG: 250001 LDAP: Authentication failed from 10.113.216.196: no such user in realm 'AD_ldap'(102089) NORMAL_EVENT realm_ldap.cpp 2634";
+
+		List<JSONObject> result = bbp.parse(testString.getBytes());
+		
+		JSONObject jo = result.get(0);
+
+		assertEquals(jo.get("event_type"), "authentication failure");
+		assertEquals(jo.get("event_code"), "250001");
+		assertEquals(jo.get("realm"), "AD_ldap");
+		assertEquals(jo.get("priority"), "29");
+		assertEquals(jo.get("designated_host"), "10.113.216.196");
+
+		System.out.println(result);
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/645d8292/metron-platform/metron-parsers/src/test/java/org/apache/metron/parsers/integration/BluecoatIntegrationTest.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-parsers/src/test/java/org/apache/metron/parsers/integration/BluecoatIntegrationTest.java b/metron-platform/metron-parsers/src/test/java/org/apache/metron/parsers/integration/BluecoatIntegrationTest.java
new file mode 100644
index 0000000..8b6207c
--- /dev/null
+++ b/metron-platform/metron-parsers/src/test/java/org/apache/metron/parsers/integration/BluecoatIntegrationTest.java
@@ -0,0 +1,48 @@
+/**
+ * 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.integration;
+
+import org.apache.metron.TestConstants;
+
+public class BluecoatIntegrationTest extends ParserIntegrationTest {
+
+  @Override
+  public String getFluxPath() {
+    return "./src/main/flux/bluecoat/test.yaml";
+  }
+
+  @Override
+  public String getSampleInputPath() {
+    return TestConstants.SAMPLE_DATA_INPUT_PATH + "BluecoatSyslog.txt";
+  }
+
+  @Override
+  public String getSampleParsedPath() {
+    return TestConstants.SAMPLE_DATA_PARSED_PATH + "BluecoatParsed";
+  }
+
+  @Override
+  public String getSensorType() {
+    return "bluecoat";
+  }
+
+  @Override
+  public String getFluxTopicProperty() {
+    return "spout.kafka.topic.yaf";
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/645d8292/metron-platform/metron-parsers/src/test/java/org/apache/metron/parsers/integration/ParserIntegrationTest.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-parsers/src/test/java/org/apache/metron/parsers/integration/ParserIntegrationTest.java b/metron-platform/metron-parsers/src/test/java/org/apache/metron/parsers/integration/ParserIntegrationTest.java
index a21ebf8..e0b9432 100644
--- a/metron-platform/metron-parsers/src/test/java/org/apache/metron/parsers/integration/ParserIntegrationTest.java
+++ b/metron-platform/metron-parsers/src/test/java/org/apache/metron/parsers/integration/ParserIntegrationTest.java
@@ -52,9 +52,7 @@ public abstract class ParserIntegrationTest extends BaseIntegrationTest {
 
     final List<byte[]> inputMessages = TestUtils.readSampleData(getSampleInputPath());
 
-    final Properties topologyProperties = new Properties() {{
-      setProperty(getFluxTopicProperty(), kafkaTopic);
-    }};
+    final Properties topologyProperties = new Properties();
     final KafkaWithZKComponent kafkaComponent = getKafkaComponent(topologyProperties, new ArrayList<KafkaWithZKComponent.Topic>() {{
       add(new KafkaWithZKComponent.Topic(kafkaTopic, 1));
     }});
@@ -100,7 +98,13 @@ public abstract class ParserIntegrationTest extends BaseIntegrationTest {
     for (int i = 0; i < outputMessages.size(); i++) {
       String sampleParsedMessage = new String(sampleParsedMessages.get(i));
       String outputMessage = new String(outputMessages.get(i));
-      assertJSONEqual(sampleParsedMessage, outputMessage);
+      try {
+        assertJSONEqual(sampleParsedMessage, outputMessage);
+      } catch (Throwable t) {
+        System.out.println("expected: " + sampleParsedMessage);
+        System.out.println("actual: " + outputMessage);
+        throw t;
+      }
     }
     runner.stop();
 
@@ -110,7 +114,6 @@ public abstract class ParserIntegrationTest extends BaseIntegrationTest {
     ObjectMapper mapper = new ObjectMapper();
     Map m1 = mapper.readValue(doc1, Map.class);
     Map m2 = mapper.readValue(doc2, Map.class);
-    Assert.assertEquals(m1.size(), m2.size());
     for(Object k : m1.keySet()) {
       Object v1 = m1.get(k);
       Object v2 = m2.get(k);
@@ -126,6 +129,7 @@ public abstract class ParserIntegrationTest extends BaseIntegrationTest {
         Assert.assertEquals("value mismatch for " + k ,v1, v2);
       }
     }
+    Assert.assertEquals(m1.size(), m2.size());
   }
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/645d8292/metron-platform/metron-parsers/src/test/java/org/apache/metron/parsers/integration/SquidIntegrationTest.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-parsers/src/test/java/org/apache/metron/parsers/integration/SquidIntegrationTest.java b/metron-platform/metron-parsers/src/test/java/org/apache/metron/parsers/integration/SquidIntegrationTest.java
new file mode 100644
index 0000000..023278d
--- /dev/null
+++ b/metron-platform/metron-parsers/src/test/java/org/apache/metron/parsers/integration/SquidIntegrationTest.java
@@ -0,0 +1,49 @@
+/**
+ * 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.integration;
+
+import org.apache.metron.TestConstants;
+import org.json.simple.JSONObject;
+
+public class SquidIntegrationTest extends ParserIntegrationTest {
+
+  @Override
+  public String getFluxPath() {
+    return "./src/main/flux/squid/test.yaml";
+  }
+
+  @Override
+  public String getSampleInputPath() {
+    return TestConstants.SAMPLE_DATA_INPUT_PATH + "SquidExampleOutput";
+  }
+
+  @Override
+  public String getSampleParsedPath() {
+    return TestConstants.SAMPLE_DATA_PARSED_PATH + "SquidExampleParsed";
+  }
+
+  @Override
+  public String getSensorType() {
+    return "squid";
+  }
+
+  @Override
+  public String getFluxTopicProperty() {
+    return "spout.kafka.topic.yaf";
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/645d8292/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 8554419..fe1ecd8 100644
--- a/pom.xml
+++ b/pom.xml
@@ -76,6 +76,7 @@
                         <exclude>metron-ui/lib/public/font/**</exclude>
                         <exclude>metron-ui/node_modules/**</exclude>
                         <exclude>**/src/main/resources/patterns/**</exclude>
+                        <exclude>**/src/main/resources/sample/patterns/**</exclude>
                         <exclude>**/src/test/resources/**</exclude>
                         <exclude>**/src/main/resources/sample/data/Sample*/**</exclude>
                         <exclude>**/dependency-reduced-pom.xml</exclude>


[2/3] incubator-metron git commit: METRON-122 Create generic unit test framework for testing grok statements (merrimanr) closes apache/incubator-metron#96

Posted by rm...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/645d8292/metron-platform/metron-integration-test/src/main/resources/sample/data/SampleParsed/BluecoatParsed
----------------------------------------------------------------------
diff --git a/metron-platform/metron-integration-test/src/main/resources/sample/data/SampleParsed/BluecoatParsed b/metron-platform/metron-integration-test/src/main/resources/sample/data/SampleParsed/BluecoatParsed
new file mode 100644
index 0000000..201c972
--- /dev/null
+++ b/metron-platform/metron-integration-test/src/main/resources/sample/data/SampleParsed/BluecoatParsed
@@ -0,0 +1,144 @@
+{"eid":"WJS310","original_string":"<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.212.29.36: user 'WJS310' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113","event_type":"authentication failure","event_code":"250017","designated_host":"10.212.29.36","realm":"AD_ldap","priority":"29","message":" Authentication failed from 10.212.29.36: user 'WJS310' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113","timestamp":1460683867000,"source.type":"bluecoat"}
+{"eid":"vwv149","original_string":"<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.212.21.253: user 'vwv149' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113","event_type":"authentication failure","event_code":"250017","designated_host":"10.212.21.253","realm":"iwa_realm","priority":"29","message":" Authentication failed from 10.212.21.253: user 'vwv149' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113","timestamp":1460683867000,"source.type":"bluecoat"}
+{"eid":"vwv149","original_string":"<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.212.21.253: user 'vwv149' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113","event_type":"authentication failure","event_code":"250017","designated_host":"10.212.21.253","realm":"iwa_realm","priority":"29","message":" Authentication failed from 10.212.21.253: user 'vwv149' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113","timestamp":1460683867000,"source.type":"bluecoat"}
+{"eid":"yaw983","original_string":"<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.118.216.222: user 'yaw983' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113","event_type":"authentication failure","event_code":"250017","designated_host":"10.118.216.222","realm":"AD_ldap","priority":"29","message":" Authentication failed from 10.118.216.222: user 'yaw983' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113","timestamp":1460683867000,"source.type":"bluecoat"}
+{"eid":"yaw983","original_string":"<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.118.216.222: user 'yaw983' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113","event_type":"authentication failure","event_code":"250017","designated_host":"10.118.216.222","realm":"iwa_realm","priority":"29","message":" Authentication failed from 10.118.216.222: user 'yaw983' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113","timestamp":1460683867000,"source.type":"bluecoat"}
+{"eid":"yaw983","original_string":"<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.118.216.222: user 'yaw983' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113","event_type":"authentication failure","event_code":"250017","designated_host":"10.118.216.222","realm":"AD_ldap","priority":"29","message":" Authentication failed from 10.118.216.222: user 'yaw983' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113","timestamp":1460683867000,"source.type":"bluecoat"}
+{"eid":"qwn225","original_string":"<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.115.220.223: user 'qwn225' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113","event_type":"authentication failure","event_code":"250017","designated_host":"10.115.220.223","realm":"iwa_realm","priority":"29","message":" Authentication failed from 10.115.220.223: user 'qwn225' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113","timestamp":1460683867000,"source.type":"bluecoat"}
+{"eid":"ags432","original_string":"<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.114.217.29: user 'ags432' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113","event_type":"authentication failure","event_code":"250017","designated_host":"10.114.217.29","realm":"AD_ldap","priority":"29","message":" Authentication failed from 10.114.217.29: user 'ags432' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113","timestamp":1460683867000,"source.type":"bluecoat"}
+{"eid":"u62206","original_string":"<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.113.216.196: user 'u62206' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113","event_type":"authentication failure","event_code":"250017","designated_host":"10.113.216.196","realm":"iwa_realm","priority":"29","message":" Authentication failed from 10.113.216.196: user 'u62206' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113","timestamp":1460683867000,"source.type":"bluecoat"}
+{"eid":"CXI886","original_string":"<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.127.221.164: user 'CXI886' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113","event_type":"authentication failure","event_code":"250017","designated_host":"10.127.221.164","realm":"iwa_realm","priority":"29","message":" Authentication failed from 10.127.221.164: user 'CXI886' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113","timestamp":1460683867000,"source.type":"bluecoat"}
+{"eid":"CXI886","original_string":"<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.127.221.164: user 'CXI886' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113","event_type":"authentication failure","event_code":"250017","designated_host":"10.127.221.164","realm":"AD_ldap","priority":"29","message":" Authentication failed from 10.127.221.164: user 'CXI886' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113","timestamp":1460683867000,"source.type":"bluecoat"}
+{"eid":"ags432","original_string":"<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.114.217.29: user 'ags432' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113","event_type":"authentication failure","event_code":"250017","designated_host":"10.114.217.29","realm":"AD_ldap","priority":"29","message":" Authentication failed from 10.114.217.29: user 'ags432' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113","timestamp":1460683867000,"source.type":"bluecoat"}
+{"eid":"fjl928","original_string":"<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.218.165.248: user 'fjl928' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113","event_type":"authentication failure","event_code":"250017","designated_host":"10.218.165.248","realm":"iwa_realm","priority":"29","message":" Authentication failed from 10.218.165.248: user 'fjl928' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113","timestamp":1460683867000,"source.type":"bluecoat"}
+{"eid":"LOCAL\\uzl193","original_string":"<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.118.29.228: user 'LOCAL\\uzl193' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113","event_type":"authentication failure","event_code":"250017","designated_host":"10.118.29.228","realm":"iwa_realm","priority":"29","message":" Authentication failed from 10.118.29.228: user 'LOCAL\\uzl193' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113","timestamp":1460683867000,"source.type":"bluecoat"}
+{"eid":"UZL193","original_string":"<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.118.29.228: user 'UZL193' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113","event_type":"authentication failure","event_code":"250017","designated_host":"10.118.29.228","realm":"iwa_realm","priority":"29","message":" Authentication failed from 10.118.29.228: user 'UZL193' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113","timestamp":1460683867000,"source.type":"bluecoat"}
+{"eid":"vwv149","original_string":"<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.212.21.253: user 'vwv149' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113","event_type":"authentication failure","event_code":"250017","designated_host":"10.212.21.253","realm":"iwa_realm","priority":"29","message":" Authentication failed from 10.212.21.253: user 'vwv149' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113","timestamp":1460683867000,"source.type":"bluecoat"}
+{"eid":"ugs662","original_string":"<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.152.102.72: user 'ugs662' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113","event_type":"authentication failure","event_code":"250017","designated_host":"10.152.102.72","realm":"AD_ldap","priority":"29","message":" Authentication failed from 10.152.102.72: user 'ugs662' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113","timestamp":1460683867000,"source.type":"bluecoat"}
+{"eid":"XGZ521","original_string":"<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.119.223.52: user 'XGZ521' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113","event_type":"authentication failure","event_code":"250017","designated_host":"10.119.223.52","realm":"iwa_realm","priority":"29","message":" Authentication failed from 10.119.223.52: user 'XGZ521' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113","timestamp":1460683867000,"source.type":"bluecoat"}
+{"original_string":"<29>Apr 14 20:31:07 ProxySG: 250018 LDAP: invalid credentials: reason: '80090308: LdapErr: DSID-0C0903AA, comment: AcceptSecurityContext error, data 52e, v1772' dn: 'CN=FJL928,OU=Developers,OU=All Users,DC=cof,DC=ds,DC=capitalone,DC=com' realm: 'AD_ldap'(2425130) NORMAL_EVENT realm_ldap.cpp 2833","event_type":"authentication failure","event_code":"250018","realm":"AD_ldap","priority":"29","message":" LDAP: invalid credentials: reason: '80090308: LdapErr: DSID-0C0903AA, comment: AcceptSecurityContext error, data 52e, v1772' dn: 'CN=FJL928,OU=Developers,OU=All Users,DC=cof,DC=ds,DC=capitalone,DC=com' realm: 'AD_ldap'(2425130) NORMAL_EVENT realm_ldap.cpp 2833","timestamp":1460683867000,"source.type":"bluecoat"}
+{"eid":"fjl928","original_string":"<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.218.165.248: user 'fjl928' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113","event_type":"authentication failure","event_code":"250017","designated_host":"10.218.165.248","realm":"AD_ldap","priority":"29","message":" Authentication failed from 10.218.165.248: user 'fjl928' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113","timestamp":1460683867000,"source.type":"bluecoat"}
+{"original_string":"<29>Apr 14 20:31:07 ProxySG: 250018 LDAP: invalid credentials: reason: '80090308: LdapErr: DSID-0C0903AA, comment: AcceptSecurityContext error, data 52e, v1772' dn: 'CN=UZL193,OU=User Lock Policy 00,OU=All Users,DC=cof,DC=ds,DC=capitalone,DC=com' realm: 'AD_ldap'(2425130) NORMAL_EVENT realm_ldap.cpp 2833","event_type":"authentication failure","event_code":"250018","realm":"0,OU=Al","priority":"29","message":" LDAP: invalid credentials: reason: '80090308: LdapErr: DSID-0C0903AA, comment: AcceptSecurityContext error, data 52e, v1772' dn: 'CN=UZL193,OU=User Lock Policy 00,OU=All Users,DC=cof,DC=ds,DC=capitalone,DC=com' realm: 'AD_ldap'(2425130) NORMAL_EVENT realm_ldap.cpp 2833","timestamp":1460683867000,"source.type":"bluecoat"}
+{"eid":"UZL193","original_string":"<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.118.29.228: user 'UZL193' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113","event_type":"authentication failure","event_code":"250017","designated_host":"10.118.29.228","realm":"iwa_realm","priority":"29","message":" Authentication failed from 10.118.29.228: user 'UZL193' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113","timestamp":1460683867000,"source.type":"bluecoat"}
+{"eid":"UZL193","original_string":"<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.118.29.228: user 'UZL193' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113","event_type":"authentication failure","event_code":"250017","designated_host":"10.118.29.228","realm":"iwa_realm","priority":"29","message":" Authentication failed from 10.118.29.228: user 'UZL193' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113","timestamp":1460683867000,"source.type":"bluecoat"}
+{"eid":"XGZ521","original_string":"<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.119.223.52: user 'XGZ521' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113","event_type":"authentication failure","event_code":"250017","designated_host":"10.119.223.52","realm":"AD_ldap","priority":"29","message":" Authentication failed from 10.119.223.52: user 'XGZ521' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113","timestamp":1460683867000,"source.type":"bluecoat"}
+{"eid":"ugs662","original_string":"<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.152.102.72: user 'ugs662' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113","event_type":"authentication failure","event_code":"250017","designated_host":"10.152.102.72","realm":"iwa_realm","priority":"29","message":" Authentication failed from 10.152.102.72: user 'ugs662' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113","timestamp":1460683867000,"source.type":"bluecoat"}
+{"eid":"UZL193","original_string":"<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.118.29.228: user 'UZL193' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113","event_type":"authentication failure","event_code":"250017","designated_host":"10.118.29.228","realm":"AD_ldap","priority":"29","message":" Authentication failed from 10.118.29.228: user 'UZL193' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113","timestamp":1460683867000,"source.type":"bluecoat"}
+{"eid":"vwv149","original_string":"<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.212.21.253: user 'vwv149' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113","event_type":"authentication failure","event_code":"250017","designated_host":"10.212.21.253","realm":"iwa_realm","priority":"29","message":" Authentication failed from 10.212.21.253: user 'vwv149' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113","timestamp":1460683867000,"source.type":"bluecoat"}
+{"eid":"UZL193","original_string":"<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.118.29.228: user 'UZL193' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113","event_type":"authentication failure","event_code":"250017","designated_host":"10.118.29.228","realm":"AD_ldap","priority":"29","message":" Authentication failed from 10.118.29.228: user 'UZL193' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113","timestamp":1460683867000,"source.type":"bluecoat"}
+{"eid":"vwv149","original_string":"<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.212.21.253: user 'vwv149' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113","event_type":"authentication failure","event_code":"250017","designated_host":"10.212.21.253","realm":"AD_ldap","priority":"29","message":" Authentication failed from 10.212.21.253: user 'vwv149' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113","timestamp":1460683867000,"source.type":"bluecoat"}
+{"eid":"UZL193","original_string":"<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.118.29.228: user 'UZL193' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113","event_type":"authentication failure","event_code":"250017","designated_host":"10.118.29.228","realm":"iwa_realm","priority":"29","message":" Authentication failed from 10.118.29.228: user 'UZL193' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113","timestamp":1460683867000,"source.type":"bluecoat"}
+{"eid":"UZL193","original_string":"<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.118.29.228: user 'UZL193' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113","event_type":"authentication failure","event_code":"250017","designated_host":"10.118.29.228","realm":"iwa_realm","priority":"29","message":" Authentication failed from 10.118.29.228: user 'UZL193' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113","timestamp":1460683867000,"source.type":"bluecoat"}
+{"eid":"UZL193","original_string":"<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.118.29.228: user 'UZL193' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113","event_type":"authentication failure","event_code":"250017","designated_host":"10.118.29.228","realm":"AD_ldap","priority":"29","message":" Authentication failed from 10.118.29.228: user 'UZL193' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113","timestamp":1460683867000,"source.type":"bluecoat"}
+{"original_string":"<29>Apr 14 20:31:07 ProxySG: 250018 LDAP: invalid credentials: reason: '80090308: LdapErr: DSID-0C0903AA, comment: AcceptSecurityContext error, data 52e, v1772' dn: 'CN=EPL857,OU=User Lock Policy 05,OU=All Users,DC=cof,DC=ds,DC=capitalone,DC=com' realm: 'AD_ldap'(2425130) NORMAL_EVENT realm_ldap.cpp 2833","event_type":"authentication failure","event_code":"250018","realm":"5,OU=Al","priority":"29","message":" LDAP: invalid credentials: reason: '80090308: LdapErr: DSID-0C0903AA, comment: AcceptSecurityContext error, data 52e, v1772' dn: 'CN=EPL857,OU=User Lock Policy 05,OU=All Users,DC=cof,DC=ds,DC=capitalone,DC=com' realm: 'AD_ldap'(2425130) NORMAL_EVENT realm_ldap.cpp 2833","timestamp":1460683867000,"source.type":"bluecoat"}
+{"eid":"epl857","original_string":"<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.210.223.65: user 'epl857' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113","event_type":"authentication failure","event_code":"250017","designated_host":"10.210.223.65","realm":"AD_ldap","priority":"29","message":" Authentication failed from 10.210.223.65: user 'epl857' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113","timestamp":1460683867000,"source.type":"bluecoat"}
+{"eid":"epl857","original_string":"<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.210.223.65: user 'epl857' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113","event_type":"authentication failure","event_code":"250017","designated_host":"10.210.223.65","realm":"iwa_realm","priority":"29","message":" Authentication failed from 10.210.223.65: user 'epl857' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113","timestamp":1460683867000,"source.type":"bluecoat"}
+{"eid":"UZL193","original_string":"<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.118.29.228: user 'UZL193' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113","event_type":"authentication failure","event_code":"250017","designated_host":"10.118.29.228","realm":"iwa_realm","priority":"29","message":" Authentication failed from 10.118.29.228: user 'UZL193' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113","timestamp":1460683867000,"source.type":"bluecoat"}
+{"eid":"UZL193","original_string":"<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.118.29.228: user 'UZL193' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113","event_type":"authentication failure","event_code":"250017","designated_host":"10.118.29.228","realm":"AD_ldap","priority":"29","message":" Authentication failed from 10.118.29.228: user 'UZL193' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113","timestamp":1460683867000,"source.type":"bluecoat"}
+{"eid":"UZL193","original_string":"<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.118.29.228: user 'UZL193' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113","event_type":"authentication failure","event_code":"250017","designated_host":"10.118.29.228","realm":"iwa_realm","priority":"29","message":" Authentication failed from 10.118.29.228: user 'UZL193' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113","timestamp":1460683867000,"source.type":"bluecoat"}
+{"eid":"UZL193","original_string":"<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.118.29.228: user 'UZL193' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113","event_type":"authentication failure","event_code":"250017","designated_host":"10.118.29.228","realm":"AD_ldap","priority":"29","message":" Authentication failed from 10.118.29.228: user 'UZL193' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113","timestamp":1460683867000,"source.type":"bluecoat"}
+{"eid":"LOCAL\\sdq302","original_string":"<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.127.216.106: user 'LOCAL\\sdq302' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113","event_type":"authentication failure","event_code":"250017","designated_host":"10.127.216.106","realm":"iwa_realm","priority":"29","message":" Authentication failed from 10.127.216.106: user 'LOCAL\\sdq302' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113","timestamp":1460683867000,"source.type":"bluecoat"}
+{"eid":"UZL193","original_string":"<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.118.29.228: user 'UZL193' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113","event_type":"authentication failure","event_code":"250017","designated_host":"10.118.29.228","realm":"AD_ldap","priority":"29","message":" Authentication failed from 10.118.29.228: user 'UZL193' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113","timestamp":1460683867000,"source.type":"bluecoat"}
+{"eid":"LOCAL\\kon313","original_string":"<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.219.15.104: user 'LOCAL\\kon313' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113","event_type":"authentication failure","event_code":"250017","designated_host":"10.219.15.104","realm":"iwa_realm","priority":"29","message":" Authentication failed from 10.219.15.104: user 'LOCAL\\kon313' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113","timestamp":1460683867000,"source.type":"bluecoat"}
+{"eid":"UZL193","original_string":"<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.118.29.228: user 'UZL193' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113","event_type":"authentication failure","event_code":"250017","designated_host":"10.118.29.228","realm":"iwa_realm","priority":"29","message":" Authentication failed from 10.118.29.228: user 'UZL193' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113","timestamp":1460683867000,"source.type":"bluecoat"}
+{"eid":"vwv149","original_string":"<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.212.21.253: user 'vwv149' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113","event_type":"authentication failure","event_code":"250017","designated_host":"10.212.21.253","realm":"AD_ldap","priority":"29","message":" Authentication failed from 10.212.21.253: user 'vwv149' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113","timestamp":1460683867000,"source.type":"bluecoat"}
+{"original_string":"<29>Apr 14 20:31:07 ProxySG: 250018 LDAP: invalid credentials: reason: '80090308: LdapErr: DSID-0C0903AA, comment: AcceptSecurityContext error, data 52e, v1772' dn: 'CN=SDQ302,OU=Developers,OU=All Users,DC=cof,DC=ds,DC=capitalone,DC=com' realm: 'AD_ldap'(2425130) NORMAL_EVENT realm_ldap.cpp 2833","event_type":"authentication failure","event_code":"250018","realm":"AD_ldap","priority":"29","message":" LDAP: invalid credentials: reason: '80090308: LdapErr: DSID-0C0903AA, comment: AcceptSecurityContext error, data 52e, v1772' dn: 'CN=SDQ302,OU=Developers,OU=All Users,DC=cof,DC=ds,DC=capitalone,DC=com' realm: 'AD_ldap'(2425130) NORMAL_EVENT realm_ldap.cpp 2833","timestamp":1460683867000,"source.type":"bluecoat"}
+{"eid":"sdq302","original_string":"<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.127.216.106: user 'sdq302' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113","event_type":"authentication failure","event_code":"250017","designated_host":"10.127.216.106","realm":"AD_ldap","priority":"29","message":" Authentication failed from 10.127.216.106: user 'sdq302' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113","timestamp":1460683867000,"source.type":"bluecoat"}
+{"eid":"UZL193","original_string":"<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.118.29.228: user 'UZL193' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113","event_type":"authentication failure","event_code":"250017","designated_host":"10.118.29.228","realm":"AD_ldap","priority":"29","message":" Authentication failed from 10.118.29.228: user 'UZL193' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113","timestamp":1460683867000,"source.type":"bluecoat"}
+{"eid":"UZL193","original_string":"<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.118.29.228: user 'UZL193' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113","event_type":"authentication failure","event_code":"250017","designated_host":"10.118.29.228","realm":"AD_ldap","priority":"29","message":" Authentication failed from 10.118.29.228: user 'UZL193' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113","timestamp":1460683867000,"source.type":"bluecoat"}
+{"eid":"LOCAL\\dkg773","original_string":"<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.120.144.20: user 'LOCAL\\dkg773' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113","event_type":"authentication failure","event_code":"250017","designated_host":"10.120.144.20","realm":"iwa_realm","priority":"29","message":" Authentication failed from 10.120.144.20: user 'LOCAL\\dkg773' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113","timestamp":1460683867000,"source.type":"bluecoat"}
+{"eid":"dkg773","original_string":"<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.120.144.20: user 'dkg773' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113","event_type":"authentication failure","event_code":"250017","designated_host":"10.120.144.20","realm":"iwa_realm","priority":"29","message":" Authentication failed from 10.120.144.20: user 'dkg773' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113","timestamp":1460683867000,"source.type":"bluecoat"}
+{"eid":"kon313","original_string":"<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.219.15.104: user 'kon313' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113","event_type":"authentication failure","event_code":"250017","designated_host":"10.219.15.104","realm":"AD_ldap","priority":"29","message":" Authentication failed from 10.219.15.104: user 'kon313' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113","timestamp":1460683867000,"source.type":"bluecoat"}
+{"eid":"UZL193","original_string":"<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.118.29.228: user 'UZL193' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113","event_type":"authentication failure","event_code":"250017","designated_host":"10.118.29.228","realm":"AD_ldap","priority":"29","message":" Authentication failed from 10.118.29.228: user 'UZL193' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113","timestamp":1460683867000,"source.type":"bluecoat"}
+{"eid":"UZL193","original_string":"<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.118.29.228: user 'UZL193' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113","event_type":"authentication failure","event_code":"250017","designated_host":"10.118.29.228","realm":"iwa_realm","priority":"29","message":" Authentication failed from 10.118.29.228: user 'UZL193' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113","timestamp":1460683867000,"source.type":"bluecoat"}
+{"eid":"UZL193","original_string":"<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.118.29.228: user 'UZL193' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113","event_type":"authentication failure","event_code":"250017","designated_host":"10.118.29.228","realm":"iwa_realm","priority":"29","message":" Authentication failed from 10.118.29.228: user 'UZL193' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113","timestamp":1460683867000,"source.type":"bluecoat"}
+{"eid":"UZL193","original_string":"<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.118.29.228: user 'UZL193' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113","event_type":"authentication failure","event_code":"250017","designated_host":"10.118.29.228","realm":"AD_ldap","priority":"29","message":" Authentication failed from 10.118.29.228: user 'UZL193' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113","timestamp":1460683867000,"source.type":"bluecoat"}
+{"eid":"UZL193","original_string":"<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.118.29.228: user 'UZL193' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113","event_type":"authentication failure","event_code":"250017","designated_host":"10.118.29.228","realm":"iwa_realm","priority":"29","message":" Authentication failed from 10.118.29.228: user 'UZL193' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113","timestamp":1460683867000,"source.type":"bluecoat"}
+{"eid":"UZL193","original_string":"<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.118.29.228: user 'UZL193' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113","event_type":"authentication failure","event_code":"250017","designated_host":"10.118.29.228","realm":"iwa_realm","priority":"29","message":" Authentication failed from 10.118.29.228: user 'UZL193' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113","timestamp":1460683867000,"source.type":"bluecoat"}
+{"eid":"qwn225","original_string":"<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.115.220.223: user 'qwn225' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113","event_type":"authentication failure","event_code":"250017","designated_host":"10.115.220.223","realm":"iwa_realm","priority":"29","message":" Authentication failed from 10.115.220.223: user 'qwn225' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113","timestamp":1460683867000,"source.type":"bluecoat"}
+{"eid":"qwn225","original_string":"<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.115.220.223: user 'qwn225' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113","event_type":"authentication failure","event_code":"250017","designated_host":"10.115.220.223","realm":"AD_ldap","priority":"29","message":" Authentication failed from 10.115.220.223: user 'qwn225' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113","timestamp":1460683867000,"source.type":"bluecoat"}
+{"eid":"dkg773","original_string":"<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.120.144.20: user 'dkg773' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113","event_type":"authentication failure","event_code":"250017","designated_host":"10.120.144.20","realm":"AD_ldap","priority":"29","message":" Authentication failed from 10.120.144.20: user 'dkg773' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113","timestamp":1460683867000,"source.type":"bluecoat"}
+{"eid":"ugs662","original_string":"<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.152.102.72: user 'ugs662' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113","event_type":"authentication failure","event_code":"250017","designated_host":"10.152.102.72","realm":"AD_ldap","priority":"29","message":" Authentication failed from 10.152.102.72: user 'ugs662' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113","timestamp":1460683867000,"source.type":"bluecoat"}
+{"eid":"UZL193","original_string":"<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.118.29.228: user 'UZL193' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113","event_type":"authentication failure","event_code":"250017","designated_host":"10.118.29.228","realm":"AD_ldap","priority":"29","message":" Authentication failed from 10.118.29.228: user 'UZL193' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113","timestamp":1460683867000,"source.type":"bluecoat"}
+{"eid":"UZL193","original_string":"<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.118.29.228: user 'UZL193' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113","event_type":"authentication failure","event_code":"250017","designated_host":"10.118.29.228","realm":"AD_ldap","priority":"29","message":" Authentication failed from 10.118.29.228: user 'UZL193' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113","timestamp":1460683867000,"source.type":"bluecoat"}
+{"eid":"UZL193","original_string":"<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.118.29.228: user 'UZL193' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113","event_type":"authentication failure","event_code":"250017","designated_host":"10.118.29.228","realm":"iwa_realm","priority":"29","message":" Authentication failed from 10.118.29.228: user 'UZL193' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113","timestamp":1460683867000,"source.type":"bluecoat"}
+{"eid":"UZL193","original_string":"<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.118.29.228: user 'UZL193' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113","event_type":"authentication failure","event_code":"250017","designated_host":"10.118.29.228","realm":"AD_ldap","priority":"29","message":" Authentication failed from 10.118.29.228: user 'UZL193' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113","timestamp":1460683867000,"source.type":"bluecoat"}
+{"eid":"UZL193","original_string":"<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.118.29.228: user 'UZL193' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113","event_type":"authentication failure","event_code":"250017","designated_host":"10.118.29.228","realm":"AD_ldap","priority":"29","message":" Authentication failed from 10.118.29.228: user 'UZL193' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113","timestamp":1460683867000,"source.type":"bluecoat"}
+{"eid":"UZL193","original_string":"<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.118.29.228: user 'UZL193' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113","event_type":"authentication failure","event_code":"250017","designated_host":"10.118.29.228","realm":"AD_ldap","priority":"29","message":" Authentication failed from 10.118.29.228: user 'UZL193' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113","timestamp":1460683867000,"source.type":"bluecoat"}
+{"eid":"UZL193","original_string":"<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.118.29.228: user 'UZL193' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113","event_type":"authentication failure","event_code":"250017","designated_host":"10.118.29.228","realm":"iwa_realm","priority":"29","message":" Authentication failed from 10.118.29.228: user 'UZL193' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113","timestamp":1460683867000,"source.type":"bluecoat"}
+{"eid":"UZL193","original_string":"<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.118.29.228: user 'UZL193' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113","event_type":"authentication failure","event_code":"250017","designated_host":"10.118.29.228","realm":"AD_ldap","priority":"29","message":" Authentication failed from 10.118.29.228: user 'UZL193' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113","timestamp":1460683867000,"source.type":"bluecoat"}
+{"eid":"LOCAL\\uua398","original_string":"<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.118.219.193: user 'LOCAL\\uua398' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113","event_type":"authentication failure","event_code":"250017","designated_host":"10.118.219.193","realm":"iwa_realm","priority":"29","message":" Authentication failed from 10.118.219.193: user 'LOCAL\\uua398' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113","timestamp":1460683867000,"source.type":"bluecoat"}
+{"eid":"ugs662","original_string":"<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.152.102.72: user 'ugs662' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113","event_type":"authentication failure","event_code":"250017","designated_host":"10.152.102.72","realm":"AD_ldap","priority":"29","message":" Authentication failed from 10.152.102.72: user 'ugs662' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113","timestamp":1460683867000,"source.type":"bluecoat"}
+{"eid":"qwn225","original_string":"<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.115.220.223: user 'qwn225' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113","event_type":"authentication failure","event_code":"250017","designated_host":"10.115.220.223","realm":"AD_ldap","priority":"29","message":" Authentication failed from 10.115.220.223: user 'qwn225' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113","timestamp":1460683867000,"source.type":"bluecoat"}
+{"eid":"qwn225","original_string":"<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.115.220.223: user 'qwn225' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113","event_type":"authentication failure","event_code":"250017","designated_host":"10.115.220.223","realm":"iwa_realm","priority":"29","message":" Authentication failed from 10.115.220.223: user 'qwn225' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113","timestamp":1460683867000,"source.type":"bluecoat"}
+{"eid":"qwn225","original_string":"<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.115.220.223: user 'qwn225' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113","event_type":"authentication failure","event_code":"250017","designated_host":"10.115.220.223","realm":"AD_ldap","priority":"29","message":" Authentication failed from 10.115.220.223: user 'qwn225' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113","timestamp":1460683867000,"source.type":"bluecoat"}
+{"eid":"kon313","original_string":"<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.219.15.104: user 'kon313' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113","event_type":"authentication failure","event_code":"250017","designated_host":"10.219.15.104","realm":"AD_ldap","priority":"29","message":" Authentication failed from 10.219.15.104: user 'kon313' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113","timestamp":1460683867000,"source.type":"bluecoat"}
+{"eid":"vwv149","original_string":"<29>Apr 14 20:31:06 ProxySG: 250017 Authentication failed from 10.212.21.253: user 'vwv149' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113","event_type":"authentication failure","event_code":"250017","designated_host":"10.212.21.253","realm":"iwa_realm","priority":"29","message":" Authentication failed from 10.212.21.253: user 'vwv149' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113","timestamp":1460683866000,"source.type":"bluecoat"}
+{"eid":"vwv149","original_string":"<29>Apr 14 20:31:06 ProxySG: 250017 Authentication failed from 10.212.21.253: user 'vwv149' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113","event_type":"authentication failure","event_code":"250017","designated_host":"10.212.21.253","realm":"AD_ldap","priority":"29","message":" Authentication failed from 10.212.21.253: user 'vwv149' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113","timestamp":1460683866000,"source.type":"bluecoat"}
+{"eid":"XGZ521","original_string":"<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.119.223.52: user 'XGZ521' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113","event_type":"authentication failure","event_code":"250017","designated_host":"10.119.223.52","realm":"iwa_realm","priority":"29","message":" Authentication failed from 10.119.223.52: user 'XGZ521' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113","timestamp":1460683867000,"source.type":"bluecoat"}
+{"eid":"LOCAL\\wjs310","original_string":"<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.212.29.36: user 'LOCAL\\wjs310' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113","event_type":"authentication failure","event_code":"250017","designated_host":"10.212.29.36","realm":"iwa_realm","priority":"29","message":" Authentication failed from 10.212.29.36: user 'LOCAL\\wjs310' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113","timestamp":1460683867000,"source.type":"bluecoat"}
+{"eid":"kon313","original_string":"<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.219.15.104: user 'kon313' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113","event_type":"authentication failure","event_code":"250017","designated_host":"10.219.15.104","realm":"iwa_realm","priority":"29","message":" Authentication failed from 10.219.15.104: user 'kon313' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113","timestamp":1460683867000,"source.type":"bluecoat"}
+{"eid":"kon313","original_string":"<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.219.15.104: user 'kon313' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113","event_type":"authentication failure","event_code":"250017","designated_host":"10.219.15.104","realm":"AD_ldap","priority":"29","message":" Authentication failed from 10.219.15.104: user 'kon313' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113","timestamp":1460683867000,"source.type":"bluecoat"}
+{"eid":"kon313","original_string":"<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.219.15.104: user 'kon313' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113","event_type":"authentication failure","event_code":"250017","designated_host":"10.219.15.104","realm":"iwa_realm","priority":"29","message":" Authentication failed from 10.219.15.104: user 'kon313' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113","timestamp":1460683867000,"source.type":"bluecoat"}
+{"original_string":"<29>Apr 14 20:31:07 ProxySG: 250018 LDAP: invalid credentials: reason: '80090308: LdapErr: DSID-0C0903AA, comment: AcceptSecurityContext error, data 52e, v1772' dn: 'CN=WJS310,OU=Developers,OU=All Users,DC=cof,DC=ds,DC=capitalone,DC=com' realm: 'AD_ldap'(2425130) NORMAL_EVENT realm_ldap.cpp 2833","event_type":"authentication failure","event_code":"250018","realm":"AD_ldap","priority":"29","message":" LDAP: invalid credentials: reason: '80090308: LdapErr: DSID-0C0903AA, comment: AcceptSecurityContext error, data 52e, v1772' dn: 'CN=WJS310,OU=Developers,OU=All Users,DC=cof,DC=ds,DC=capitalone,DC=com' realm: 'AD_ldap'(2425130) NORMAL_EVENT realm_ldap.cpp 2833","timestamp":1460683867000,"source.type":"bluecoat"}
+{"eid":"WJS310","original_string":"<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.212.29.36: user 'WJS310' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113","event_type":"authentication failure","event_code":"250017","designated_host":"10.212.29.36","realm":"AD_ldap","priority":"29","message":" Authentication failed from 10.212.29.36: user 'WJS310' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113","timestamp":1460683867000,"source.type":"bluecoat"}
+{"eid":"WJS310","original_string":"<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.212.29.36: user 'WJS310' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113","event_type":"authentication failure","event_code":"250017","designated_host":"10.212.29.36","realm":"AD_ldap","priority":"29","message":" Authentication failed from 10.212.29.36: user 'WJS310' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113","timestamp":1460683867000,"source.type":"bluecoat"}
+{"eid":"vwv149","original_string":"<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.212.21.253: user 'vwv149' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113","event_type":"authentication failure","event_code":"250017","designated_host":"10.212.21.253","realm":"AD_ldap","priority":"29","message":" Authentication failed from 10.212.21.253: user 'vwv149' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113","timestamp":1460683867000,"source.type":"bluecoat"}
+{"eid":"vwv149","original_string":"<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.212.21.253: user 'vwv149' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113","event_type":"authentication failure","event_code":"250017","designated_host":"10.212.21.253","realm":"AD_ldap","priority":"29","message":" Authentication failed from 10.212.21.253: user 'vwv149' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113","timestamp":1460683867000,"source.type":"bluecoat"}
+{"eid":"LOCAL\\yaw983","original_string":"<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.118.216.222: user 'LOCAL\\yaw983' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113","event_type":"authentication failure","event_code":"250017","designated_host":"10.118.216.222","realm":"iwa_realm","priority":"29","message":" Authentication failed from 10.118.216.222: user 'LOCAL\\yaw983' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113","timestamp":1460683867000,"source.type":"bluecoat"}
+{"eid":"yaw983","original_string":"<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.118.216.222: user 'yaw983' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113","event_type":"authentication failure","event_code":"250017","designated_host":"10.118.216.222","realm":"iwa_realm","priority":"29","message":" Authentication failed from 10.118.216.222: user 'yaw983' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113","timestamp":1460683867000,"source.type":"bluecoat"}
+{"eid":"yaw983","original_string":"<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.118.216.222: user 'yaw983' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113","event_type":"authentication failure","event_code":"250017","designated_host":"10.118.216.222","realm":"AD_ldap","priority":"29","message":" Authentication failed from 10.118.216.222: user 'yaw983' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113","timestamp":1460683867000,"source.type":"bluecoat"}
+{"eid":"qwn225","original_string":"<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.115.220.223: user 'qwn225' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113","event_type":"authentication failure","event_code":"250017","designated_host":"10.115.220.223","realm":"AD_ldap","priority":"29","message":" Authentication failed from 10.115.220.223: user 'qwn225' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113","timestamp":1460683867000,"source.type":"bluecoat"}
+{"eid":"ags432","original_string":"<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.114.217.29: user 'ags432' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113","event_type":"authentication failure","event_code":"250017","designated_host":"10.114.217.29","realm":"iwa_realm","priority":"29","message":" Authentication failed from 10.114.217.29: user 'ags432' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113","timestamp":1460683867000,"source.type":"bluecoat"}
+{"eid":"qwn225","original_string":"<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.115.220.223: user 'qwn225' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113","event_type":"authentication failure","event_code":"250017","designated_host":"10.115.220.223","realm":"iwa_realm","priority":"29","message":" Authentication failed from 10.115.220.223: user 'qwn225' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113","timestamp":1460683867000,"source.type":"bluecoat"}
+{"eid":"LOCAL\\u62206","original_string":"<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.113.216.196: user 'LOCAL\\u62206' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113","event_type":"authentication failure","event_code":"250017","designated_host":"10.113.216.196","realm":"iwa_realm","priority":"29","message":" Authentication failed from 10.113.216.196: user 'LOCAL\\u62206' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113","timestamp":1460683867000,"source.type":"bluecoat"}
+{"eid":"qwn225","original_string":"<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.115.220.223: user 'qwn225' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113","event_type":"authentication failure","event_code":"250017","designated_host":"10.115.220.223","realm":"AD_ldap","priority":"29","message":" Authentication failed from 10.115.220.223: user 'qwn225' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113","timestamp":1460683867000,"source.type":"bluecoat"}
+{"original_string":"<29>Apr 14 20:31:07 ProxySG: 250001 LDAP: Authentication failed from 10.113.216.196: no such user in realm 'AD_ldap'(102089) NORMAL_EVENT realm_ldap.cpp 2634","event_type":"authentication failure","event_code":"250001","designated_host":"10.113.216.196","realm":"AD_ldap","priority":"29","message":" LDAP: Authentication failed from 10.113.216.196: no such user in realm 'AD_ldap'(102089) NORMAL_EVENT realm_ldap.cpp 2634","timestamp":1460683867000,"source.type":"bluecoat"}
+{"eid":"ags432","original_string":"<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.114.217.29: user 'ags432' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113","event_type":"authentication failure","event_code":"250017","designated_host":"10.114.217.29","realm":"iwa_realm","priority":"29","message":" Authentication failed from 10.114.217.29: user 'ags432' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113","timestamp":1460683867000,"source.type":"bluecoat"}
+{"eid":"LOCAL\\fjl928","original_string":"<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.218.165.248: user 'LOCAL\\fjl928' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113","event_type":"authentication failure","event_code":"250017","designated_host":"10.218.165.248","realm":"iwa_realm","priority":"29","message":" Authentication failed from 10.218.165.248: user 'LOCAL\\fjl928' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113","timestamp":1460683867000,"source.type":"bluecoat"}
+{"eid":"fjl928","original_string":"<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.218.165.248: user 'fjl928' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113","event_type":"authentication failure","event_code":"250017","designated_host":"10.218.165.248","realm":"iwa_realm","priority":"29","message":" Authentication failed from 10.218.165.248: user 'fjl928' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113","timestamp":1460683867000,"source.type":"bluecoat"}
+{"eid":"vwv149","original_string":"<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.212.21.253: user 'vwv149' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113","event_type":"authentication failure","event_code":"250017","designated_host":"10.212.21.253","realm":"AD_ldap","priority":"29","message":" Authentication failed from 10.212.21.253: user 'vwv149' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113","timestamp":1460683867000,"source.type":"bluecoat"}
+{"eid":"vwv149","original_string":"<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.212.21.253: user 'vwv149' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113","event_type":"authentication failure","event_code":"250017","designated_host":"10.212.21.253","realm":"iwa_realm","priority":"29","message":" Authentication failed from 10.212.21.253: user 'vwv149' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113","timestamp":1460683867000,"source.type":"bluecoat"}
+{"eid":"vwv149","original_string":"<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.212.21.253: user 'vwv149' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113","event_type":"authentication failure","event_code":"250017","designated_host":"10.212.21.253","realm":"AD_ldap","priority":"29","message":" Authentication failed from 10.212.21.253: user 'vwv149' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113","timestamp":1460683867000,"source.type":"bluecoat"}
+{"eid":"ugs662","original_string":"<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.152.102.72: user 'ugs662' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113","event_type":"authentication failure","event_code":"250017","designated_host":"10.152.102.72","realm":"iwa_realm","priority":"29","message":" Authentication failed from 10.152.102.72: user 'ugs662' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113","timestamp":1460683867000,"source.type":"bluecoat"}
+{"eid":"XGZ521","original_string":"<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.119.223.52: user 'XGZ521' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113","event_type":"authentication failure","event_code":"250017","designated_host":"10.119.223.52","realm":"iwa_realm","priority":"29","message":" Authentication failed from 10.119.223.52: user 'XGZ521' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113","timestamp":1460683867000,"source.type":"bluecoat"}
+{"eid":"XGZ521","original_string":"<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.119.223.52: user 'XGZ521' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113","event_type":"authentication failure","event_code":"250017","designated_host":"10.119.223.52","realm":"AD_ldap","priority":"29","message":" Authentication failed from 10.119.223.52: user 'XGZ521' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113","timestamp":1460683867000,"source.type":"bluecoat"}
+{"eid":"kon313","original_string":"<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.219.15.104: user 'kon313' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113","event_type":"authentication failure","event_code":"250017","designated_host":"10.219.15.104","realm":"iwa_realm","priority":"29","message":" Authentication failed from 10.219.15.104: user 'kon313' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113","timestamp":1460683867000,"source.type":"bluecoat"}
+{"eid":"kon313","original_string":"<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.219.15.104: user 'kon313' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113","event_type":"authentication failure","event_code":"250017","designated_host":"10.219.15.104","realm":"AD_ldap","priority":"29","message":" Authentication failed from 10.219.15.104: user 'kon313' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113","timestamp":1460683867000,"source.type":"bluecoat"}
+{"eid":"XGZ521","original_string":"<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.119.223.52: user 'XGZ521' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113","event_type":"authentication failure","event_code":"250017","designated_host":"10.119.223.52","realm":"AD_ldap","priority":"29","message":" Authentication failed from 10.119.223.52: user 'XGZ521' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113","timestamp":1460683867000,"source.type":"bluecoat"}
+{"eid":"fjl928","original_string":"<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.218.165.248: user 'fjl928' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113","event_type":"authentication failure","event_code":"250017","designated_host":"10.218.165.248","realm":"AD_ldap","priority":"29","message":" Authentication failed from 10.218.165.248: user 'fjl928' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113","timestamp":1460683867000,"source.type":"bluecoat"}
+{"eid":"fjl928","original_string":"<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.218.165.248: user 'fjl928' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113","event_type":"authentication failure","event_code":"250017","designated_host":"10.218.165.248","realm":"AD_ldap","priority":"29","message":" Authentication failed from 10.218.165.248: user 'fjl928' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113","timestamp":1460683867000,"source.type":"bluecoat"}
+{"eid":"UZL193","original_string":"<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.118.29.228: user 'UZL193' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113","event_type":"authentication failure","event_code":"250017","designated_host":"10.118.29.228","realm":"AD_ldap","priority":"29","message":" Authentication failed from 10.118.29.228: user 'UZL193' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113","timestamp":1460683867000,"source.type":"bluecoat"}
+{"eid":"UZL193","original_string":"<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.118.29.228: user 'UZL193' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113","event_type":"authentication failure","event_code":"250017","designated_host":"10.118.29.228","realm":"AD_ldap","priority":"29","message":" Authentication failed from 10.118.29.228: user 'UZL193' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113","timestamp":1460683867000,"source.type":"bluecoat"}
+{"eid":"LOCAL\\epl857","original_string":"<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.210.223.65: user 'LOCAL\\epl857' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113","event_type":"authentication failure","event_code":"250017","designated_host":"10.210.223.65","realm":"iwa_realm","priority":"29","message":" Authentication failed from 10.210.223.65: user 'LOCAL\\epl857' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113","timestamp":1460683867000,"source.type":"bluecoat"}
+{"eid":"UZL193","original_string":"<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.118.29.228: user 'UZL193' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113","event_type":"authentication failure","event_code":"250017","designated_host":"10.118.29.228","realm":"AD_ldap","priority":"29","message":" Authentication failed from 10.118.29.228: user 'UZL193' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113","timestamp":1460683867000,"source.type":"bluecoat"}
+{"eid":"UZL193","original_string":"<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.118.29.228: user 'UZL193' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113","event_type":"authentication failure","event_code":"250017","designated_host":"10.118.29.228","realm":"AD_ldap","priority":"29","message":" Authentication failed from 10.118.29.228: user 'UZL193' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113","timestamp":1460683867000,"source.type":"bluecoat"}
+{"eid":"vwv149","original_string":"<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.212.21.253: user 'vwv149' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113","event_type":"authentication failure","event_code":"250017","designated_host":"10.212.21.253","realm":"iwa_realm","priority":"29","message":" Authentication failed from 10.212.21.253: user 'vwv149' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113","timestamp":1460683867000,"source.type":"bluecoat"}
+{"eid":"vwv149","original_string":"<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.212.21.253: user 'vwv149' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113","event_type":"authentication failure","event_code":"250017","designated_host":"10.212.21.253","realm":"AD_ldap","priority":"29","message":" Authentication failed from 10.212.21.253: user 'vwv149' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113","timestamp":1460683867000,"source.type":"bluecoat"}
+{"eid":"XGZ521","original_string":"<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.119.223.52: user 'XGZ521' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113","event_type":"authentication failure","event_code":"250017","designated_host":"10.119.223.52","realm":"iwa_realm","priority":"29","message":" Authentication failed from 10.119.223.52: user 'XGZ521' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113","timestamp":1460683867000,"source.type":"bluecoat"}
+{"eid":"ugs662","original_string":"<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.152.102.72: user 'ugs662' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113","event_type":"authentication failure","event_code":"250017","designated_host":"10.152.102.72","realm":"AD_ldap","priority":"29","message":" Authentication failed from 10.152.102.72: user 'ugs662' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113","timestamp":1460683867000,"source.type":"bluecoat"}
+{"eid":"epl857","original_string":"<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.210.223.65: user 'epl857' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113","event_type":"authentication failure","event_code":"250017","designated_host":"10.210.223.65","realm":"iwa_realm","priority":"29","message":" Authentication failed from 10.210.223.65: user 'epl857' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113","timestamp":1460683867000,"source.type":"bluecoat"}
+{"eid":"UZL193","original_string":"<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.118.29.228: user 'UZL193' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113","event_type":"authentication failure","event_code":"250017","designated_host":"10.118.29.228","realm":"iwa_realm","priority":"29","message":" Authentication failed from 10.118.29.228: user 'UZL193' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113","timestamp":1460683867000,"source.type":"bluecoat"}
+{"eid":"UZL193","original_string":"<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.118.29.228: user 'UZL193' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113","event_type":"authentication failure","event_code":"250017","designated_host":"10.118.29.228","realm":"iwa_realm","priority":"29","message":" Authentication failed from 10.118.29.228: user 'UZL193' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113","timestamp":1460683867000,"source.type":"bluecoat"}
+{"eid":"UZL193","original_string":"<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.118.29.228: user 'UZL193' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113","event_type":"authentication failure","event_code":"250017","designated_host":"10.118.29.228","realm":"AD_ldap","priority":"29","message":" Authentication failed from 10.118.29.228: user 'UZL193' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113","timestamp":1460683867000,"source.type":"bluecoat"}
+{"eid":"epl857","original_string":"<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.210.223.65: user 'epl857' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113","event_type":"authentication failure","event_code":"250017","designated_host":"10.210.223.65","realm":"AD_ldap","priority":"29","message":" Authentication failed from 10.210.223.65: user 'epl857' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113","timestamp":1460683867000,"source.type":"bluecoat"}
+{"eid":"epl857","original_string":"<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.210.223.65: user 'epl857' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113","event_type":"authentication failure","event_code":"250017","designated_host":"10.210.223.65","realm":"AD_ldap","priority":"29","message":" Authentication failed from 10.210.223.65: user 'epl857' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113","timestamp":1460683867000,"source.type":"bluecoat"}
+{"eid":"sdq302","original_string":"<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.127.216.106: user 'sdq302' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113","event_type":"authentication failure","event_code":"250017","designated_host":"10.127.216.106","realm":"iwa_realm","priority":"29","message":" Authentication failed from 10.127.216.106: user 'sdq302' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113","timestamp":1460683867000,"source.type":"bluecoat"}
+{"eid":"UZL193","original_string":"<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.118.29.228: user 'UZL193' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113","event_type":"authentication failure","event_code":"250017","designated_host":"10.118.29.228","realm":"iwa_realm","priority":"29","message":" Authentication failed from 10.118.29.228: user 'UZL193' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113","timestamp":1460683867000,"source.type":"bluecoat"}
+{"eid":"UZL193","original_string":"<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.118.29.228: user 'UZL193' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113","event_type":"authentication failure","event_code":"250017","designated_host":"10.118.29.228","realm":"iwa_realm","priority":"29","message":" Authentication failed from 10.118.29.228: user 'UZL193' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113","timestamp":1460683867000,"source.type":"bluecoat"}
+{"eid":"UZL193","original_string":"<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.118.29.228: user 'UZL193' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113","event_type":"authentication failure","event_code":"250017","designated_host":"10.118.29.228","realm":"AD_ldap","priority":"29","message":" Authentication failed from 10.118.29.228: user 'UZL193' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113","timestamp":1460683867000,"source.type":"bluecoat"}
+{"eid":"sdq302","original_string":"<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.127.216.106: user 'sdq302' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113","event_type":"authentication failure","event_code":"250017","designated_host":"10.127.216.106","realm":"iwa_realm","priority":"29","message":" Authentication failed from 10.127.216.106: user 'sdq302' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113","timestamp":1460683867000,"source.type":"bluecoat"}
+{"eid":"UZL193","original_string":"<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.118.29.228: user 'UZL193' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113","event_type":"authentication failure","event_code":"250017","designated_host":"10.118.29.228","realm":"iwa_realm","priority":"29","message":" Authentication failed from 10.118.29.228: user 'UZL193' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113","timestamp":1460683867000,"source.type":"bluecoat"}
+{"eid":"UZL193","original_string":"<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.118.29.228: user 'UZL193' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113","event_type":"authentication failure","event_code":"250017","designated_host":"10.118.29.228","realm":"AD_ldap","priority":"29","message":" Authentication failed from 10.118.29.228: user 'UZL193' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113","timestamp":1460683867000,"source.type":"bluecoat"}
+{"eid":"UZL193","original_string":"<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.118.29.228: user 'UZL193' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113","event_type":"authentication failure","event_code":"250017","designated_host":"10.118.29.228","realm":"iwa_realm","priority":"29","message":" Authentication failed from 10.118.29.228: user 'UZL193' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113","timestamp":1460683867000,"source.type":"bluecoat"}
+{"eid":"UZL193","original_string":"<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.118.29.228: user 'UZL193' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113","event_type":"authentication failure","event_code":"250017","designated_host":"10.118.29.228","realm":"AD_ldap","priority":"29","message":" Authentication failed from 10.118.29.228: user 'UZL193' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113","timestamp":1460683867000,"source.type":"bluecoat"}
+{"original_string":"<29>Apr 14 20:31:07 ProxySG: 250018 LDAP: invalid credentials: reason: '80090308: LdapErr: DSID-0C0903AA, comment: AcceptSecurityContext error, data 52e, v1772' dn: 'CN=UUA398,OU=Developers,OU=All Users,DC=cof,DC=ds,DC=capitalone,DC=com' realm: 'AD_ldap'(2425130) NORMAL_EVENT realm_ldap.cpp 2833","event_type":"authentication failure","event_code":"250018","realm":"AD_ldap","priority":"29","message":" LDAP: invalid credentials: reason: '80090308: LdapErr: DSID-0C0903AA, comment: AcceptSecurityContext error, data 52e, v1772' dn: 'CN=UUA398,OU=Developers,OU=All Users,DC=cof,DC=ds,DC=capitalone,DC=com' realm: 'AD_ldap'(2425130) NORMAL_EVENT realm_ldap.cpp 2833","timestamp":1460683867000,"source.type":"bluecoat"}
+{"eid":"uua398","original_string":"<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.118.219.193: user 'uua398' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113","event_type":"authentication failure","event_code":"250017","designated_host":"10.118.219.193","realm":"AD_ldap","priority":"29","message":" Authentication failed from 10.118.219.193: user 'uua398' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113","timestamp":1460683867000,"source.type":"bluecoat"}
+{"eid":"vwv149","original_string":"<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.212.21.253: user 'vwv149' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113","event_type":"authentication failure","event_code":"250017","designated_host":"10.212.21.253","realm":"iwa_realm","priority":"29","message":" Authentication failed from 10.212.21.253: user 'vwv149' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113","timestamp":1460683867000,"source.type":"bluecoat"}
+{"eid":"LOCAL\\uzl193","original_string":"<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.118.29.228: user 'LOCAL\\uzl193' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113","event_type":"authentication failure","event_code":"250017","designated_host":"10.118.29.228","realm":"iwa_realm","priority":"29","message":" Authentication failed from 10.118.29.228: user 'LOCAL\\uzl193' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113","timestamp":1460683867000,"source.type":"bluecoat"}
+{"eid":"UZL193","original_string":"<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.118.29.228: user 'UZL193' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113","event_type":"authentication failure","event_code":"250017","designated_host":"10.118.29.228","realm":"iwa_realm","priority":"29","message":" Authentication failed from 10.118.29.228: user 'UZL193' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113","timestamp":1460683867000,"source.type":"bluecoat"}
+{"eid":"vwv149","original_string":"<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.212.21.253: user 'vwv149' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113","event_type":"authentication failure","event_code":"250017","designated_host":"10.212.21.253","realm":"iwa_realm","priority":"29","message":" Authentication failed from 10.212.21.253: user 'vwv149' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113","timestamp":1460683867000,"source.type":"bluecoat"}
+{"eid":"uua398","original_string":"<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.118.219.193: user 'uua398' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113","event_type":"authentication failure","event_code":"250017","designated_host":"10.118.219.193","realm":"AD_ldap","priority":"29","message":" Authentication failed from 10.118.219.193: user 'uua398' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113","timestamp":1460683867000,"source.type":"bluecoat"}
+{"eid":"kon313","original_string":"<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.219.15.104: user 'kon313' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113","event_type":"authentication failure","event_code":"250017","designated_host":"10.219.15.104","realm":"iwa_realm","priority":"29","message":" Authentication failed from 10.219.15.104: user 'kon313' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113","timestamp":1460683867000,"source.type":"bluecoat"}
+{"eid":"kon313","original_string":"<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.219.15.104: user 'kon313' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113","event_type":"authentication failure","event_code":"250017","designated_host":"10.219.15.104","realm":"AD_ldap","priority":"29","message":" Authentication failed from 10.219.15.104: user 'kon313' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113","timestamp":1460683867000,"source.type":"bluecoat"}
+{"eid":"u62206","original_string":"<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.113.216.196: user 'u62206' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113","event_type":"authentication failure","event_code":"250017","designated_host":"10.113.216.196","realm":"iwa_realm","priority":"29","message":" Authentication failed from 10.113.216.196: user 'u62206' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113","timestamp":1460683867000,"source.type":"bluecoat"}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/645d8292/metron-platform/metron-integration-test/src/main/resources/sample/data/SampleParsed/SquidExampleParsed
----------------------------------------------------------------------
diff --git a/metron-platform/metron-integration-test/src/main/resources/sample/data/SampleParsed/SquidExampleParsed b/metron-platform/metron-integration-test/src/main/resources/sample/data/SampleParsed/SquidExampleParsed
new file mode 100644
index 0000000..9643c25
--- /dev/null
+++ b/metron-platform/metron-integration-test/src/main/resources/sample/data/SampleParsed/SquidExampleParsed
@@ -0,0 +1,2 @@
+{"elapsed":161,"code":200,"ip_dst_addr":"199.27.79.73","original_string":"1461576382.642    161 127.0.0.1 TCP_MISS\/200 103701 GET http:\/\/www.cnn.com\/ - DIRECT\/199.27.79.73 text\/html","method":"GET","bytes":103701,"action":"TCP_MISS","ip_src_addr":"127.0.0.1","url":"cnn.com","timestamp":1461576382642,"source.type":"squid"}
+{"elapsed":159,"code":200,"ip_dst_addr":"66.210.41.9","original_string":"1461576442.228    159 127.0.0.1 TCP_MISS\/200 137183 GET http:\/\/www.nba.com\/ - DIRECT\/66.210.41.9 text\/html","method":"GET","bytes":137183,"action":"TCP_MISS","ip_src_addr":"127.0.0.1","url":"nba.com","timestamp":1461576442228,"source.type":"squid"}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/645d8292/metron-platform/metron-integration-test/src/main/resources/sample/patterns/test
----------------------------------------------------------------------
diff --git a/metron-platform/metron-integration-test/src/main/resources/sample/patterns/test b/metron-platform/metron-integration-test/src/main/resources/sample/patterns/test
new file mode 100644
index 0000000..a88a255
--- /dev/null
+++ b/metron-platform/metron-integration-test/src/main/resources/sample/patterns/test
@@ -0,0 +1,2 @@
+YAF_TIME_FORMAT %{YEAR:UNWANTED}-%{MONTHNUM:UNWANTED}-%{MONTHDAY:UNWANTED}[T ]%{HOUR:UNWANTED}:%{MINUTE:UNWANTED}:%{SECOND:UNWANTED}
+YAF_DELIMITED %{NUMBER:start_time}\|%{YAF_TIME_FORMAT:end_time}\|%{SPACE:UNWANTED}%{BASE10NUM:duration}\|%{SPACE:UNWANTED}%{BASE10NUM:rtt}\|%{SPACE:UNWANTED}%{INT:protocol}\|%{SPACE:UNWANTED}%{IP:ip_src_addr}\|%{SPACE:UNWANTED}%{INT:ip_src_port}\|%{SPACE:UNWANTED}%{IP:ip_dst_addr}\|%{SPACE:UNWANTED}%{INT:ip_dst_port}\|%{SPACE:UNWANTED}%{DATA:iflags}\|%{SPACE:UNWANTED}%{DATA:uflags}\|%{SPACE:UNWANTED}%{DATA:riflags}\|%{SPACE:UNWANTED}%{DATA:ruflags}\|%{SPACE:UNWANTED}%{WORD:isn}\|%{SPACE:UNWANTED}%{DATA:risn}\|%{SPACE:UNWANTED}%{DATA:tag}\|%{GREEDYDATA:rtag}\|%{SPACE:UNWANTED}%{INT:pkt}\|%{SPACE:UNWANTED}%{INT:oct}\|%{SPACE:UNWANTED}%{INT:rpkt}\|%{SPACE:UNWANTED}%{INT:roct}\|%{SPACE:UNWANTED}%{INT:app}\|%{GREEDYDATA:end_reason}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/645d8292/metron-platform/metron-parsers/pom.xml
----------------------------------------------------------------------
diff --git a/metron-platform/metron-parsers/pom.xml b/metron-platform/metron-parsers/pom.xml
index 0462ba9..2630ef3 100644
--- a/metron-platform/metron-parsers/pom.xml
+++ b/metron-platform/metron-parsers/pom.xml
@@ -48,6 +48,17 @@
             </exclusions>
         </dependency>
         <dependency>
+            <groupId>org.apache.hadoop</groupId>
+            <artifactId>hadoop-hdfs</artifactId>
+            <version>${global_hadoop_version}</version>
+            <exclusions>
+                <exclusion>
+                    <artifactId>servlet-api</artifactId>
+                    <groupId>javax.servlet</groupId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+        <dependency>
             <groupId>org.apache.hbase</groupId>
             <artifactId>hbase-client</artifactId>
             <version>${global_hbase_version}</version>

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/645d8292/metron-platform/metron-parsers/src/main/flux/bluecoat/remote.yaml
----------------------------------------------------------------------
diff --git a/metron-platform/metron-parsers/src/main/flux/bluecoat/remote.yaml b/metron-platform/metron-parsers/src/main/flux/bluecoat/remote.yaml
new file mode 100644
index 0000000..1f2cd14
--- /dev/null
+++ b/metron-platform/metron-parsers/src/main/flux/bluecoat/remote.yaml
@@ -0,0 +1,71 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#     http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+name: "bluecoat"
+config:
+    topology.workers: 1
+
+components:
+    -   id: "parser"
+        className: "org.apache.metron.parsers.bluecoat.BasicBluecoatParser"
+    -   id: "writer"
+        className: "org.apache.metron.parsers.writer.KafkaWriter"
+        constructorArgs:
+            - "${kafka.broker}"
+    -   id: "zkHosts"
+        className: "storm.kafka.ZkHosts"
+        constructorArgs:
+            - "${kafka.zk}"
+    -   id: "kafkaConfig"
+        className: "storm.kafka.SpoutConfig"
+        constructorArgs:
+            # zookeeper hosts
+            - ref: "zkHosts"
+            # topic name
+            - "bluecoat"
+            # zk root
+            - ""
+            # id
+            - "bluecoat"
+        properties:
+            -   name: "ignoreZkOffsets"
+                value: true
+            -   name: "startOffsetTime"
+                value: -1
+            -   name: "socketTimeoutMs"
+                value: 1000000
+
+spouts:
+    -   id: "kafkaSpout"
+        className: "storm.kafka.KafkaSpout"
+        constructorArgs:
+            - ref: "kafkaConfig"
+
+bolts:
+    -   id: "parserBolt"
+        className: "org.apache.metron.parsers.bolt.ParserBolt"
+        constructorArgs:
+            - "${kafka.zk}"
+            - "bluecoat"
+            - ref: "parser"
+            - ref: "writer"
+
+streams:
+    -   name: "spout -> bolt"
+        from: "kafkaSpout"
+        to: "parserBolt"
+        grouping:
+            type: SHUFFLE

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/645d8292/metron-platform/metron-parsers/src/main/flux/bluecoat/test.yaml
----------------------------------------------------------------------
diff --git a/metron-platform/metron-parsers/src/main/flux/bluecoat/test.yaml b/metron-platform/metron-parsers/src/main/flux/bluecoat/test.yaml
new file mode 100644
index 0000000..f1016e6
--- /dev/null
+++ b/metron-platform/metron-parsers/src/main/flux/bluecoat/test.yaml
@@ -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.
+
+name: "bluecoat-test"
+config:
+    topology.workers: 1
+
+
+components:
+    -   id: "parser"
+        className: "org.apache.metron.parsers.bluecoat.BasicBluecoatParser"
+    -   id: "writer"
+        className: "org.apache.metron.parsers.writer.KafkaWriter"
+        constructorArgs:
+            - "${kafka.broker}"
+    -   id: "zkHosts"
+        className: "storm.kafka.ZkHosts"
+        constructorArgs:
+            - "${kafka.zk}"
+    -   id: "kafkaConfig"
+        className: "storm.kafka.SpoutConfig"
+        constructorArgs:
+            # zookeeper hosts
+            - ref: "zkHosts"
+            # topic name
+            - "bluecoat"
+            # zk root
+            - ""
+            # id
+            - "bluecoat"
+        properties:
+            -   name: "ignoreZkOffsets"
+                value: true
+            -   name: "startOffsetTime"
+                value: -2
+            -   name: "socketTimeoutMs"
+                value: 1000000
+
+spouts:
+    -   id: "kafkaSpout"
+        className: "storm.kafka.KafkaSpout"
+        constructorArgs:
+            - ref: "kafkaConfig"
+
+bolts:
+    -   id: "parserBolt"
+        className: "org.apache.metron.parsers.bolt.ParserBolt"
+        constructorArgs:
+            - "${kafka.zk}"
+            - "bluecoat"
+            - ref: "parser"
+            - ref: "writer"
+
+streams:
+    -   name: "spout -> bolt"
+        from: "kafkaSpout"
+        to: "parserBolt"
+        grouping:
+            type: SHUFFLE

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/645d8292/metron-platform/metron-parsers/src/main/flux/squid/remote.yaml
----------------------------------------------------------------------
diff --git a/metron-platform/metron-parsers/src/main/flux/squid/remote.yaml b/metron-platform/metron-parsers/src/main/flux/squid/remote.yaml
new file mode 100644
index 0000000..119f03e
--- /dev/null
+++ b/metron-platform/metron-parsers/src/main/flux/squid/remote.yaml
@@ -0,0 +1,78 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#     http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+name: "squid"
+config:
+    topology.workers: 1
+
+components:
+    -   id: "parser"
+        className: "org.apache.metron.parsers.GrokParser"
+        constructorArgs:
+            - "/apps/metron/patterns/squid"
+            - "SQUID_DELIMITED"
+        configMethods:
+            -   name: "withTimestampField"
+                args:
+                    - "timestamp"
+    -   id: "writer"
+        className: "org.apache.metron.parsers.writer.KafkaWriter"
+        constructorArgs:
+            - "${kafka.broker}"
+    -   id: "zkHosts"
+        className: "storm.kafka.ZkHosts"
+        constructorArgs:
+            - "${kafka.zk}"
+    -   id: "kafkaConfig"
+        className: "storm.kafka.SpoutConfig"
+        constructorArgs:
+            # zookeeper hosts
+            - ref: "zkHosts"
+            # topic name
+            - "squid"
+            # zk root
+            - ""
+            # id
+            - "squid"
+        properties:
+            -   name: "ignoreZkOffsets"
+                value: true
+            -   name: "startOffsetTime"
+                value: -1
+            -   name: "socketTimeoutMs"
+                value: 1000000
+
+spouts:
+    -   id: "kafkaSpout"
+        className: "storm.kafka.KafkaSpout"
+        constructorArgs:
+            - ref: "kafkaConfig"
+
+bolts:
+    -   id: "parserBolt"
+        className: "org.apache.metron.parsers.bolt.ParserBolt"
+        constructorArgs:
+            - "${kafka.zk}"
+            - "squid"
+            - ref: "parser"
+            - ref: "writer"
+
+streams:
+    -   name: "spout -> bolt"
+        from: "kafkaSpout"
+        to: "parserBolt"
+        grouping:
+            type: SHUFFLE

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/645d8292/metron-platform/metron-parsers/src/main/flux/squid/test.yaml
----------------------------------------------------------------------
diff --git a/metron-platform/metron-parsers/src/main/flux/squid/test.yaml b/metron-platform/metron-parsers/src/main/flux/squid/test.yaml
new file mode 100644
index 0000000..77893d2
--- /dev/null
+++ b/metron-platform/metron-parsers/src/main/flux/squid/test.yaml
@@ -0,0 +1,78 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#     http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+name: "squid"
+config:
+    topology.workers: 1
+
+components:
+    -   id: "parser"
+        className: "org.apache.metron.parsers.GrokParser"
+        constructorArgs:
+            - "../metron-parsers/src/main/resources/patterns/squid"
+            - "SQUID_DELIMITED"
+        configMethods:
+            -   name: "withTimestampField"
+                args:
+                    - "timestamp"
+    -   id: "writer"
+        className: "org.apache.metron.parsers.writer.KafkaWriter"
+        constructorArgs:
+            - "${kafka.broker}"
+    -   id: "zkHosts"
+        className: "storm.kafka.ZkHosts"
+        constructorArgs:
+            - "${kafka.zk}"
+    -   id: "kafkaConfig"
+        className: "storm.kafka.SpoutConfig"
+        constructorArgs:
+            # zookeeper hosts
+            - ref: "zkHosts"
+            # topic name
+            - "squid"
+            # zk root
+            - ""
+            # id
+            - "squid"
+        properties:
+            -   name: "ignoreZkOffsets"
+                value: false
+            -   name: "startOffsetTime"
+                value: -2
+            -   name: "socketTimeoutMs"
+                value: 1000000
+
+spouts:
+    -   id: "kafkaSpout"
+        className: "storm.kafka.KafkaSpout"
+        constructorArgs:
+            - ref: "kafkaConfig"
+
+bolts:
+    -   id: "parserBolt"
+        className: "org.apache.metron.parsers.bolt.ParserBolt"
+        constructorArgs:
+            - "${kafka.zk}"
+            - "squid"
+            - ref: "parser"
+            - ref: "writer"
+
+streams:
+    -   name: "spout -> bolt"
+        from: "kafkaSpout"
+        to: "parserBolt"
+        grouping:
+            type: SHUFFLE

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/645d8292/metron-platform/metron-parsers/src/main/flux/yaf/test.yaml
----------------------------------------------------------------------
diff --git a/metron-platform/metron-parsers/src/main/flux/yaf/test.yaml b/metron-platform/metron-parsers/src/main/flux/yaf/test.yaml
index 0f6031c..e2985b8 100644
--- a/metron-platform/metron-parsers/src/main/flux/yaf/test.yaml
+++ b/metron-platform/metron-parsers/src/main/flux/yaf/test.yaml
@@ -35,9 +35,6 @@ components:
             -   name: "withDateFormat"
                 args:
                     - "yyyy-MM-dd HH:mm:ss.S"
-            -   name: "withMetronHDFSHome"
-                args:
-                    - ""
     -   id: "writer"
         className: "org.apache.metron.parsers.writer.KafkaWriter"
         constructorArgs:


[3/3] incubator-metron git commit: METRON-122 Create generic unit test framework for testing grok statements (merrimanr) closes apache/incubator-metron#96

Posted by rm...@apache.org.
METRON-122 Create generic unit test framework for testing grok statements (merrimanr) closes apache/incubator-metron#96


Project: http://git-wip-us.apache.org/repos/asf/incubator-metron/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-metron/commit/645d8292
Tree: http://git-wip-us.apache.org/repos/asf/incubator-metron/tree/645d8292
Diff: http://git-wip-us.apache.org/repos/asf/incubator-metron/diff/645d8292

Branch: refs/heads/master
Commit: 645d8292b5600d227b51bd80e73982a56d1cc77d
Parents: 28c250d
Author: merrimanr <me...@gmail.com>
Authored: Mon May 2 12:26:05 2016 -0500
Committer: rmerriman <rm...@hortonworks.com>
Committed: Mon May 2 12:26:05 2016 -0500

----------------------------------------------------------------------
 .../common/configuration/EnrichmentConfig.java  |  14 +-
 .../configuration/SensorEnrichmentConfig.java   |   4 +-
 .../enrichment/bolt/EnrichmentSplitterBolt.java |  13 +-
 .../bolt/ThreatIntelSplitterBolt.java           |  14 +-
 .../sample/data/SampleInput/BluecoatSyslog.txt  | 144 +++++++++++++++++++
 .../sample/data/SampleInput/SquidExampleOutput  |   2 +
 .../sample/data/SampleParsed/BluecoatParsed     | 144 +++++++++++++++++++
 .../sample/data/SampleParsed/SquidExampleParsed |   2 +
 .../src/main/resources/sample/patterns/test     |   2 +
 metron-platform/metron-parsers/pom.xml          |  11 ++
 .../src/main/flux/bluecoat/remote.yaml          |  71 +++++++++
 .../src/main/flux/bluecoat/test.yaml            |  72 ++++++++++
 .../src/main/flux/squid/remote.yaml             |  78 ++++++++++
 .../src/main/flux/squid/test.yaml               |  78 ++++++++++
 .../metron-parsers/src/main/flux/yaf/test.yaml  |   3 -
 .../org/apache/metron/parsers/GrokParser.java   |  37 +++--
 .../parsers/bluecoat/BasicBluecoatParser.java   | 101 +++++++++++++
 .../src/main/resources/patterns/squid           |   2 +
 .../apache/metron/parsers/GrokParserTest.java   |  63 +++-----
 .../metron/parsers/SampleGrokParserTest.java    |  82 +++++++++++
 .../apache/metron/parsers/SquidParserTest.java  |  75 ++++++++++
 .../apache/metron/parsers/YafParserTest.java    |  90 ++++++++++++
 .../bluecoat/BasicBluecoatParserTest.java       | 100 +++++++++++++
 .../integration/BluecoatIntegrationTest.java    |  48 +++++++
 .../integration/ParserIntegrationTest.java      |  14 +-
 .../integration/SquidIntegrationTest.java       |  49 +++++++
 pom.xml                                         |   1 +
 27 files changed, 1246 insertions(+), 68 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/645d8292/metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/EnrichmentConfig.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/EnrichmentConfig.java b/metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/EnrichmentConfig.java
index 2ead81e..bcc91fa 100644
--- a/metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/EnrichmentConfig.java
+++ b/metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/EnrichmentConfig.java
@@ -22,6 +22,7 @@ import com.google.common.base.Joiner;
 import org.apache.curator.framework.CuratorFramework;
 import org.apache.metron.common.Constants;
 import org.apache.metron.common.cli.ConfigurationsUtils;
+import org.apache.zookeeper.KeeperException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -96,7 +97,14 @@ public class EnrichmentConfig {
     }
     @Override
     public SensorEnrichmentConfig readConfig(String sensor) throws Exception {
-      return SensorEnrichmentConfig.fromBytes(ConfigurationsUtils.readSensorEnrichmentConfigBytesFromZookeeper(sensor, client));
+      SensorEnrichmentConfig sensorEnrichmentConfig = new SensorEnrichmentConfig();
+      try {
+        sensorEnrichmentConfig = SensorEnrichmentConfig.fromBytes(ConfigurationsUtils.readSensorEnrichmentConfigBytesFromZookeeper(sensor, client));
+      }catch (KeeperException.NoNodeException e) {
+        sensorEnrichmentConfig.setIndex(sensor);
+        sensorEnrichmentConfig.setBatchSize(1);
+      }
+      return sensorEnrichmentConfig;
     }
 
     @Override
@@ -125,6 +133,8 @@ public class EnrichmentConfig {
         fieldMap = config.getThreatIntelFieldMap();
         if(fieldMap!= null) {
           fieldList = fieldMap.get(Constants.SIMPLE_HBASE_THREAT_INTEL);
+        } else {
+          fieldMap = new HashMap<>();
         }
         if(fieldList == null) {
           fieldList = new ArrayList<>();
@@ -140,6 +150,8 @@ public class EnrichmentConfig {
         fieldMap = config.getEnrichmentFieldMap();
         if(fieldMap!= null) {
           fieldList = fieldMap.get(Constants.SIMPLE_HBASE_ENRICHMENT);
+        } else {
+          fieldMap = new HashMap<>();
         }
         if(fieldList == null) {
           fieldList = new ArrayList<>();

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/645d8292/metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/SensorEnrichmentConfig.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/SensorEnrichmentConfig.java b/metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/SensorEnrichmentConfig.java
index bc30327..6a45ec9 100644
--- a/metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/SensorEnrichmentConfig.java
+++ b/metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/SensorEnrichmentConfig.java
@@ -28,8 +28,8 @@ import java.util.Map;
 public class SensorEnrichmentConfig {
 
   private String index;
-  private Map<String, List<String>> enrichmentFieldMap;
-  private Map<String, List<String>> threatIntelFieldMap;
+  private Map<String, List<String>> enrichmentFieldMap = new HashMap<>();
+  private Map<String, List<String>> threatIntelFieldMap = new HashMap<>();
   private Map<String, List<String>> fieldToEnrichmentTypeMap = new HashMap<>();
   private Map<String, List<String>> fieldToThreatIntelTypeMap = new HashMap<>();
   private int batchSize;

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/645d8292/metron-platform/metron-enrichment/src/main/java/org/apache/metron/enrichment/bolt/EnrichmentSplitterBolt.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-enrichment/src/main/java/org/apache/metron/enrichment/bolt/EnrichmentSplitterBolt.java b/metron-platform/metron-enrichment/src/main/java/org/apache/metron/enrichment/bolt/EnrichmentSplitterBolt.java
index e713d69..6b49edb 100644
--- a/metron-platform/metron-enrichment/src/main/java/org/apache/metron/enrichment/bolt/EnrichmentSplitterBolt.java
+++ b/metron-platform/metron-enrichment/src/main/java/org/apache/metron/enrichment/bolt/EnrichmentSplitterBolt.java
@@ -21,6 +21,7 @@ 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.configuration.SensorEnrichmentConfig;
 import org.apache.metron.enrichment.configuration.Enrichment;
 import org.apache.metron.enrichment.utils.EnrichmentUtils;
 import org.apache.metron.common.utils.MessageUtils;
@@ -123,7 +124,17 @@ public class EnrichmentSplitterBolt extends SplitBolt<JSONObject> {
     }
 
     protected Map<String, List<String>> getFieldMap(String sensorType) {
-        return configurations.getSensorEnrichmentConfig(sensorType).getEnrichmentFieldMap();
+        if(sensorType != null) {
+            SensorEnrichmentConfig config = configurations.getSensorEnrichmentConfig(sensorType);
+            if (config != null) {
+                return config.getEnrichmentFieldMap();
+            } else {
+                LOG.error("Unable to retrieve a sensor enrichment config of " + sensorType);
+            }
+        } else {
+            LOG.error("Trying to retrieve a field map with sensor type of null");
+        }
+        return new HashMap<>();
     }
 
     protected String getKeyName(String type, String field) {

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/645d8292/metron-platform/metron-enrichment/src/main/java/org/apache/metron/enrichment/bolt/ThreatIntelSplitterBolt.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-enrichment/src/main/java/org/apache/metron/enrichment/bolt/ThreatIntelSplitterBolt.java b/metron-platform/metron-enrichment/src/main/java/org/apache/metron/enrichment/bolt/ThreatIntelSplitterBolt.java
index 692c327..1429b2c 100644
--- a/metron-platform/metron-enrichment/src/main/java/org/apache/metron/enrichment/bolt/ThreatIntelSplitterBolt.java
+++ b/metron-platform/metron-enrichment/src/main/java/org/apache/metron/enrichment/bolt/ThreatIntelSplitterBolt.java
@@ -17,8 +17,10 @@
  */
 package org.apache.metron.enrichment.bolt;
 
+import org.apache.metron.common.configuration.SensorEnrichmentConfig;
 import org.apache.metron.enrichment.utils.ThreatIntelUtils;
 
+import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 
@@ -30,7 +32,17 @@ public class ThreatIntelSplitterBolt extends EnrichmentSplitterBolt {
 
   @Override
   protected Map<String, List<String>> getFieldMap(String sensorType) {
-    return configurations.getSensorEnrichmentConfig(sensorType).getThreatIntelFieldMap();
+    if (sensorType != null) {
+      SensorEnrichmentConfig config = configurations.getSensorEnrichmentConfig(sensorType);
+      if (config != null) {
+        return config.getThreatIntelFieldMap();
+      } else {
+        LOG.error("Unable to retrieve sensor config: " + sensorType);
+      }
+    } else {
+      LOG.error("Trying to retrieve a field map with sensor type of null");
+    }
+    return new HashMap<>();
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/645d8292/metron-platform/metron-integration-test/src/main/resources/sample/data/SampleInput/BluecoatSyslog.txt
----------------------------------------------------------------------
diff --git a/metron-platform/metron-integration-test/src/main/resources/sample/data/SampleInput/BluecoatSyslog.txt b/metron-platform/metron-integration-test/src/main/resources/sample/data/SampleInput/BluecoatSyslog.txt
new file mode 100644
index 0000000..9738691
--- /dev/null
+++ b/metron-platform/metron-integration-test/src/main/resources/sample/data/SampleInput/BluecoatSyslog.txt
@@ -0,0 +1,144 @@
+<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.212.29.36: user 'WJS310' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113
+<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.212.21.253: user 'vwv149' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113
+<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.212.21.253: user 'vwv149' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113
+<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.118.216.222: user 'yaw983' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113
+<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.118.216.222: user 'yaw983' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113
+<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.118.216.222: user 'yaw983' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113
+<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.115.220.223: user 'qwn225' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113
+<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.114.217.29: user 'ags432' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113
+<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.113.216.196: user 'u62206' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113
+<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.127.221.164: user 'CXI886' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113
+<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.127.221.164: user 'CXI886' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113
+<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.114.217.29: user 'ags432' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113
+<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.218.165.248: user 'fjl928' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113
+<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.118.29.228: user 'LOCAL\uzl193' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113
+<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.118.29.228: user 'UZL193' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113
+<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.212.21.253: user 'vwv149' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113
+<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.152.102.72: user 'ugs662' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113
+<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.119.223.52: user 'XGZ521' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113
+<29>Apr 14 20:31:07 ProxySG: 250018 LDAP: invalid credentials: reason: '80090308: LdapErr: DSID-0C0903AA, comment: AcceptSecurityContext error, data 52e, v1772' dn: 'CN=FJL928,OU=Developers,OU=All Users,DC=cof,DC=ds,DC=capitalone,DC=com' realm: 'AD_ldap'(2425130) NORMAL_EVENT realm_ldap.cpp 2833
+<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.218.165.248: user 'fjl928' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113
+<29>Apr 14 20:31:07 ProxySG: 250018 LDAP: invalid credentials: reason: '80090308: LdapErr: DSID-0C0903AA, comment: AcceptSecurityContext error, data 52e, v1772' dn: 'CN=UZL193,OU=User Lock Policy 00,OU=All Users,DC=cof,DC=ds,DC=capitalone,DC=com' realm: 'AD_ldap'(2425130) NORMAL_EVENT realm_ldap.cpp 2833
+<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.118.29.228: user 'UZL193' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113
+<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.118.29.228: user 'UZL193' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113
+<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.119.223.52: user 'XGZ521' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113
+<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.152.102.72: user 'ugs662' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113
+<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.118.29.228: user 'UZL193' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113
+<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.212.21.253: user 'vwv149' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113
+<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.118.29.228: user 'UZL193' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113
+<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.212.21.253: user 'vwv149' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113
+<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.118.29.228: user 'UZL193' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113
+<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.118.29.228: user 'UZL193' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113
+<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.118.29.228: user 'UZL193' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113
+<29>Apr 14 20:31:07 ProxySG: 250018 LDAP: invalid credentials: reason: '80090308: LdapErr: DSID-0C0903AA, comment: AcceptSecurityContext error, data 52e, v1772' dn: 'CN=EPL857,OU=User Lock Policy 05,OU=All Users,DC=cof,DC=ds,DC=capitalone,DC=com' realm: 'AD_ldap'(2425130) NORMAL_EVENT realm_ldap.cpp 2833
+<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.210.223.65: user 'epl857' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113
+<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.210.223.65: user 'epl857' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113
+<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.118.29.228: user 'UZL193' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113
+<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.118.29.228: user 'UZL193' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113
+<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.118.29.228: user 'UZL193' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113
+<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.118.29.228: user 'UZL193' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113
+<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.127.216.106: user 'LOCAL\sdq302' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113
+<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.118.29.228: user 'UZL193' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113
+<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.219.15.104: user 'LOCAL\kon313' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113
+<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.118.29.228: user 'UZL193' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113
+<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.212.21.253: user 'vwv149' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113
+<29>Apr 14 20:31:07 ProxySG: 250018 LDAP: invalid credentials: reason: '80090308: LdapErr: DSID-0C0903AA, comment: AcceptSecurityContext error, data 52e, v1772' dn: 'CN=SDQ302,OU=Developers,OU=All Users,DC=cof,DC=ds,DC=capitalone,DC=com' realm: 'AD_ldap'(2425130) NORMAL_EVENT realm_ldap.cpp 2833
+<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.127.216.106: user 'sdq302' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113
+<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.118.29.228: user 'UZL193' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113
+<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.118.29.228: user 'UZL193' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113
+<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.120.144.20: user 'LOCAL\dkg773' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113
+<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.120.144.20: user 'dkg773' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113
+<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.219.15.104: user 'kon313' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113
+<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.118.29.228: user 'UZL193' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113
+<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.118.29.228: user 'UZL193' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113
+<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.118.29.228: user 'UZL193' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113
+<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.118.29.228: user 'UZL193' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113
+<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.118.29.228: user 'UZL193' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113
+<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.118.29.228: user 'UZL193' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113
+<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.115.220.223: user 'qwn225' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113
+<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.115.220.223: user 'qwn225' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113
+<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.120.144.20: user 'dkg773' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113
+<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.152.102.72: user 'ugs662' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113
+<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.118.29.228: user 'UZL193' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113
+<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.118.29.228: user 'UZL193' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113
+<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.118.29.228: user 'UZL193' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113
+<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.118.29.228: user 'UZL193' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113
+<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.118.29.228: user 'UZL193' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113
+<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.118.29.228: user 'UZL193' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113
+<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.118.29.228: user 'UZL193' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113
+<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.118.29.228: user 'UZL193' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113
+<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.118.219.193: user 'LOCAL\uua398' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113
+<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.152.102.72: user 'ugs662' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113
+<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.115.220.223: user 'qwn225' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113
+<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.115.220.223: user 'qwn225' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113
+<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.115.220.223: user 'qwn225' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113
+<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.219.15.104: user 'kon313' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113
+<29>Apr 14 20:31:06 ProxySG: 250017 Authentication failed from 10.212.21.253: user 'vwv149' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113
+<29>Apr 14 20:31:06 ProxySG: 250017 Authentication failed from 10.212.21.253: user 'vwv149' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113
+<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.119.223.52: user 'XGZ521' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113
+<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.212.29.36: user 'LOCAL\wjs310' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113
+<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.219.15.104: user 'kon313' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113
+<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.219.15.104: user 'kon313' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113
+<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.219.15.104: user 'kon313' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113
+<29>Apr 14 20:31:07 ProxySG: 250018 LDAP: invalid credentials: reason: '80090308: LdapErr: DSID-0C0903AA, comment: AcceptSecurityContext error, data 52e, v1772' dn: 'CN=WJS310,OU=Developers,OU=All Users,DC=cof,DC=ds,DC=capitalone,DC=com' realm: 'AD_ldap'(2425130) NORMAL_EVENT realm_ldap.cpp 2833
+<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.212.29.36: user 'WJS310' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113
+<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.212.29.36: user 'WJS310' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113
+<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.212.21.253: user 'vwv149' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113
+<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.212.21.253: user 'vwv149' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113
+<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.118.216.222: user 'LOCAL\yaw983' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113
+<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.118.216.222: user 'yaw983' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113
+<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.118.216.222: user 'yaw983' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113
+<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.115.220.223: user 'qwn225' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113
+<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.114.217.29: user 'ags432' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113
+<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.115.220.223: user 'qwn225' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113
+<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.113.216.196: user 'LOCAL\u62206' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113
+<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.115.220.223: user 'qwn225' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113
+<29>Apr 14 20:31:07 ProxySG: 250001 LDAP: Authentication failed from 10.113.216.196: no such user in realm 'AD_ldap'(102089) NORMAL_EVENT realm_ldap.cpp 2634
+<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.114.217.29: user 'ags432' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113
+<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.218.165.248: user 'LOCAL\fjl928' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113
+<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.218.165.248: user 'fjl928' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113
+<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.212.21.253: user 'vwv149' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113
+<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.212.21.253: user 'vwv149' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113
+<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.212.21.253: user 'vwv149' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113
+<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.152.102.72: user 'ugs662' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113
+<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.119.223.52: user 'XGZ521' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113
+<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.119.223.52: user 'XGZ521' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113
+<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.219.15.104: user 'kon313' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113
+<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.219.15.104: user 'kon313' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113
+<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.119.223.52: user 'XGZ521' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113
+<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.218.165.248: user 'fjl928' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113
+<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.218.165.248: user 'fjl928' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113
+<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.118.29.228: user 'UZL193' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113
+<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.118.29.228: user 'UZL193' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113
+<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.210.223.65: user 'LOCAL\epl857' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113
+<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.118.29.228: user 'UZL193' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113
+<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.118.29.228: user 'UZL193' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113
+<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.212.21.253: user 'vwv149' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113
+<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.212.21.253: user 'vwv149' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113
+<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.119.223.52: user 'XGZ521' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113
+<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.152.102.72: user 'ugs662' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113
+<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.210.223.65: user 'epl857' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113
+<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.118.29.228: user 'UZL193' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113
+<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.118.29.228: user 'UZL193' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113
+<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.118.29.228: user 'UZL193' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113
+<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.210.223.65: user 'epl857' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113
+<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.210.223.65: user 'epl857' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113
+<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.127.216.106: user 'sdq302' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113
+<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.118.29.228: user 'UZL193' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113
+<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.118.29.228: user 'UZL193' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113
+<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.118.29.228: user 'UZL193' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113
+<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.127.216.106: user 'sdq302' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113
+<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.118.29.228: user 'UZL193' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113
+<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.118.29.228: user 'UZL193' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113
+<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.118.29.228: user 'UZL193' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113
+<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.118.29.228: user 'UZL193' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113
+<29>Apr 14 20:31:07 ProxySG: 250018 LDAP: invalid credentials: reason: '80090308: LdapErr: DSID-0C0903AA, comment: AcceptSecurityContext error, data 52e, v1772' dn: 'CN=UUA398,OU=Developers,OU=All Users,DC=cof,DC=ds,DC=capitalone,DC=com' realm: 'AD_ldap'(2425130) NORMAL_EVENT realm_ldap.cpp 2833
+<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.118.219.193: user 'uua398' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113
+<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.212.21.253: user 'vwv149' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113
+<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.118.29.228: user 'LOCAL\uzl193' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113
+<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.118.29.228: user 'UZL193' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113
+<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.212.21.253: user 'vwv149' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113
+<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.118.219.193: user 'uua398' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113
+<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.219.15.104: user 'kon313' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113
+<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.219.15.104: user 'kon313' (realm AD_ldap)(0) NORMAL_EVENT authutility.cpp 113
+<29>Apr 14 20:31:07 ProxySG: 250017 Authentication failed from 10.113.216.196: user 'u62206' (realm iwa_realm)(0) NORMAL_EVENT authutility.cpp 113
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/645d8292/metron-platform/metron-integration-test/src/main/resources/sample/data/SampleInput/SquidExampleOutput
----------------------------------------------------------------------
diff --git a/metron-platform/metron-integration-test/src/main/resources/sample/data/SampleInput/SquidExampleOutput b/metron-platform/metron-integration-test/src/main/resources/sample/data/SampleInput/SquidExampleOutput
new file mode 100644
index 0000000..ae70fb9
--- /dev/null
+++ b/metron-platform/metron-integration-test/src/main/resources/sample/data/SampleInput/SquidExampleOutput
@@ -0,0 +1,2 @@
+1461576382.642    161 127.0.0.1 TCP_MISS/200 103701 GET http://www.cnn.com/ - DIRECT/199.27.79.73 text/html
+1461576442.228    159 127.0.0.1 TCP_MISS/200 137183 GET http://www.nba.com/ - DIRECT/66.210.41.9 text/html
\ No newline at end of file