You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@metron.apache.org by ce...@apache.org on 2016/05/16 18:07:42 UTC

[01/15] incubator-metron git commit: METRON-142 Simplify Parser configuration (merrimanr via cestella) closes apache/incubator-metron#120

Repository: incubator-metron
Updated Branches:
  refs/heads/master 00f8588d8 -> df8d682e8


http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/df8d682e/metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/topology/ParserTopologyBuilder.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/topology/ParserTopologyBuilder.java b/metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/topology/ParserTopologyBuilder.java
new file mode 100644
index 0000000..68ee36b
--- /dev/null
+++ b/metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/topology/ParserTopologyBuilder.java
@@ -0,0 +1,58 @@
+/**
+ * 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.topology;
+
+import backtype.storm.topology.TopologyBuilder;
+import org.apache.curator.framework.CuratorFramework;
+import org.apache.metron.common.configuration.ConfigurationsUtils;
+import org.apache.metron.common.configuration.SensorParserConfig;
+import org.apache.metron.common.spout.kafka.SpoutConfig;
+import org.apache.metron.common.utils.ReflectionUtils;
+import org.apache.metron.parsers.bolt.ParserBolt;
+import org.apache.metron.parsers.interfaces.MessageParser;
+import org.apache.metron.parsers.writer.KafkaWriter;
+import org.json.simple.JSONObject;
+import storm.kafka.KafkaSpout;
+import storm.kafka.ZkHosts;
+
+public class ParserTopologyBuilder {
+
+  public static TopologyBuilder build(String zookeeperUrl,
+                         String brokerUrl,
+                         String sensorType,
+                         SpoutConfig.Offset offset,
+                         int spoutParallelism,
+                         int parserParallelism) throws Exception {
+    CuratorFramework client = ConfigurationsUtils.getClient(zookeeperUrl);
+    client.start();
+    SensorParserConfig sensorParserConfig = ConfigurationsUtils.readSensorParserConfigFromZookeeper(sensorType, client);
+    client.close();
+    String sensorTopic = sensorParserConfig.getSensorTopic() != null ? sensorParserConfig.getSensorTopic() : sensorType;
+    TopologyBuilder builder = new TopologyBuilder();
+    ZkHosts zkHosts = new ZkHosts(zookeeperUrl);
+    SpoutConfig spoutConfig = new SpoutConfig(zkHosts, sensorTopic, "", sensorTopic).from(offset);
+    KafkaSpout kafkaSpout = new KafkaSpout(spoutConfig);
+    builder.setSpout("kafkaSpout", kafkaSpout, spoutParallelism);
+    MessageParser<JSONObject> parser = ReflectionUtils.createInstance(sensorParserConfig.getParserClassName());
+    parser.configure(sensorParserConfig.getParserConfig());
+    KafkaWriter writer = new KafkaWriter(brokerUrl);
+    ParserBolt parserBolt = new ParserBolt(zookeeperUrl, sensorType, parser, writer);
+    builder.setBolt("parserBolt", parserBolt, parserParallelism).shuffleGrouping("kafkaSpout");
+    return builder;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/df8d682e/metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/topology/ParserTopologyCLI.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/topology/ParserTopologyCLI.java b/metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/topology/ParserTopologyCLI.java
new file mode 100644
index 0000000..b26cbc3
--- /dev/null
+++ b/metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/topology/ParserTopologyCLI.java
@@ -0,0 +1,127 @@
+/**
+ * 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.topology;
+
+import backtype.storm.Config;
+import backtype.storm.LocalCluster;
+import backtype.storm.StormSubmitter;
+import backtype.storm.topology.TopologyBuilder;
+import backtype.storm.utils.Utils;
+import org.apache.commons.cli.CommandLine;
+import org.apache.commons.cli.CommandLineParser;
+import org.apache.commons.cli.HelpFormatter;
+import org.apache.commons.cli.Option;
+import org.apache.commons.cli.Options;
+import org.apache.commons.cli.ParseException;
+import org.apache.commons.cli.PosixParser;
+import org.apache.metron.common.spout.kafka.SpoutConfig;
+
+import java.util.HashMap;
+import java.util.Map;
+
+public class ParserTopologyCLI {
+
+  public static void main(String[] args) {
+    Options options = new Options();
+    {
+      Option o = new Option("h", "help", false, "This screen");
+      o.setRequired(false);
+      options.addOption(o);
+    }
+    {
+      Option o = new Option("z", "zk", true, "Zookeeper Quroum URL (zk1:2181,zk2:2181,...");
+      o.setArgName("ZK_QUORUM");
+      o.setRequired(true);
+      options.addOption(o);
+    }
+    {
+      Option o = new Option("k", "kafka", true, "Kafka Broker URL");
+      o.setArgName("BROKER_URL");
+      o.setRequired(true);
+      options.addOption(o);
+    }
+    {
+      Option o = new Option("s", "sensor", true, "Sensor Type");
+      o.setArgName("SENSOR_TYPE");
+      o.setRequired(true);
+      options.addOption(o);
+    }
+    {
+      Option o = new Option("sp", "spout_p", true, "Spout Parallelism");
+      o.setArgName("SPOUT_PARALLELISM");
+      o.setRequired(false);
+      o.setType(Number.class);
+      options.addOption(o);
+    }
+    {
+      Option o = new Option("pp", "parser_p", true, "Parser Parallelism");
+      o.setArgName("PARSER_PARALLELISM");
+      o.setRequired(false);
+      o.setType(Number.class);
+      options.addOption(o);
+    }
+    {
+      Option o = new Option("t", "test", true, "Run in Test Mode");
+      o.setArgName("TEST");
+      o.setRequired(false);
+      options.addOption(o);
+    }
+    try {
+      CommandLineParser parser = new PosixParser();
+      CommandLine cmd = null;
+      try {
+        cmd = parser.parse(options, args);
+      } catch (ParseException pe) {
+        pe.printStackTrace();
+        final HelpFormatter usageFormatter = new HelpFormatter();
+        usageFormatter.printHelp("ParserTopologyCLI", null, options, null, true);
+        System.exit(-1);
+      }
+      if (cmd.hasOption("h")) {
+        final HelpFormatter usageFormatter = new HelpFormatter();
+        usageFormatter.printHelp("ParserTopologyCLI", null, options, null, true);
+        System.exit(0);
+      }
+      String zookeeperUrl = cmd.getOptionValue("z");
+      String brokerUrl = cmd.getOptionValue("k");
+      String sensoryType = cmd.getOptionValue("s");
+      int spoutParallelism = Integer.parseInt(cmd.getOptionValue("sp", "1"));
+      int parserParallelism = Integer.parseInt(cmd.getOptionValue("pp", "1"));
+      SpoutConfig.Offset offset = cmd.hasOption("t") ? SpoutConfig.Offset.BEGINNING : SpoutConfig.Offset.WHERE_I_LEFT_OFF;
+      TopologyBuilder builder = ParserTopologyBuilder.build(zookeeperUrl,
+              brokerUrl,
+              sensoryType,
+              offset,
+              spoutParallelism,
+              parserParallelism);
+      if (cmd.hasOption("t")) {
+        Map<String, Object> stormConf = new HashMap<>();
+        stormConf.put(Config.TOPOLOGY_DEBUG, true);
+        LocalCluster cluster = new LocalCluster();
+        cluster.submitTopology(sensoryType, stormConf, builder.createTopology());
+        Utils.sleep(300000);
+        cluster.shutdown();
+      } else {
+        StormSubmitter.submitTopology(sensoryType, new HashMap<>(), builder.createTopology());
+      }
+    } catch (Exception e) {
+      e.printStackTrace();
+      System.exit(-1);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/df8d682e/metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/websphere/GrokWebSphereParser.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/websphere/GrokWebSphereParser.java b/metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/websphere/GrokWebSphereParser.java
index 254b97a..178719b 100644
--- a/metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/websphere/GrokWebSphereParser.java
+++ b/metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/websphere/GrokWebSphereParser.java
@@ -29,10 +29,6 @@ public class GrokWebSphereParser extends GrokParser {
 
 	private static final long serialVersionUID = 4860439408055777358L;
 
-	public GrokWebSphereParser(String grokHdfsPath, String patternLabel) {
-		super(grokHdfsPath, patternLabel);
-	}
-
 	@Override
 	protected long formatTimestamp(Object value) {
 		long epochTimestamp = System.currentTimeMillis();

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/df8d682e/metron-platform/metron-parsers/src/main/scripts/start_parser_topology.sh
----------------------------------------------------------------------
diff --git a/metron-platform/metron-parsers/src/main/scripts/start_parser_topology.sh b/metron-platform/metron-parsers/src/main/scripts/start_parser_topology.sh
index c5a5629..05fb9f7 100755
--- a/metron-platform/metron-parsers/src/main/scripts/start_parser_topology.sh
+++ b/metron-platform/metron-parsers/src/main/scripts/start_parser_topology.sh
@@ -19,4 +19,4 @@
 METRON_VERSION=${project.version}
 METRON_HOME=/usr/metron/$METRON_VERSION
 TOPOLOGY_JAR=metron-parsers-$METRON_VERSION.jar
-storm jar $METRON_HOME/lib/$TOPOLOGY_JAR org.apache.storm.flux.Flux --remote $METRON_HOME/flux/$1/remote.yaml --filter $METRON_HOME/config/parsers.properties
+storm jar $METRON_HOME/lib/$TOPOLOGY_JAR org.apache.metron.parsers.topology.ParserTopologyCLI "$@"

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/df8d682e/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 7abcc87..b1deb0e 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
@@ -27,10 +27,7 @@ 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;
+import java.util.*;
 
 public abstract class GrokParserTest {
 
@@ -44,17 +41,14 @@ public abstract class GrokParserTest {
 
   @Test
   public void test() throws IOException, ParseException {
-    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());
+    Map<String, Object> parserConfig = new HashMap<>();
+    parserConfig.put("grokPath", getGrokPath());
+    parserConfig.put("patternLabel", getGrokPatternLabel());
+    parserConfig.put("timestampField", getTimestampField());
+    parserConfig.put("dateFormat", getDateFormat());
+    parserConfig.put("timeFields", getTimeFields());
+    GrokParser grokParser = new GrokParser();
+    grokParser.configure(parserConfig);
     grokParser.init();
     byte[] rawMessage = getRawMessage().getBytes();
     List<JSONObject> parsedList = grokParser.parse(rawMessage);
@@ -91,7 +85,7 @@ public abstract class GrokParserTest {
   public abstract String getExpectedParsedString();
   public abstract String getGrokPath();
   public abstract String getGrokPatternLabel();
-  public abstract String[] getTimeFields();
+  public abstract List<String> getTimeFields();
   public abstract String getDateFormat();
   public abstract String getTimestampField();
 }

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/df8d682e/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
index 6f489a4..fe11c3f 100644
--- 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
@@ -19,6 +19,9 @@ package org.apache.metron.parsers;
 
 import org.adrianwalker.multilinestring.Multiline;
 
+import java.util.ArrayList;
+import java.util.List;
+
 public class SampleGrokParserTest extends GrokParserTest {
 
   /**
@@ -61,15 +64,17 @@ public class SampleGrokParserTest extends GrokParserTest {
   }
 
   public String getGrokPath() {
-    return "../metron-integration-test/src/main/resources/sample/patterns/test";
+    return "../metron-integration-test/src/main/sample/patterns/test";
   }
 
   public String getGrokPatternLabel() {
     return "YAF_DELIMITED";
   }
 
-  public String[] getTimeFields() {
-    return new String[]{"end_time"};
+  public List<String> getTimeFields() {
+    return new ArrayList<String>() {{
+      add("end_time");
+    }};
   }
 
   public String getDateFormat() {

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/df8d682e/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
index 1218595..d97025e 100644
--- 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
@@ -19,6 +19,9 @@ package org.apache.metron.parsers;
 
 import org.adrianwalker.multilinestring.Multiline;
 
+import java.util.ArrayList;
+import java.util.List;
+
 public class SquidParserTest extends GrokParserTest {
 
   @Override
@@ -59,8 +62,8 @@ public class SquidParserTest extends GrokParserTest {
   }
 
   @Override
-  public String[] getTimeFields() {
-    return new String[0];
+  public List<String> getTimeFields() {
+    return new ArrayList<>();
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/df8d682e/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
index 9e887f5..1c1da4d 100644
--- 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
@@ -19,6 +19,9 @@ package org.apache.metron.parsers;
 
 import org.adrianwalker.multilinestring.Multiline;
 
+import java.util.ArrayList;
+import java.util.List;
+
 public class YafParserTest extends GrokParserTest {
 
   @Override
@@ -74,8 +77,11 @@ public class YafParserTest extends GrokParserTest {
   }
 
   @Override
-  public String[] getTimeFields() {
-    return new String[]{"start_time", "end_time"};
+  public List<String> getTimeFields() {
+    return new ArrayList<String>() {{
+      add("start_time");
+      add("end_time");
+    }};
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/df8d682e/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
index 8b6207c..e732761 100644
--- 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
@@ -17,32 +17,21 @@
  */
 package org.apache.metron.parsers.integration;
 
-import org.apache.metron.TestConstants;
+import org.apache.metron.parsers.integration.validation.SampleDataValidation;
 
-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";
-  }
+import java.util.ArrayList;
+import java.util.List;
 
+public class BluecoatIntegrationTest extends ParserIntegrationTest {
   @Override
-  public String getSensorType() {
+  String getSensorType() {
     return "bluecoat";
   }
 
   @Override
-  public String getFluxTopicProperty() {
-    return "spout.kafka.topic.yaf";
+  List<ParserValidation> getValidations() {
+    return new ArrayList<ParserValidation>() {{
+      add(new SampleDataValidation());
+    }};
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/df8d682e/metron-platform/metron-parsers/src/test/java/org/apache/metron/parsers/integration/BroIntegrationTest.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-parsers/src/test/java/org/apache/metron/parsers/integration/BroIntegrationTest.java b/metron-platform/metron-parsers/src/test/java/org/apache/metron/parsers/integration/BroIntegrationTest.java
new file mode 100644
index 0000000..c9cfe0a
--- /dev/null
+++ b/metron-platform/metron-parsers/src/test/java/org/apache/metron/parsers/integration/BroIntegrationTest.java
@@ -0,0 +1,37 @@
+/**
+ * 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.parsers.integration.validation.SampleDataValidation;
+
+import java.util.ArrayList;
+import java.util.List;
+
+public class BroIntegrationTest extends ParserIntegrationTest {
+  @Override
+  String getSensorType() {
+    return "bro";
+  }
+
+  @Override
+  List<ParserValidation> getValidations() {
+    return new ArrayList<ParserValidation>() {{
+      add(new SampleDataValidation());
+    }};
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/df8d682e/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 e0b9432..116e262 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
@@ -17,63 +17,61 @@
  */
 package org.apache.metron.parsers.integration;
 
+import junit.framework.Assert;
+import org.apache.commons.io.FilenameUtils;
+import org.apache.metron.TestConstants;
 import org.apache.metron.common.Constants;
 import org.apache.metron.integration.BaseIntegrationTest;
-import org.apache.metron.integration.utils.TestUtils;
-import org.apache.metron.test.utils.UnitTestHelper;
 import org.apache.metron.integration.ComponentRunner;
 import org.apache.metron.integration.Processor;
 import org.apache.metron.integration.ReadinessState;
-import org.apache.metron.integration.components.FluxTopologyComponent;
+import org.apache.metron.integration.components.ConfigUploadComponent;
 import org.apache.metron.integration.components.KafkaWithZKComponent;
-import org.codehaus.jackson.map.ObjectMapper;
-import org.junit.Assert;
+import org.apache.metron.integration.utils.TestUtils;
+import org.apache.metron.parsers.integration.components.ParserTopologyComponent;
+import org.apache.metron.parsers.integration.validation.SampleDataValidation;
+import org.apache.metron.test.TestDataType;
+import org.apache.metron.test.utils.SampleDataUtils;
+import org.apache.metron.test.utils.UnitTestHelper;
+import org.junit.Before;
 import org.junit.Test;
 
 import java.io.File;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Map;
-import java.util.Properties;
+import java.util.*;
 
 public abstract class ParserIntegrationTest extends BaseIntegrationTest {
 
-  public abstract String getFluxPath();
-  public abstract String getSampleInputPath();
-  public abstract String getSampleParsedPath();
-  public abstract String getSensorType();
-  public abstract String getFluxTopicProperty();
-
   @Test
   public void test() throws Exception {
-
-    final String kafkaTopic = getSensorType();
-
-    final List<byte[]> inputMessages = TestUtils.readSampleData(getSampleInputPath());
+    final String sensorType = getSensorType();
+    final List<byte[]> inputMessages = TestUtils.readSampleData(SampleDataUtils.getSampleDataPath(sensorType, TestDataType.RAW));
 
     final Properties topologyProperties = new Properties();
     final KafkaWithZKComponent kafkaComponent = getKafkaComponent(topologyProperties, new ArrayList<KafkaWithZKComponent.Topic>() {{
-      add(new KafkaWithZKComponent.Topic(kafkaTopic, 1));
+      add(new KafkaWithZKComponent.Topic(sensorType, 1));
     }});
-
     topologyProperties.setProperty("kafka.broker", kafkaComponent.getBrokerList());
-    FluxTopologyComponent fluxComponent = new FluxTopologyComponent.Builder()
-            .withTopologyLocation(new File(getFluxPath()))
-            .withTopologyName("test")
+
+    ConfigUploadComponent configUploadComponent = new ConfigUploadComponent()
             .withTopologyProperties(topologyProperties)
-            .build();
+            .withGlobalConfigsPath(TestConstants.SAMPLE_CONFIG_PATH)
+            .withParserConfigsPath(TestConstants.PARSER_CONFIGS_PATH);
+
+    ParserTopologyComponent parserTopologyComponent = new ParserTopologyComponent.Builder()
+            .withSensorType(sensorType)
+            .withTopologyProperties(topologyProperties)
+            .withBrokerUrl(kafkaComponent.getBrokerList()).build();
 
     UnitTestHelper.verboseLogging();
     ComponentRunner runner = new ComponentRunner.Builder()
             .withComponent("kafka", kafkaComponent)
-            .withComponent("storm", fluxComponent)
+            .withComponent("config", configUploadComponent)
+            .withComponent("storm", parserTopologyComponent)
             .withMillisecondsBetweenAttempts(5000)
             .withNumRetries(10)
             .build();
     runner.start();
-    fluxComponent.submitTopology();
-    kafkaComponent.writeMessages(kafkaTopic, inputMessages);
+    kafkaComponent.writeMessages(sensorType, inputMessages);
     List<byte[]> outputMessages =
             runner.process(new Processor<List<byte[]>>() {
               List<byte[]> messages = null;
@@ -93,43 +91,29 @@ public abstract class ParserIntegrationTest extends BaseIntegrationTest {
                 return messages;
               }
             });
-    List<byte[]> sampleParsedMessages = TestUtils.readSampleData(getSampleParsedPath());
-    Assert.assertEquals(sampleParsedMessages.size(), outputMessages.size());
-    for (int i = 0; i < outputMessages.size(); i++) {
-      String sampleParsedMessage = new String(sampleParsedMessages.get(i));
-      String outputMessage = new String(outputMessages.get(i));
-      try {
-        assertJSONEqual(sampleParsedMessage, outputMessage);
-      } catch (Throwable t) {
-        System.out.println("expected: " + sampleParsedMessage);
-        System.out.println("actual: " + outputMessage);
-        throw t;
+    List<ParserValidation> validations = getValidations();
+    if (validations == null || validations.isEmpty()) {
+      System.out.println("No validations configured for sensorType " + sensorType + ".  Dumping parsed messages");
+      System.out.println();
+      dumpParsedMessages(outputMessages);
+      System.out.println();
+      Assert.fail();
+    } else {
+      for (ParserValidation validation : validations) {
+        System.out.println("Running " + validation.getName() + " on sensorType " + sensorType);
+        validation.validate(sensorType, outputMessages);
       }
     }
     runner.stop();
-
   }
 
-  public static void assertJSONEqual(String doc1, String doc2) throws IOException {
-    ObjectMapper mapper = new ObjectMapper();
-    Map m1 = mapper.readValue(doc1, Map.class);
-    Map m2 = mapper.readValue(doc2, Map.class);
-    for(Object k : m1.keySet()) {
-      Object v1 = m1.get(k);
-      Object v2 = m2.get(k);
-
-      if(v2 == null) {
-        Assert.fail("Unable to find key: " + k + " in output");
-      }
-      if(k.equals("timestamp")) {
-        //TODO: Take the ?!?@ timestamps out of the reference file.
-        Assert.assertEquals(v1.toString().length(), v2.toString().length());
-      }
-      else if(!v2.equals(v1)) {
-        Assert.assertEquals("value mismatch for " + k ,v1, v2);
-      }
+  public void dumpParsedMessages(List<byte[]> outputMessages) {
+    for (byte[] outputMessage : outputMessages) {
+      System.out.println(new String(outputMessage));
     }
-    Assert.assertEquals(m1.size(), m2.size());
   }
 
+  abstract String getSensorType();
+  abstract List<ParserValidation> getValidations();
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/df8d682e/metron-platform/metron-parsers/src/test/java/org/apache/metron/parsers/integration/ParserValidation.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-parsers/src/test/java/org/apache/metron/parsers/integration/ParserValidation.java b/metron-platform/metron-parsers/src/test/java/org/apache/metron/parsers/integration/ParserValidation.java
new file mode 100644
index 0000000..a92fd32
--- /dev/null
+++ b/metron-platform/metron-parsers/src/test/java/org/apache/metron/parsers/integration/ParserValidation.java
@@ -0,0 +1,26 @@
+/**
+ * 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 java.util.List;
+
+public interface ParserValidation {
+
+  String getName();
+  void validate(String sensorType, List<byte[]> actualMessages) throws Exception;
+}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/df8d682e/metron-platform/metron-parsers/src/test/java/org/apache/metron/parsers/integration/SnortIntegrationTest.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-parsers/src/test/java/org/apache/metron/parsers/integration/SnortIntegrationTest.java b/metron-platform/metron-parsers/src/test/java/org/apache/metron/parsers/integration/SnortIntegrationTest.java
index 983f7e3..244dd89 100644
--- a/metron-platform/metron-parsers/src/test/java/org/apache/metron/parsers/integration/SnortIntegrationTest.java
+++ b/metron-platform/metron-parsers/src/test/java/org/apache/metron/parsers/integration/SnortIntegrationTest.java
@@ -17,32 +17,21 @@
  */
 package org.apache.metron.parsers.integration;
 
-import org.apache.metron.TestConstants;
+import org.apache.metron.parsers.integration.validation.SampleDataValidation;
 
-public class SnortIntegrationTest extends ParserIntegrationTest {
-
-  @Override
-  public String getFluxPath() {
-    return "./src/main/flux/snort/test.yaml";
-  }
-
-  @Override
-  public String getSampleInputPath() {
-    return TestConstants.SAMPLE_DATA_INPUT_PATH + "SnortOutput";
-  }
-
-  @Override
-  public String getSampleParsedPath() {
-    return TestConstants.SAMPLE_DATA_PARSED_PATH + "SnortParsed";
-  }
+import java.util.ArrayList;
+import java.util.List;
 
+public class SnortIntegrationTest extends ParserIntegrationTest {
   @Override
-  public String getSensorType() {
+  String getSensorType() {
     return "snort";
   }
 
   @Override
-  public String getFluxTopicProperty() {
-    return "spout.kafka.topic.snort";
+  List<ParserValidation> getValidations() {
+    return new ArrayList<ParserValidation>() {{
+      add(new SampleDataValidation());
+    }};
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/df8d682e/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
index 023278d..9c03e81 100644
--- 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
@@ -17,33 +17,21 @@
  */
 package org.apache.metron.parsers.integration;
 
-import org.apache.metron.TestConstants;
-import org.json.simple.JSONObject;
+import org.apache.metron.parsers.integration.validation.SampleDataValidation;
 
-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";
-  }
+import java.util.ArrayList;
+import java.util.List;
 
+public class SquidIntegrationTest extends ParserIntegrationTest {
   @Override
-  public String getSensorType() {
+  String getSensorType() {
     return "squid";
   }
 
   @Override
-  public String getFluxTopicProperty() {
-    return "spout.kafka.topic.yaf";
+  List<ParserValidation> getValidations() {
+    return new ArrayList<ParserValidation>() {{
+      add(new SampleDataValidation());
+    }};
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/df8d682e/metron-platform/metron-parsers/src/test/java/org/apache/metron/parsers/integration/WebSphereIntegrationTest.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-parsers/src/test/java/org/apache/metron/parsers/integration/WebSphereIntegrationTest.java b/metron-platform/metron-parsers/src/test/java/org/apache/metron/parsers/integration/WebSphereIntegrationTest.java
index 04761b2..5dc0ac6 100644
--- a/metron-platform/metron-parsers/src/test/java/org/apache/metron/parsers/integration/WebSphereIntegrationTest.java
+++ b/metron-platform/metron-parsers/src/test/java/org/apache/metron/parsers/integration/WebSphereIntegrationTest.java
@@ -18,33 +18,23 @@
 
 package org.apache.metron.parsers.integration;
 
-import org.apache.metron.TestConstants;
+import org.apache.metron.parsers.integration.validation.SampleDataValidation;
 
-public class WebSphereIntegrationTest extends ParserIntegrationTest {
-	
-	  @Override
-	  public String getFluxPath() {
-	    return "./src/main/flux/websphere/test.yaml";
-	  }
-
-	  @Override
-	  public String getSampleInputPath() {
-	    return TestConstants.SAMPLE_DATA_INPUT_PATH + "WebsphereOutput.txt";
-	  }
+import java.util.ArrayList;
+import java.util.List;
 
-	  @Override
-	  public String getSampleParsedPath() {
-	    return TestConstants.SAMPLE_DATA_PARSED_PATH + "WebsphereParsed";
-	  }
-
-	  @Override
-	  public String getSensorType() {
-	    return "websphere";
-	  }
+public class WebSphereIntegrationTest extends ParserIntegrationTest {
 
-	  @Override
-	  public String getFluxTopicProperty() {
-	    return "spout.kafka.topic.websphere";
-	  }
+	@Override
+	public String getSensorType() {
+		return "websphere";
+	}
+
+	@Override
+	List<ParserValidation> getValidations() {
+		return new ArrayList<ParserValidation>() {{
+			add(new SampleDataValidation());
+		}};
+	}
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/df8d682e/metron-platform/metron-parsers/src/test/java/org/apache/metron/parsers/integration/YafIntegrationTest.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-parsers/src/test/java/org/apache/metron/parsers/integration/YafIntegrationTest.java b/metron-platform/metron-parsers/src/test/java/org/apache/metron/parsers/integration/YafIntegrationTest.java
index 67fe2d6..9930c2c 100644
--- a/metron-platform/metron-parsers/src/test/java/org/apache/metron/parsers/integration/YafIntegrationTest.java
+++ b/metron-platform/metron-parsers/src/test/java/org/apache/metron/parsers/integration/YafIntegrationTest.java
@@ -17,32 +17,21 @@
  */
 package org.apache.metron.parsers.integration;
 
-import org.apache.metron.TestConstants;
+import org.apache.metron.parsers.integration.validation.SampleDataValidation;
 
-public class YafIntegrationTest extends ParserIntegrationTest {
-
-  @Override
-  public String getFluxPath() {
-    return "./src/main/flux/yaf/test.yaml";
-  }
-
-  @Override
-  public String getSampleInputPath() {
-    return TestConstants.SAMPLE_DATA_INPUT_PATH + "YafExampleOutput";
-  }
-
-  @Override
-  public String getSampleParsedPath() {
-    return TestConstants.SAMPLE_DATA_PARSED_PATH + "YafExampleParsed";
-  }
+import java.util.ArrayList;
+import java.util.List;
 
+public class YafIntegrationTest extends ParserIntegrationTest {
   @Override
-  public String getSensorType() {
+  String getSensorType() {
     return "yaf";
   }
 
   @Override
-  public String getFluxTopicProperty() {
-    return "spout.kafka.topic.yaf";
+  List<ParserValidation> getValidations() {
+    return new ArrayList<ParserValidation>() {{
+      add(new SampleDataValidation());
+    }};
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/df8d682e/metron-platform/metron-parsers/src/test/java/org/apache/metron/parsers/integration/components/ParserTopologyComponent.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-parsers/src/test/java/org/apache/metron/parsers/integration/components/ParserTopologyComponent.java b/metron-platform/metron-parsers/src/test/java/org/apache/metron/parsers/integration/components/ParserTopologyComponent.java
new file mode 100644
index 0000000..a5c6659
--- /dev/null
+++ b/metron-platform/metron-parsers/src/test/java/org/apache/metron/parsers/integration/components/ParserTopologyComponent.java
@@ -0,0 +1,84 @@
+/**
+ * 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.components;
+
+import backtype.storm.Config;
+import backtype.storm.LocalCluster;
+import backtype.storm.topology.TopologyBuilder;
+import org.apache.metron.common.spout.kafka.SpoutConfig;
+import org.apache.metron.integration.InMemoryComponent;
+import org.apache.metron.integration.UnableToStartException;
+import org.apache.metron.parsers.topology.ParserTopologyBuilder;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Properties;
+
+public class ParserTopologyComponent implements InMemoryComponent {
+
+  private Properties topologyProperties;
+  private String brokerUrl;
+  private String sensorType;
+  private LocalCluster stormCluster;
+
+  public static class Builder {
+    Properties topologyProperties;
+    String brokerUrl;
+    String sensorType;
+    public Builder withTopologyProperties(Properties topologyProperties) {
+      this.topologyProperties = topologyProperties;
+      return this;
+    }
+    public Builder withBrokerUrl(String brokerUrl) {
+      this.brokerUrl = brokerUrl;
+      return this;
+    }
+    public Builder withSensorType(String sensorType) {
+      this.sensorType = sensorType;
+      return this;
+    }
+
+    public ParserTopologyComponent build() {
+      return new ParserTopologyComponent(topologyProperties, brokerUrl, sensorType);
+    }
+  }
+
+  public ParserTopologyComponent(Properties topologyProperties, String brokerUrl, String sensorType) {
+    this.topologyProperties = topologyProperties;
+    this.brokerUrl = brokerUrl;
+    this.sensorType = sensorType;
+  }
+
+  @Override
+  public void start() throws UnableToStartException {
+    try {
+      TopologyBuilder topologyBuilder = ParserTopologyBuilder.build(topologyProperties.getProperty("kafka.zk"), brokerUrl, sensorType, SpoutConfig.Offset.BEGINNING, 1, 1);
+      Map<String, Object> stormConf = new HashMap<>();
+      stormConf.put(Config.TOPOLOGY_DEBUG, true);
+      stormCluster = new LocalCluster();
+      stormCluster.submitTopology(sensorType, stormConf, topologyBuilder.createTopology());
+    } catch (Exception e) {
+      throw new UnableToStartException("Unable to start parser topology for sensorType: " + sensorType, e);
+    }
+  }
+
+  @Override
+  public void stop() {
+    stormCluster.shutdown();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/df8d682e/metron-platform/metron-parsers/src/test/java/org/apache/metron/parsers/integration/validation/SampleDataValidation.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-parsers/src/test/java/org/apache/metron/parsers/integration/validation/SampleDataValidation.java b/metron-platform/metron-parsers/src/test/java/org/apache/metron/parsers/integration/validation/SampleDataValidation.java
new file mode 100644
index 0000000..9ea9b71
--- /dev/null
+++ b/metron-platform/metron-parsers/src/test/java/org/apache/metron/parsers/integration/validation/SampleDataValidation.java
@@ -0,0 +1,52 @@
+/**
+ * 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.validation;
+
+import org.apache.metron.integration.utils.TestUtils;
+import org.apache.metron.parsers.integration.ParserValidation;
+import org.apache.metron.test.TestDataType;
+import org.apache.metron.test.utils.SampleDataUtils;
+import org.apache.metron.test.utils.ValidationUtils;
+import org.junit.Assert;
+
+import java.util.List;
+
+public class SampleDataValidation implements ParserValidation {
+
+  @Override
+  public String getName() {
+    return "Sample Data Validation";
+  }
+
+  @Override
+  public void validate(String sensorType, List<byte[]> actualMessages) throws Exception {
+    List<byte[]> expectedMessages = TestUtils.readSampleData(SampleDataUtils.getSampleDataPath(sensorType, TestDataType.PARSED));
+    Assert.assertEquals(expectedMessages.size(), actualMessages.size());
+    for (int i = 0; i < actualMessages.size(); i++) {
+      String expectedMessage = new String(expectedMessages.get(i));
+      String actualMessage = new String(actualMessages.get(i));
+      try {
+        ValidationUtils.assertJSONEqual(expectedMessage, actualMessage);
+      } catch (Throwable t) {
+        System.out.println("expected: " + expectedMessage);
+        System.out.println("actual: " + actualMessage);
+        throw t;
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/df8d682e/metron-platform/metron-parsers/src/test/java/org/apache/metron/parsers/websphere/GrokWebSphereParserTest.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-parsers/src/test/java/org/apache/metron/parsers/websphere/GrokWebSphereParserTest.java b/metron-platform/metron-parsers/src/test/java/org/apache/metron/parsers/websphere/GrokWebSphereParserTest.java
index b90c906..a4789cd 100644
--- a/metron-platform/metron-parsers/src/test/java/org/apache/metron/parsers/websphere/GrokWebSphereParserTest.java
+++ b/metron-platform/metron-parsers/src/test/java/org/apache/metron/parsers/websphere/GrokWebSphereParserTest.java
@@ -19,23 +19,34 @@
 package org.apache.metron.parsers.websphere;
 
 import static org.junit.Assert.assertEquals;
+
+import java.util.HashMap;
 import java.util.List;
+import java.util.Map;
+
 import org.json.simple.JSONObject;
+import org.junit.Before;
 import org.junit.Test;
 
 public class GrokWebSphereParserTest {
 
-	private final String grokPath = "../metron-parsers/src/main/resources/patterns/websphere";
-	private final String grokLabel = "WEBSPHERE";
-	private final String dateFormat = "yyyy MMM dd HH:mm:ss";
-	private final String timestampField = "timestamp_string";
+	private Map<String, Object> parserConfig;
+
+	@Before
+	public void setup() {
+		parserConfig = new HashMap<>();
+		parserConfig.put("grokPath", "../metron-parsers/src/main/resources/patterns/websphere");
+		parserConfig.put("patternLabel", "WEBSPHERE");
+		parserConfig.put("timestampField", "timestamp_string");
+		parserConfig.put("dateFormat", "yyyy MMM dd HH:mm:ss");
+	}
 	
 	@Test
 	public void testParseLoginLine() throws Exception {
 		
 		//Set up parser, parse message
-		GrokWebSphereParser parser = new GrokWebSphereParser(grokPath, grokLabel);
-		parser.withDateFormat(dateFormat).withTimestampField(timestampField);
+		GrokWebSphereParser parser = new GrokWebSphereParser();
+		parser.configure(parserConfig);
 		String testString = "<133>Apr 15 17:47:28 ABCXML1413 [rojOut][0x81000033][auth][notice] user(rick007): "
 				+ "[120.43.200.6]: User logged into 'cohlOut'.";
 		List<JSONObject> result = parser.parse(testString.getBytes());
@@ -58,8 +69,8 @@ public class GrokWebSphereParserTest {
 	public void tetsParseLogoutLine() throws Exception {
 		
 		//Set up parser, parse message
-		GrokWebSphereParser parser = new GrokWebSphereParser(grokPath, grokLabel);
-		parser.withDateFormat(dateFormat).withTimestampField(timestampField);
+		GrokWebSphereParser parser = new GrokWebSphereParser();
+		parser.configure(parserConfig);
 		String testString = "<134>Apr 15 18:02:27 PHIXML3RWD [0x81000019][auth][info] [14.122.2.201]: "
 				+ "User 'hjpotter' logged out from 'default'.";
 		List<JSONObject> result = parser.parse(testString.getBytes());
@@ -81,8 +92,8 @@ public class GrokWebSphereParserTest {
 	public void tetsParseRBMLine() throws Exception {
 		
 		//Set up parser, parse message
-		GrokWebSphereParser parser = new GrokWebSphereParser(grokPath, grokLabel);
-		parser.withDateFormat(dateFormat).withTimestampField(timestampField);
+		GrokWebSphereParser parser = new GrokWebSphereParser();
+		parser.configure(parserConfig);
 		String testString = "<131>Apr 15 17:36:35 ROBXML3QRS [0x80800018][auth][error] rbm(RBM-Settings): "
 				+ "trans(3502888135)[request] gtid(3502888135): RBM: Resource access denied.";
 		List<JSONObject> result = parser.parse(testString.getBytes());
@@ -103,8 +114,8 @@ public class GrokWebSphereParserTest {
 	public void tetsParseOtherLine() throws Exception {
 		
 		//Set up parser, parse message
-		GrokWebSphereParser parser = new GrokWebSphereParser(grokPath, grokLabel);
-		parser.withDateFormat(dateFormat).withTimestampField(timestampField);
+		GrokWebSphereParser parser = new GrokWebSphereParser();
+		parser.configure(parserConfig);
 		String testString = "<134>Apr 15 17:17:34 SAGPXMLQA333 [0x8240001c][audit][info] trans(191): (admin:default:system:*): "
 				+ "ntp-service 'NTP Service' - Operational state down";
 		List<JSONObject> result = parser.parse(testString.getBytes());
@@ -125,8 +136,8 @@ public class GrokWebSphereParserTest {
 	public void testParseMalformedLoginLine() throws Exception {
 		
 		//Set up parser, attempt to parse malformed message
-		GrokWebSphereParser parser = new GrokWebSphereParser(grokPath, grokLabel);
-		parser.withDateFormat(dateFormat).withTimestampField(timestampField);
+		GrokWebSphereParser parser = new GrokWebSphereParser();
+		parser.configure(parserConfig);
 		String testString = "<133>Apr 15 17:47:28 ABCXML1413 [rojOut][0x81000033][auth][notice] rick007): "
 				+ "[120.43.200. User logged into 'cohlOut'.";
 		List<JSONObject> result = parser.parse(testString.getBytes());		
@@ -149,8 +160,8 @@ public class GrokWebSphereParserTest {
 	public void tetsParseMalformedLogoutLine() throws Exception {
 		
 		//Set up parser, attempt to parse malformed message
-		GrokWebSphereParser parser = new GrokWebSphereParser(grokPath, grokLabel);
-		parser.withDateFormat(dateFormat).withTimestampField(timestampField);
+		GrokWebSphereParser parser = new GrokWebSphereParser();
+		parser.configure(parserConfig);
 		String testString = "<134>Apr 15 18:02:27 PHIXML3RWD [0x81000019][auth][info] [14.122.2.201: "
 				+ "User 'hjpotter' logged out from 'default.";
 		List<JSONObject> result = parser.parse(testString.getBytes());
@@ -172,8 +183,8 @@ public class GrokWebSphereParserTest {
 	public void tetsParseMalformedRBMLine() throws Exception {
 		
 		//Set up parser, parse message
-		GrokWebSphereParser parser = new GrokWebSphereParser(grokPath, grokLabel);
-		parser.withDateFormat(dateFormat).withTimestampField(timestampField);
+		GrokWebSphereParser parser = new GrokWebSphereParser();
+		parser.configure(parserConfig);
 		String testString = "<131>Apr 15 17:36:35 ROBXML3QRS [0x80800018][auth][error] rbmRBM-Settings): "
 				+ "trans3502888135)[request] gtid3502888135) RBM: Resource access denied.";
 		List<JSONObject> result = parser.parse(testString.getBytes());
@@ -194,8 +205,8 @@ public class GrokWebSphereParserTest {
 	public void tetsParseMalformedOtherLine() throws Exception {
 		
 		//Set up parser, parse message
-		GrokWebSphereParser parser = new GrokWebSphereParser(grokPath, grokLabel);
-		parser.withDateFormat(dateFormat).withTimestampField(timestampField);
+		GrokWebSphereParser parser = new GrokWebSphereParser();
+		parser.configure(parserConfig);
 		String testString = "<134>Apr 15 17:17:34 SAGPXMLQA333 [0x8240001c][audit][info] trans 191)  admindefaultsystem*): "
 				+ "ntp-service 'NTP Service' - Operational state down:";
 		List<JSONObject> result = parser.parse(testString.getBytes());
@@ -218,7 +229,8 @@ public class GrokWebSphereParserTest {
 	public void testParseEmptyLine() throws Exception {
 		
 		//Set up parser, attempt to parse malformed message
-		GrokWebSphereParser parser = new GrokWebSphereParser(grokPath, grokLabel);
+		GrokWebSphereParser parser = new GrokWebSphereParser();
+		parser.configure(parserConfig);
 		String testString = "";
 		List<JSONObject> result = parser.parse(testString.getBytes());		
 		assertEquals(null, result);

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/df8d682e/metron-platform/metron-pcap-backend/src/test/java/org/apache/metron/pcap/integration/PcapTopologyIntegrationTest.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-pcap-backend/src/test/java/org/apache/metron/pcap/integration/PcapTopologyIntegrationTest.java b/metron-platform/metron-pcap-backend/src/test/java/org/apache/metron/pcap/integration/PcapTopologyIntegrationTest.java
index 587ec7c..1142da6 100644
--- a/metron-platform/metron-pcap-backend/src/test/java/org/apache/metron/pcap/integration/PcapTopologyIntegrationTest.java
+++ b/metron-platform/metron-pcap-backend/src/test/java/org/apache/metron/pcap/integration/PcapTopologyIntegrationTest.java
@@ -195,7 +195,7 @@ public class PcapTopologyIntegrationTest {
     //Assert.assertEquals(0, numFiles(outDir));
     Assert.assertNotNull(topologiesDir);
     Assert.assertNotNull(targetDir);
-    Path pcapFile = new Path("../metron-integration-test/src/main/resources/sample/data/SampleInput/PCAPExampleOutput");
+    Path pcapFile = new Path("../metron-integration-test/src/main/sample/data/SampleInput/PCAPExampleOutput");
     final List<Map.Entry<byte[], byte[]>> pcapEntries = Lists.newArrayList(readPcaps(pcapFile, withHeaders));
     Assert.assertTrue(Iterables.size(pcapEntries) > 0);
     final Properties topologyProperties = new Properties() {{

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/df8d682e/metron-platform/metron-pcap/src/test/java/org/apache/metron/pcap/PcapHelperTest.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-pcap/src/test/java/org/apache/metron/pcap/PcapHelperTest.java b/metron-platform/metron-pcap/src/test/java/org/apache/metron/pcap/PcapHelperTest.java
index b86aaac..5d2bee0 100644
--- a/metron-platform/metron-pcap/src/test/java/org/apache/metron/pcap/PcapHelperTest.java
+++ b/metron-platform/metron-pcap/src/test/java/org/apache/metron/pcap/PcapHelperTest.java
@@ -57,7 +57,7 @@ public class PcapHelperTest {
 
   @Test
   public void testLittleEndianHeaderization() throws Exception {
-    String pcapSampleFiles = "../metron-integration-test/src/main/resources/sample/data/SampleInput/PCAPExampleOutput";
+    String pcapSampleFiles = "../metron-integration-test/src/main/sample/data/SampleInput/PCAPExampleOutput";
     List<byte[]> pcaps = readSamplePackets(pcapSampleFiles);
     for(byte[] pcap : pcaps)
     {

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/df8d682e/metron-platform/metron-solr/src/main/java/org/apache/metron/solr/writer/SolrWriter.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-solr/src/main/java/org/apache/metron/solr/writer/SolrWriter.java b/metron-platform/metron-solr/src/main/java/org/apache/metron/solr/writer/SolrWriter.java
index 45d5615..bdf61e8 100644
--- a/metron-platform/metron-solr/src/main/java/org/apache/metron/solr/writer/SolrWriter.java
+++ b/metron-platform/metron-solr/src/main/java/org/apache/metron/solr/writer/SolrWriter.java
@@ -19,6 +19,7 @@ package org.apache.metron.solr.writer;
 
 import backtype.storm.tuple.Tuple;
 import org.apache.metron.common.configuration.Configurations;
+import org.apache.metron.common.configuration.EnrichmentConfigurations;
 import org.apache.metron.common.interfaces.BulkMessageWriter;
 import org.apache.solr.client.solrj.SolrServerException;
 import org.apache.solr.client.solrj.response.UpdateResponse;
@@ -53,7 +54,7 @@ public class SolrWriter implements BulkMessageWriter<JSONObject>, Serializable {
   }
 
   @Override
-  public void init(Map stormConf, Configurations configurations) throws IOException, SolrServerException {
+  public void init(Map stormConf, EnrichmentConfigurations configurations) throws IOException, SolrServerException {
     Map<String, Object> globalConfiguration = configurations.getGlobalConfig();
     if(solr == null) solr = new MetronSolrClient((String) globalConfiguration.get("solr.zookeeper"));
     String collection = getCollection(configurations);
@@ -62,7 +63,7 @@ public class SolrWriter implements BulkMessageWriter<JSONObject>, Serializable {
   }
 
   @Override
-  public void write(String sourceType, Configurations configurations, List<Tuple> tuples, List<JSONObject> messages) throws Exception {
+  public void write(String sourceType, EnrichmentConfigurations configurations, List<Tuple> tuples, List<JSONObject> messages) throws Exception {
     for(JSONObject message: messages) {
       SolrInputDocument document = new SolrInputDocument();
       document.addField("id", getIdValue(message));

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/df8d682e/metron-platform/metron-solr/src/test/java/org/apache/metron/solr/integration/SolrEnrichmentIntegrationTest.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-solr/src/test/java/org/apache/metron/solr/integration/SolrEnrichmentIntegrationTest.java b/metron-platform/metron-solr/src/test/java/org/apache/metron/solr/integration/SolrEnrichmentIntegrationTest.java
index e878912..8aa201e 100644
--- a/metron-platform/metron-solr/src/test/java/org/apache/metron/solr/integration/SolrEnrichmentIntegrationTest.java
+++ b/metron-platform/metron-solr/src/test/java/org/apache/metron/solr/integration/SolrEnrichmentIntegrationTest.java
@@ -24,6 +24,7 @@ import org.apache.metron.integration.ComponentRunner;
 import org.apache.metron.integration.InMemoryComponent;
 import org.apache.metron.integration.Processor;
 import org.apache.metron.integration.ReadinessState;
+import org.apache.metron.integration.components.KafkaWithZKComponent;
 import org.apache.metron.solr.integration.components.SolrComponent;
 import org.apache.metron.integration.utils.SampleUtil;
 import org.apache.metron.common.configuration.ConfigurationsUtils;
@@ -38,7 +39,6 @@ import java.util.Properties;
 public class SolrEnrichmentIntegrationTest extends EnrichmentIntegrationTest {
 
   private String collection = "metron";
-  private String solrZookeeperUrl;
 
   @Override
   public InMemoryComponent getSearchComponent(final Properties topologyProperties) throws Exception {
@@ -50,7 +50,7 @@ public class SolrEnrichmentIntegrationTest extends EnrichmentIntegrationTest {
               public Void apply(@Nullable SolrComponent solrComponent) {
                 topologyProperties.setProperty("solr.zk", solrComponent.getZookeeperUrl());
                 try {
-                  String testZookeeperUrl = topologyProperties.getProperty("kafka.zk");
+                  String testZookeeperUrl = topologyProperties.getProperty(KafkaWithZKComponent.ZOOKEEPER_PROPERTY);
                   Configurations configurations = SampleUtil.getSampleConfigs();
                   Map<String, Object> globalConfig = configurations.getGlobalConfig();
                   globalConfig.put("solr.zookeeper", solrComponent.getZookeeperUrl());

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/df8d682e/metron-platform/metron-solr/src/test/java/org/apache/metron/solr/writer/SolrWriterTest.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-solr/src/test/java/org/apache/metron/solr/writer/SolrWriterTest.java b/metron-platform/metron-solr/src/test/java/org/apache/metron/solr/writer/SolrWriterTest.java
index 24c8eab..580fd31 100644
--- a/metron-platform/metron-solr/src/test/java/org/apache/metron/solr/writer/SolrWriterTest.java
+++ b/metron-platform/metron-solr/src/test/java/org/apache/metron/solr/writer/SolrWriterTest.java
@@ -19,6 +19,7 @@ package org.apache.metron.solr.writer;
 
 import backtype.storm.tuple.Tuple;
 import org.apache.metron.common.configuration.Configurations;
+import org.apache.metron.common.configuration.EnrichmentConfigurations;
 import org.apache.metron.integration.utils.SampleUtil;
 import org.apache.solr.client.solrj.request.QueryRequest;
 import org.apache.solr.common.SolrInputDocument;
@@ -92,7 +93,7 @@ public class SolrWriterTest {
 
   @Test
   public void testWriter() throws Exception {
-    Configurations configurations = SampleUtil.getSampleConfigs();
+    EnrichmentConfigurations configurations = SampleUtil.getSampleEnrichmentConfigs();
     JSONObject message1 = new JSONObject();
     message1.put("intField", 100);
     message1.put("doubleField", 100.0);

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/df8d682e/metron-platform/metron-test-utilities/src/main/java/org/apache/metron/TestConstants.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-test-utilities/src/main/java/org/apache/metron/TestConstants.java b/metron-platform/metron-test-utilities/src/main/java/org/apache/metron/TestConstants.java
index c798158..2b39f9d 100644
--- a/metron-platform/metron-test-utilities/src/main/java/org/apache/metron/TestConstants.java
+++ b/metron-platform/metron-test-utilities/src/main/java/org/apache/metron/TestConstants.java
@@ -19,8 +19,11 @@ package org.apache.metron;
 
 public class TestConstants {
 
-  public final static String SAMPLE_CONFIG_PATH = "../metron-integration-test/src/main/resources/sample/config/";
-  public final static String SAMPLE_DATA_INPUT_PATH = "../metron-integration-test/src/main/resources/sample/data/SampleInput/";
-  public final static String SAMPLE_DATA_PARSED_PATH = "../metron-integration-test/src/main/resources/sample/data/SampleParsed/";
-  public final static String SAMPLE_DATA_INDEXED_PATH = "../metron-integration-test/src/main/resources/sample/data/SampleIndexed/";
+  public final static String SAMPLE_CONFIG_PATH = "../metron-integration-test/src/main/config/zookeeper/";
+  public final static String PARSER_CONFIGS_PATH = "../metron-parsers/src/main/config/zookeeper/";
+  public final static String ENRICHMENTS_CONFIGS_PATH = "../metron-enrichment/src/main/config/zookeeper/";
+  public final static String SAMPLE_DATA_PATH = "../metron-integration-test/src/main/sample/data/";
+  public final static String SAMPLE_DATA_INPUT_PATH = "../metron-integration-test/src/main/sample/data/yaf/raw/";
+  public final static String SAMPLE_DATA_PARSED_PATH = "../metron-integration-test/src/main/sample/data/test/parsed/";
+  public final static String SAMPLE_DATA_INDEXED_PATH = "../metron-integration-test/src/main/sample/data/test/indexed/";
 }

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/df8d682e/metron-platform/metron-test-utilities/src/main/java/org/apache/metron/test/TestDataType.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-test-utilities/src/main/java/org/apache/metron/test/TestDataType.java b/metron-platform/metron-test-utilities/src/main/java/org/apache/metron/test/TestDataType.java
new file mode 100644
index 0000000..d31cbfe
--- /dev/null
+++ b/metron-platform/metron-test-utilities/src/main/java/org/apache/metron/test/TestDataType.java
@@ -0,0 +1,31 @@
+/**
+ * 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.test;
+
+public enum TestDataType {
+
+  RAW("raw"),PARSED("parsed"),INDEXED("indexed");
+
+  private String directoryName;
+  TestDataType(String directoryName) {
+    this.directoryName = directoryName;
+  }
+  public String getDirectoryName() {
+    return directoryName;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/df8d682e/metron-platform/metron-test-utilities/src/main/java/org/apache/metron/test/bolt/BaseEnrichmentBoltTest.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-test-utilities/src/main/java/org/apache/metron/test/bolt/BaseEnrichmentBoltTest.java b/metron-platform/metron-test-utilities/src/main/java/org/apache/metron/test/bolt/BaseEnrichmentBoltTest.java
index 83bcb92..69c2174 100644
--- a/metron-platform/metron-test-utilities/src/main/java/org/apache/metron/test/bolt/BaseEnrichmentBoltTest.java
+++ b/metron-platform/metron-test-utilities/src/main/java/org/apache/metron/test/bolt/BaseEnrichmentBoltTest.java
@@ -29,16 +29,16 @@ import java.util.Set;
 
 public class BaseEnrichmentBoltTest extends BaseBoltTest {
 
-  public String sampleSensorEnrichmentConfigPath = TestConstants.SAMPLE_CONFIG_PATH + "sensors/yaf.json";
+  public String sampleSensorEnrichmentConfigPath = TestConstants.SAMPLE_CONFIG_PATH + "enrichments/test.json";
   protected Set<String> streamIds = new HashSet<>();
   protected String key = "someKey";
-  protected String sensorType = "yaf";
+  protected String sensorType = "test";
 
   /**
    * {
    * "ip_src_addr": "ip1",
    * "ip_dst_addr": "ip2",
-   * "source.type": "yaf"
+   * "source.type": "test"
    * }
    */
   @Multiline
@@ -48,7 +48,7 @@ public class BaseEnrichmentBoltTest extends BaseBoltTest {
    * {
    * "enrichments.geo.ip_src_addr": "ip1",
    * "enrichments.geo.ip_dst_addr": "ip2",
-   * "source.type": "yaf"
+   * "source.type": "test"
    * }
    */
   @Multiline
@@ -58,7 +58,7 @@ public class BaseEnrichmentBoltTest extends BaseBoltTest {
    * {
    * "enrichments.host.ip_src_addr": "ip1",
    * "enrichments.host.ip_dst_addr": "ip2",
-   * "source.type": "yaf"
+   * "source.type": "test"
    * }
    */
   @Multiline
@@ -68,7 +68,7 @@ public class BaseEnrichmentBoltTest extends BaseBoltTest {
    * {
    * "enrichments.hbaseEnrichment.ip_src_addr": "ip1",
    * "enrichments.hbaseEnrichment.ip_dst_addr": "ip2",
-   * "source.type": "yaf"
+   * "source.type": "test"
    * }
    */
   @Multiline

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/df8d682e/metron-platform/metron-test-utilities/src/main/java/org/apache/metron/test/utils/SampleDataUtils.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-test-utilities/src/main/java/org/apache/metron/test/utils/SampleDataUtils.java b/metron-platform/metron-test-utilities/src/main/java/org/apache/metron/test/utils/SampleDataUtils.java
new file mode 100644
index 0000000..0e3e4e6
--- /dev/null
+++ b/metron-platform/metron-test-utilities/src/main/java/org/apache/metron/test/utils/SampleDataUtils.java
@@ -0,0 +1,41 @@
+/**
+ * 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.test.utils;
+
+import org.apache.metron.TestConstants;
+import org.apache.metron.test.TestDataType;
+
+import java.io.File;
+import java.io.FileNotFoundException;
+
+public class SampleDataUtils {
+
+  public static String getSampleDataPath(String sensorType, TestDataType testDataType) throws FileNotFoundException {
+    File sensorSampleDataPath = new File(TestConstants.SAMPLE_DATA_PATH, sensorType);
+    if (sensorSampleDataPath.exists() && sensorSampleDataPath.isDirectory()) {
+      File sampleDataPath = new File(sensorSampleDataPath, testDataType.getDirectoryName());
+      if (sampleDataPath.exists() && sampleDataPath.isDirectory()) {
+        File[] children = sampleDataPath.listFiles();
+        if (children != null && children.length > 0) {
+          return children[0].getAbsolutePath();
+        }
+      }
+    }
+    throw new FileNotFoundException("Could not find data in " + TestConstants.SAMPLE_DATA_PATH + sensorType + "/" + testDataType.getDirectoryName());
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/df8d682e/metron-platform/metron-test-utilities/src/main/java/org/apache/metron/test/utils/ValidationUtils.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-test-utilities/src/main/java/org/apache/metron/test/utils/ValidationUtils.java b/metron-platform/metron-test-utilities/src/main/java/org/apache/metron/test/utils/ValidationUtils.java
new file mode 100644
index 0000000..e08c697
--- /dev/null
+++ b/metron-platform/metron-test-utilities/src/main/java/org/apache/metron/test/utils/ValidationUtils.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.test.utils;
+
+import org.codehaus.jackson.map.ObjectMapper;
+import org.junit.Assert;
+
+import java.io.IOException;
+import java.util.Map;
+
+public class ValidationUtils {
+
+  public static void assertJSONEqual(String expected, String actual) throws IOException {
+    ObjectMapper mapper = new ObjectMapper();
+    Map m1 = mapper.readValue(expected, Map.class);
+    Map m2 = mapper.readValue(actual, Map.class);
+    for(Object k : m1.keySet()) {
+      Object v1 = m1.get(k);
+      Object v2 = m2.get(k);
+
+      if(v2 == null) {
+        Assert.fail("Unable to find key: " + k + " in output");
+      }
+      if(k.equals("timestamp")) {
+        //TODO: Take the ?!?@ timestamps out of the reference file.
+        Assert.assertEquals(v1.toString().length(), v2.toString().length());
+      }
+      else if(!v2.equals(v1)) {
+        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/df8d682e/metron-platform/pom.xml
----------------------------------------------------------------------
diff --git a/metron-platform/pom.xml b/metron-platform/pom.xml
index e7d818c..14bf06a 100644
--- a/metron-platform/pom.xml
+++ b/metron-platform/pom.xml
@@ -203,9 +203,9 @@
 						<exclude>**/*.json</exclude>
 						<exclude>**/*.log</exclude>
 						<exclude>**/src/main/resources/patterns/**</exclude>
-						<exclude>**/src/main/resources/sample/data/SampleIndexed/**</exclude>
-						<exclude>**/src/main/resources/sample/data/SampleInput/**</exclude>
-						<exclude>**/src/main/resources/sample/data/SampleParsed/**</exclude>
+						<exclude>**/src/main/sample/data/SampleIndexed/**</exclude>
+						<exclude>**/src/main/sample/data/SampleInput/**</exclude>
+						<exclude>**/src/main/sample/data/SampleParsed/**</exclude>
 						<exclude>**/dependency-reduced-pom.xml</exclude>
 					</excludes>
 				</configuration>

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/df8d682e/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 762abe6..f6a3b63 100644
--- a/pom.xml
+++ b/pom.xml
@@ -77,9 +77,9 @@
                         <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/main/sample/patterns/**</exclude>
                         <exclude>**/src/test/resources/**</exclude>
-                        <exclude>**/src/main/resources/sample/data/Sample*/**</exclude>
+                        <exclude>**/src/main/sample/data/**</exclude>
                         <exclude>**/dependency-reduced-pom.xml</exclude>
                         <exclude>**/files/opensoc-ui</exclude>
                         <exclude>**/target/**</exclude>


[07/15] incubator-metron git commit: METRON-142 Simplify Parser configuration (merrimanr via cestella) closes apache/incubator-metron#120

Posted by ce...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/df8d682e/metron-platform/metron-integration-test/src/main/sample/data/SampleInput/AsaOutput
----------------------------------------------------------------------
diff --git a/metron-platform/metron-integration-test/src/main/sample/data/SampleInput/AsaOutput b/metron-platform/metron-integration-test/src/main/sample/data/SampleInput/AsaOutput
new file mode 100644
index 0000000..6009d48
--- /dev/null
+++ b/metron-platform/metron-integration-test/src/main/sample/data/SampleInput/AsaOutput
@@ -0,0 +1,100 @@
+<167>Jan  5 08:52:35 10.22.8.216 %ASA-7-609001: Built local-host inside:10.22.8.205
+<166>Jan  5 08:52:35 10.22.8.216 %ASA-6-302021: Teardown ICMP connection for faddr 10.22.8.74/0(LOCAL\user.name) gaddr 10.22.8.205/0 laddr 10.22.8.205/0
+<167>Jan  5 08:52:35 10.22.8.216 %ASA-7-609002: Teardown local-host inside:10.22.8.205 duration 0:00:00
+<142>Jan  5 08:52:35 10.22.8.201 %ASA-6-302014: Teardown TCP connection 488167725 for Outside_VPN:147.111.72.16/26436 to DMZ-Inside:10.22.8.53/443 duration 0:00:00 bytes 9687 TCP FINs
+<166>Jan  5 08:52:35 10.22.8.216 %ASA-6-302014: Teardown TCP connection 212805593 for outside:10.22.8.223/59614(LOCAL\user.name) to inside:10.22.8.78/8102 duration 0:00:07 bytes 3433 TCP FINs (user.name)
+<174>Jan  5 14:52:35 10.22.8.212 %ASA-6-302013: Built inbound TCP connection 76245503 for outside:10.22.8.233/54209 (10.22.8.233/54209) to inside:198.111.72.238/443 (198.111.72.238/443) (user.name)
+<166>Jan  5 08:52:35 10.22.8.216 %ASA-6-302013: Built inbound TCP connection 212806031 for outside:10.22.8.17/58633 (10.22.8.17/58633)(LOCAL\user.name) to inside:10.22.8.12/389 (10.22.8.12/389) (user.name)
+<142>Jan  5 08:52:35 10.22.8.201 %ASA-6-302014: Teardown TCP connection 488168292 for DMZ-Inside:10.22.8.51/51231 to Inside-Trunk:10.22.8.174/40004 duration 0:00:00 bytes 2103 TCP FINs
+<142>Jan  5 08:52:35 10.22.8.201 %ASA-6-106015: Deny TCP (no connection) from 186.111.72.11/80 to 204.111.72.226/45019 flags SYN ACK  on interface Outside_VPN
+<166>Jan  5 09:52:35 10.22.8.12 %ASA-6-302014: Teardown TCP connection 17604987 for outside:209.111.72.151/443 to inside:10.22.8.188/64306 duration 0:00:31 bytes 10128 TCP FINs
+<166>Jan  5 09:52:35 10.22.8.12 %ASA-6-302014: Teardown TCP connection 17604999 for outside:209.111.72.151/443 to inside:10.22.8.188/64307 duration 0:00:30 bytes 6370 TCP FINs
+<142>Jan  5 08:52:35 10.22.8.201 %ASA-6-302014: Teardown TCP connection 488167347 for Outside_VPN:198.111.72.24/2134 to DMZ-Inside:10.22.8.53/443 duration 0:00:01 bytes 9785 TCP FINs
+<174>Jan  5 14:52:35 10.22.8.212 %ASA-6-302015: Built inbound UDP connection 76245506 for outside:10.22.8.110/49886 (10.22.8.110/49886) to inside:192.111.72.8/8612 (192.111.72.8/8612) (user.name)
+<166>Jan  5 08:52:35 10.22.8.216 %ASA-6-302014: Teardown TCP connection 212805993 for outside:10.22.8.89/56917(LOCAL\user.name) to inside:216.111.72.126/443 duration 0:00:00 bytes 0 TCP FINs (user.name)
+<167>Jan  5 08:52:35 10.22.8.216 %ASA-7-710005: UDP request discarded from 10.22.8.223/49192 to outside:224.111.72.252/5355
+<142>Jan  5 08:52:35 10.22.8.201 %ASA-6-302014: Teardown TCP connection 488166143 for Outside_VPN:198.111.72.64/80 to Inside-Trunk:10.22.8.39/54883 duration 0:00:04 bytes 1148 TCP FINs
+<166>Jan  5 08:52:35 10.22.8.216 %ASA-6-106015: Deny TCP (no connection) from 10.22.8.84/445 to 10.22.8.219/60726 flags ACK  on interface inside
+<142>Jan  5 08:52:35 10.22.8.201 %ASA-6-302014: Teardown TCP connection 488168344 for DMZ-Inside:10.22.8.53/61682 to Inside-Trunk:10.22.8.174/40004 duration 0:00:00 bytes 5648 TCP FINs
+<142>Jan  5 08:52:35 10.22.8.201 %ASA-6-302014: Teardown TCP connection 488168345 for DMZ-Inside:10.22.8.16/31454 to Inside-Trunk:10.22.8.21/443 duration 0:00:00 bytes 756 TCP FINs
+<182>Jan  5 20:22:35 10.22.8.4 %ASA-6-302020: Built inbound ICMP connection for faddr 10.22.8.12/0 gaddr 10.22.8.45/1 laddr 10.22.8.45/1
+<142>Jan  5 08:52:35 10.22.8.201 %ASA-6-106015: Deny TCP (no connection) from 50.111.72.230/80 to 204.111.72.254/53077 flags RST  on interface Outside_VPN
+<166>Jan  5 09:52:35 10.22.8.12 %ASA-6-302016: Teardown UDP connection 17603649 for outside:206.111.72.2/161 to inside:10.22.8.48/63297 duration 0:02:01 bytes 209
+<166>Jan  5 09:52:35 10.22.8.12 %ASA-6-302016: Teardown UDP connection 17603650 for outside:207.111.72.122/161 to inside:10.22.8.48/63298 duration 0:02:01 bytes 209
+<166>Jan  5 09:52:35 10.22.8.12 %ASA-6-302016: Teardown UDP connection 17603652 for outside:206.111.72.2/161 to inside:10.22.8.48/63300 duration 0:02:01 bytes 115
+<166>Jan  5 09:52:35 10.22.8.12 %ASA-6-302016: Teardown UDP connection 17603657 for outside:206.111.72.2/161 to inside:10.22.8.48/63306 duration 0:02:01 bytes 115
+<142>Jan  5 08:52:35 10.22.8.201 %ASA-6-302014: Teardown TCP connection 488168436 for DMZ-Inside:10.22.8.51/51235 to Inside-Trunk:10.22.8.174/40004 duration 0:00:00 bytes 2497 TCP FINs
+<142>Jan  5 08:52:35 10.22.8.201 %ASA-6-302014: Teardown TCP connection 488167656 for Outside_VPN:69.111.72.70/21560 to DMZ-Inside:10.22.8.53/443 duration 0:00:01 bytes 11410 TCP FINs
+<166>Jan  5 08:52:35 10.22.8.216 %ASA-6-302015: Built inbound UDP connection 212806050 for outside:10.22.8.62/53965 (10.22.8.62/53965)(LOCAL\user.name) to inside:10.22.8.85/53 (10.22.8.85/53) (user.name)
+<166>Jan  5 08:52:35 10.22.8.216 %ASA-6-302013: Built inbound TCP connection 212806052 for outside:10.22.8.62/56500 (10.22.8.62/56500)(LOCAL\user.name) to inside:198.111.72.83/443 (198.111.72.83/443) (user.name)
+<166>Jan  5 08:52:35 10.22.8.216 %ASA-6-302013: Built inbound TCP connection 212806054 for outside:10.22.8.62/56502 (10.22.8.62/56502)(LOCAL\user.name) to inside:50.111.72.252/443 (50.111.72.252/443) (user.name)
+<166>Jan  5 09:52:35 10.22.8.12 %ASA-6-305011: Built dynamic TCP translation from inside:10.22.8.188/64340 to outside:206.111.72.41/2013
+<166>Jan  5 15:52:35 10.22.8.33 %ASA-6-305012: Teardown dynamic UDP translation from inside:192.111.72.2/62251 to outside:79.111.72.174/21311 duration 0:02:30
+<166>Jan  5 08:52:35 10.22.8.216 %ASA-6-302015: Built inbound UDP connection 212806058 for outside:10.22.8.221/56631 (10.22.8.221/56631)(LOCAL\user.name) to inside:10.22.8.26/389 (10.22.8.26/389) (user.name)
+<142>Jan  5 08:52:35 10.22.8.201 %ASA-6-302014: Teardown TCP connection 488168189 for Outside_VPN:209.111.72.10/56619 to DMZ-Inside:10.22.8.53/443 duration 0:00:00 bytes 2477 TCP FINs
+<142>Jan  5 08:52:35 10.22.8.201 %ASA-6-106015: Deny TCP (no connection) from 10.22.8.112/52235 to 198.111.72.227/80 flags ACK  on interface Inside-Trunk
+<142>Jan  5 08:52:35 10.22.8.201 %ASA-6-302014: Teardown TCP connection 488167192 for Outside_VPN:115.111.72.7/49196 to DMZ-Inside:10.22.8.57/443 duration 0:00:02 bytes 20588 TCP Reset-O
+<166>Jan  5 08:52:35 10.22.8.216 %ASA-6-302016: Teardown UDP connection 212806055 for outside:10.22.8.62/55383(LOCAL\user.name) to inside:10.22.8.85/53 duration 0:00:00 bytes 349 (user.name)
+<142>Jan  5 08:52:35 10.22.8.201 %ASA-6-302014: Teardown TCP connection 488168380 for Outside_VPN:74.111.72.12/443 to Inside-Trunk:10.22.8.39/54894 duration 0:00:00 bytes 5701 TCP FINs
+<174>Jan  5 14:52:35 10.22.8.212 %ASA-6-302013: Built inbound TCP connection 76245522 for outside:10.22.8.147/56343 (10.22.8.147/56343) to inside:209.111.72.151/443 (209.111.72.151/443) (user.name)
+<142>Jan  5 08:52:35 10.22.8.201 %ASA-6-302014: Teardown TCP connection 488168443 for Outside_VPN:23.111.72.27/80 to Inside-Trunk:10.22.8.81/64713 duration 0:00:00 bytes 2426 TCP FINs
+<142>Jan  5 08:52:35 10.22.8.201 %ASA-6-302014: Teardown TCP connection 488111566 for Outside_VPN:131.111.72.49/443 to Inside-Trunk:10.22.8.127/56558 duration 0:01:57 bytes 3614 TCP Reset-O
+<166>Jan  5 08:52:35 10.22.8.216 %ASA-6-302013: Built inbound TCP connection 212806061 for outside:10.22.8.17/58635 (10.22.8.17/58635)(LOCAL\user.name) to inside:10.22.8.12/389 (10.22.8.12/389) (user.name)
+<166>Jan  5 08:52:35 10.22.8.216 %ASA-6-302014: Teardown TCP connection 212806010 for outside:10.22.8.33/60223(LOCAL\user.name) to inside:10.22.8.86/389 duration 0:00:00 bytes 416 TCP Reset-I (user.name)
+<166>Jan  5 08:52:35 10.22.8.216 %ASA-6-302015: Built inbound UDP connection 212806062 for outside:10.22.8.221/56632 (10.22.8.221/56632)(LOCAL\user.name) to inside:10.22.8.73/389 (10.22.8.73/389) (user.name)
+<167>Jan  5 08:52:35 10.22.8.216 %ASA-7-609002: Teardown local-host inside:10.22.8.205 duration 0:00:00
+<142>Jan  5 08:52:35 10.22.8.201 %ASA-6-302014: Teardown TCP connection 488168231 for Outside_VPN:204.111.72.243/3011 to Inside-Trunk:10.22.8.208/60037 duration 0:00:00 bytes 19415 TCP FINs
+<166>Jan  5 16:52:35 10.22.8.41 %ASA-6-302013: Built inbound TCP connection 45476108 for Outside:10.22.8.97/53484 (10.22.8.97/53484)(LOCAL\user.name) to Inside:141.111.72.70/7576 (141.111.72.70/7576) (user.name)
+<174>Jan  5 14:52:35 10.22.8.212 %ASA-6-302013: Built inbound TCP connection 76245527 for outside:10.22.8.97/65195 (10.22.8.97/65195) to inside:17.111.72.212/5223 (17.111.72.212/5223) (user.name)
+<166>Jan  5 08:52:35 10.22.8.216 %ASA-6-302014: Teardown TCP connection 212806018 for outside:10.22.8.17/58632(LOCAL\user.name) to inside:10.22.8.12/389 duration 0:00:00 bytes 0 TCP FINs (user.name)
+<142>Jan  5 08:52:35 10.22.8.201 %ASA-6-302014: Teardown TCP connection 488168562 for DMZ-Inside:10.22.8.51/51236 to Inside-Trunk:10.22.8.174/40004 duration 0:00:00 bytes 2273 TCP FINs
+<166>Jan  5 08:52:35 10.22.8.216 %ASA-6-302015: Built inbound UDP connection 212806065 for outside:10.22.8.62/59829 (10.22.8.62/59829)(LOCAL\user.name) to inside:10.22.8.85/53 (10.22.8.85/53) (user.name)
+<166>Jan  5 08:52:35 10.22.8.216 %ASA-6-302013: Built inbound TCP connection 212806067 for outside:10.22.8.143/62675 (10.22.8.143/62675)(LOCAL\user.name) to inside:141.111.72.12/389 (141.111.72.12/389) (user.name)
+<167>Jan  5 08:52:35 10.22.8.216 %ASA-7-710005: UDP request discarded from 10.22.8.223/61122 to outside:224.111.72.252/5355
+<166>Jan  5 08:52:35 10.22.8.216 %ASA-6-302020: Built inbound ICMP connection for faddr 10.22.8.143/0(LOCAL\user.name) gaddr 141.111.72.12/0 laddr 141.111.72.12/0 (user.name)
+<142>Jan  5 08:52:35 10.22.8.201 %ASA-6-302014: Teardown TCP connection 488168547 for Outside_VPN:107.111.72.102/80 to Inside-Trunk:10.22.8.54/61676 duration 0:00:00 bytes 1030 TCP FINs
+<166>Jan  5 08:52:35 10.22.8.216 %ASA-6-302015: Built inbound UDP connection 212806078 for outside:10.22.8.221/56633 (10.22.8.221/56633)(LOCAL\user.name) to inside:10.22.8.20/389 (10.22.8.20/389) (user.name)
+<166>Jan  5 09:52:35 10.22.8.12 %ASA-6-305011: Built dynamic TCP translation from inside:10.22.8.83/59915 to outside:206.111.72.41/22776
+<142>Jan  5 08:52:36 10.22.8.201 %ASA-6-302014: Teardown TCP connection 488168044 for Outside_VPN:50.111.72.39/80 to Inside-Trunk:10.22.8.75/60877 duration 0:00:01 bytes 13304 TCP FINs
+<142>Jan  5 08:52:36 10.22.8.201 %ASA-6-302014: Teardown TCP connection 488118326 for Outside_VPN:23.111.72.27/80 to Inside-Trunk:10.22.8.229/57901 duration 0:01:45 bytes 1942 TCP FINs
+<142>Jan  5 08:52:36 10.22.8.201 %ASA-6-302014: Teardown TCP connection 488160565 for Outside_VPN:72.111.72.29/80 to Inside-Trunk:10.22.8.42/57520 duration 0:00:15 bytes 1025 TCP FINs
+<142>Jan  5 08:52:36 10.22.8.201 %ASA-6-302014: Teardown TCP connection 488096423 for Outside_VPN:72.111.72.43/80 to Inside-Trunk:10.22.8.127/59096 duration 0:02:27 bytes 99347 TCP Reset-O
+<142>Jan  5 08:52:36 10.22.8.201 %ASA-6-302014: Teardown TCP connection 488095522 for Outside_VPN:72.111.72.43/80 to Inside-Trunk:10.22.8.127/59087 duration 0:02:29 bytes 154785 TCP Reset-O
+<142>Jan  5 08:52:36 10.22.8.201 %ASA-6-302014: Teardown TCP connection 488106557 for Outside_VPN:72.111.72.43/80 to Inside-Trunk:10.22.8.127/59134 duration 0:02:09 bytes 25319 TCP Reset-O
+<142>Jan  5 08:52:36 10.22.8.201 %ASA-6-302014: Teardown TCP connection 488096426 for Outside_VPN:72.111.72.43/80 to Inside-Trunk:10.22.8.127/59099 duration 0:02:27 bytes 26171 TCP Reset-O
+<166>Jan  5 08:52:36 10.22.8.216 %ASA-6-302014: Teardown TCP connection 212806005 for outside:10.22.8.17/58630(LOCAL\user.name) to inside:10.22.8.12/389 duration 0:00:00 bytes 3942 TCP FINs (user.name)
+<166>Jan  5 08:52:36 10.22.8.216 %ASA-6-302015: Built inbound UDP connection 212806085 for outside:10.22.8.143/54018 (10.22.8.143/54018)(LOCAL\user.name) to inside:10.22.8.85/53 (10.22.8.85/53) (user.name)
+<174>Jan  5 14:52:36 10.22.8.212 %ASA-6-302020: Built inbound ICMP connection for faddr 10.22.8.96/2708 gaddr 10.22.8.30/0 laddr 10.22.8.30/0 (user.name)
+<174>Jan  5 14:52:36 10.22.8.212 %ASA-6-302015: Built inbound UDP connection 76245537 for outside:10.22.8.110/49886 (10.22.8.110/49886) to inside:192.111.72.11/8612 (192.111.72.11/8612) (user.name)
+<166>Jan  5 16:52:36 10.22.8.41 %ASA-6-106015: Deny TCP (no connection) from 10.22.8.85/58359 to 10.22.8.11/88 flags RST ACK  on interface Outside
+<166>Jan  5 08:52:36 10.22.8.216 %ASA-6-302021: Teardown ICMP connection for faddr 10.22.8.82/0(LOCAL\user.name) gaddr 10.22.8.205/0 laddr 10.22.8.205/0
+<166>Jan  5 08:52:36 10.22.8.216 %ASA-6-302016: Teardown UDP connection 212799832 for outside:10.22.8.230/55549(LOCAL\user.name) to inside:10.22.8.11/389 duration 0:02:01 bytes 354 (user.name)
+<166>Jan  5 08:52:36 10.22.8.216 %ASA-6-302016: Teardown UDP connection 212799867 for outside:10.22.8.240/138(LOCAL\user.name) to inside:10.22.8.255/138 duration 0:02:01 bytes 214 (user.name)
+<167>Jan  5 08:52:36 10.22.8.216 %ASA-7-609001: Built local-host inside:67.111.72.204
+<174>Jan  5 14:52:36 10.22.8.212 %ASA-6-302013: Built inbound TCP connection 76245544 for outside:10.22.8.227/54540 (10.22.8.227/54540) to inside:63.111.72.124/80 (63.111.72.124/80) (user.name)
+<142>Jan  5 08:52:36 10.22.8.201 %ASA-6-302014: Teardown TCP connection 488168135 for Outside_VPN:198.111.72.66/36797 to DMZ-Inside:10.22.8.53/80 duration 0:00:01 bytes 89039 TCP FINs
+<166>Jan  5 08:52:36 10.22.8.216 %ASA-6-302014: Teardown TCP connection 212805836 for outside:10.22.8.62/56471(LOCAL\user.name) to inside:208.111.72.1/443 duration 0:00:04 bytes 1700 TCP FINs (user.name)
+<174>Jan  5 14:52:36 10.22.8.212 %ASA-6-302013: Built inbound TCP connection 76245546 for outside:10.22.8.227/54542 (10.22.8.227/54542) to inside:63.111.72.124/80 (63.111.72.124/80) (user.name)
+<166>Jan  5 08:52:36 10.22.8.216 %ASA-6-302021: Teardown ICMP connection for faddr 10.22.8.74/0(LOCAL\user.name) gaddr 10.22.8.205/0 laddr 10.22.8.205/0
+<174>Jan  5 14:52:36 10.22.8.212 %ASA-6-302020: Built outbound ICMP connection for faddr 10.22.8.96/2708 gaddr 10.22.8.30/0 laddr 10.22.8.30/0
+<142>Jan  5 08:52:36 10.22.8.201 %ASA-6-302014: Teardown TCP connection 488168388 for DMZ-Inside:10.22.8.10/49771 to Inside-Trunk:10.22.8.128/443 duration 0:00:00 bytes 19132 TCP Reset-O
+<142>Jan  5 08:52:36 10.22.8.201 %ASA-6-302014: Teardown TCP connection 488168692 for DMZ-Inside:10.22.8.53/61694 to Inside-Trunk:10.22.8.174/40004 duration 0:00:00 bytes 5660 TCP FINs
+<174>Jan  5 14:52:36 10.22.8.212 %ASA-6-302013: Built inbound TCP connection 76245552 for outside:10.22.8.92/51042 (10.22.8.92/51042) to inside:10.22.8.193/9100 (10.22.8.193/9100) (user.name)
+<166>Jan  5 16:52:36 10.22.8.41 %ASA-6-302016: Teardown UDP connection 45474680 for Outside:10.22.8.49/137(LOCAL\user.name) to Inside:10.22.8.12/137 duration 0:02:03 bytes 486 (user.name)
+<166>Jan  5 16:52:36 10.22.8.41 %ASA-6-302016: Teardown UDP connection 45474694 for Outside:10.22.8.49/138(LOCAL\user.name) to Inside:10.22.8.12/138 duration 0:02:01 bytes 184 (user.name)
+<142>Jan  5 08:52:36 10.22.8.201 %ASA-6-302014: Teardown TCP connection 488167720 for Outside_VPN:198.111.72.75/1033 to DMZ-Inside:10.22.8.53/443 duration 0:00:01 bytes 9634 TCP FINs
+<142>Jan  5 08:52:32 10.22.8.201 %ASA-6-302014: Teardown TCP connection 488165627 for Outside_VPN:170.111.72.22/27463 to DMZ-Inside:10.22.8.53/443 duration 0:00:01 bytes 9756 TCP FINs
+<166>Jan  5 08:52:32 10.22.8.216 %ASA-6-302016: Teardown UDP connection 212805854 for outside:10.22.8.62/54704(LOCAL\user.name) to inside:10.22.8.85/53 duration 0:00:00 bytes 114 (user.name)
+<166>Jan  5 09:52:32 10.22.8.12 %ASA-6-302020: Built inbound ICMP connection for faddr 207.111.72.122/0 gaddr 206.111.72.24/512 laddr 10.22.8.57/512
+<166>Jan  5 09:52:32 10.22.8.12 %ASA-6-302013: Built outbound TCP connection 17605397 for outside:69.111.72.0/80 (69.111.72.0/80) to inside:10.22.8.102/55659 (206.111.72.41/40627)
+<174>Jan  5 14:52:32 10.22.8.212 %ASA-6-302015: Built inbound UDP connection 76245230 for outside:10.22.8.96/123 (10.22.8.96/123) to inside:10.22.8.12/123 (10.22.8.12/123) (user.name)
+<142>Jan  5 08:52:32 10.22.8.201 %ASA-6-302014: Teardown TCP connection 488031413 for Outside_VPN:184.111.72.216/50341 to DMZ-Inside:10.22.8.57/443 duration 0:05:01 bytes 13543 TCP Reset-O
+<166>Jan  5 16:52:32 10.22.8.41 %ASA-6-302020: Built inbound ICMP connection for faddr 10.22.8.95/1(LOCAL\user.name) gaddr 10.22.8.12/0 laddr 10.22.8.12/0 (user.name)
+<142>Jan  5 08:52:32 10.22.8.201 %ASA-6-302014: Teardown TCP connection 488030393 for DMZ-Inside:[10.22.8.10/57109 to Inside-Trunk:10.22.8.128/443 duration 0:05:04 bytes 13541 TCP Reset-O
+<166>Jan  5 09:52:32 10.22.8.12 %ASA-6-305012: Teardown dynamic TCP translation from inside:10.22.8.149/62156 to outside:206.111.72.41/19576 duration 0:00:44
+<166>Jan  5 09:52:32 10.22.8.12 %ASA-6-305012: Teardown dynamic TCP translation from inside:10.22.8.149/62159 to outside:206.111.72.41/39634 duration 0:00:44
+<142>Jan  5 08:52:32 10.22.8.201 %ASA-6-302014: Teardown TCP connection 488031793 for Outside_VPN:198.111.72.146/28026 to DMZ-Inside:10.22.8.53/443 duration 0:05:00 bytes 119 TCP FINs
+<142>Jan  5 08:52:32 10.22.8.201 %ASA-6-302014: Teardown TCP connection 488030810 for DMZ-Inside:10.22.8.10/56930 to Inside-Trunk:10.22.8.128/443 duration 0:05:03 bytes 13543 TCP Reset-O
+<142>Jan  5 08:52:32 10.22.8.201 %ASA-6-106015: Deny TCP (no connection) from 186.111.72.11/80 to 204.111.72.199/61438 flags SYN ACK  on interface Outside_VPN
+<166>Jan  5 08:52:32 10.22.8.216 %ASA-6-302013: Built inbound TCP connection 212805863 for outside:10.22.8.144/61999 (10.22.8.144/61999)(LOCAL\user.name) to inside:10.22.8.163/80 (10.22.8.163/80) (user.name)
+<167>Jan  5 08:52:32 10.22.8.216 %ASA-7-609002: Teardown local-host inside:10.22.8.205 duration 0:00:00
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/df8d682e/metron-platform/metron-integration-test/src/main/sample/data/SampleInput/FireeyeExampleOutput
----------------------------------------------------------------------
diff --git a/metron-platform/metron-integration-test/src/main/sample/data/SampleInput/FireeyeExampleOutput b/metron-platform/metron-integration-test/src/main/sample/data/SampleInput/FireeyeExampleOutput
new file mode 100644
index 0000000..0210010
--- /dev/null
+++ b/metron-platform/metron-integration-test/src/main/sample/data/SampleInput/FireeyeExampleOutput
@@ -0,0 +1,90 @@
+<164>fenotify-3483808.2.alert: 1::~~User-Agent: WinHttpClient::~~Host: www.microads.me::~~Connection: Keep-Alive::~~::~~GET /files/microads/update/InjectScript.js HTTP/1.1::~~User-Agent: WinHttpClient::~~Host: www.microads.me::~~Connection: Keep-Alive::~~::~~GET /files/microads/update/InjectScript.js HTTP/1.1::~~User-Agent: WinHttpClient::~~Host: www.microads.me::~~Connection: Keep-Alive::~~::~~GET /files/microads/update/InjectScript.js HTTP/1.1::~~User-Agent: WinHttpClient::~~Host: www.microads.me::~~Connection: Keep-Alive::~~::~~GET /files/microads/update/InjectScript.js HTTP/1.1::~~User-Agent: WinHttpClient::~~Host: www.microads.me::~~Connection: Keep-Alive::~~::~~GET /files/microads/update/InjectScript.js HTTP/1.1::~~User-Agent: WinHttpClient::~~Host: www.microads.me::~~Connection: Keep-Alive::~~::~~GET /files/microads/update/InjectScript.js HTTP/1.1::~~User-Agent: WinHttpClient::~~Host: www.microads.me::~~Connection: Keep-Alive::~~::~~GET /files/microads/update/InjectScript.js 
 HTTP
+<164>fenotify-793972.2.alert: ontrol: no-cache::~~::~~ dmac=00:1d:a2:af:32:a1 cs1Label=sname cs1=Exploit.Kit.Magnitude 
+<164>fenotify-797180.2.alert: 0.8::~~User-Agent: Mozilla/5.0 (Windows NT 6.1; WOW64) AppleWebKit/537.36 (KHTML, like Gecko) Chrome/39.0.2171.95 Safari/537.36::~~Accept-Encoding: gzip, deflate, sdch::~~Accept-Language: en-US,en;q\=0.8::~~::~~ dmac=00:1d:a2:af:32:a1 cs1Label=sname cs1=Trojan.Redirector 
+<164>fenotify-3483808.3.alert: /1.1::~~User-Agent: WinHttpClient::~~Host: www.microads.me::~~Connection: Keep-Alive::~~::~~GET /files/microads/update/InjectScript.js HTTP/1.1::~~User-Agent: WinHttpClient::~~Host: www.microads.me::~~Connection: Keep-Alive::~~::~~GET /files/microads/update/InjectScript.js HTTP/1.1::~~User-Agent: WinHttpClient::~~Host: www.microads.me::~~Connection: Keep-Alive::~~::~~GET /files/microads/update/InjectScript.js HTTP/1.1::~~User-Agent: WinHttpClient::~~Host: www.microads.me::~~Connection: Keep-Alive::~~::~~GET /files/microads/update/InjectScript.js HTTP/1.1::~~User-Agent: WinHttpClient::~~Host: www.microads.me::~~Connection: Keep-Alive::~~::~~GET /files/microads/update/InjectScript.js HTTP/1.1::~~User-Agent: WinHttpClient::~~Host: www.microad cs2Label=anomaly cs2=misc-anomaly cs1Label=sname cs1=Malware.Binary 
+<164>fenotify-791429.2.alert: t: rapidvideohere.pw::~~Connection: Keep-Alive::~~::~~ dmac=88:43:e1:95:13:29 cs1Label=sname cs1=Exploit.Kit.Magnitude 
+<164>fenotify-851777.alert: CEF:0|FireEye|CMS|7.2.1.244420|DM|domain-match|1|rt=Feb 09 2015 00:27:43 UTC dvc=10.201.78.190 cn3Label=cncPort cn3=53 cn2Label=sid cn2=80494706 shost=dev001srv02.example.com proto=udp cs5Label=cncHost cs5=mfdclk001.org dvchost=DEVFEYE1 spt=61395 dvc=10.100.25.16 smac=00:00:0c:07:ac:00 cn1Label=vlan cn1=0 externalId=851777 cs4Label=link cs4=https://DEVCMS01.example.com/event_stream/events_for_bot?ev_id\=851777 dmac=00:1d:a2:af:32:a1 cs1Label=sname cs1=Trojan.Generic.DNS 
+<164>fenotify-851901.alert: CEF:0|FireEye|CMS|7.2.1.244420|DM|domain-match|1|rt=Feb 09 2015 08:56:45 UTC dvc=10.201.78.6 cn3Label=cncPort cn3=53 cn2Label=sid cn2=80494706 shost=dev001srv02.example.com proto=udp cs5Label=cncHost cs5=mfdclk001.org dvchost=DEVFEYE1 spt=59131 dvc=10.100.25.16 smac=00:00:0c:07:ac:c8 cn1Label=vlan cn1=0 externalId=851901 cs4Label=link cs4=https://DEVCMS01.example.com/event_stream/events_for_bot?ev_id\=851901 dmac=88:43:e1:95:13:29 cs1Label=sname cs1=Trojan.Generic.DNS 
+<164>fenotify-851980.alert: CEF:0|FireEye|CMS|7.2.1.244420|DM|domain-match|1|rt=Feb 09 2015 12:23:51 UTC dvc=10.201.78.20 cn3Label=cncPort cn3=53 cn2Label=sid cn2=80494706 shost=dev001srv02.example.com proto=udp cs5Label=cncHost cs5=mfdclk001.org dvchost=DEVFEYE1 spt=53295 dvc=10.100.25.16 smac=00:00:0c:07:ac:00 cn1Label=vlan cn1=0 externalId=851980 cs4Label=link cs4=https://DEVCMS01.example.com/event_stream/events_for_bot?ev_id\=851980 dmac=00:1d:a2:af:32:a1 cs1Label=sname cs1=Trojan.Generic.DNS 
+<164>fenotify-851795.alert: CEF:0|FireEye|CMS|7.2.1.244420|DM|domain-match|1|rt=Feb 09 2015 02:19:05 UTC dvc=10.201.78.37 cn3Label=cncPort cn3=53 cn2Label=sid cn2=80494706 shost=dev001srv02.example.com proto=udp cs5Label=cncHost cs5=mfdclk001.org dvchost=DEVFEYE1 spt=54975 dvc=10.100.25.16 smac=00:00:0c:07:ac:00 cn1Label=vlan cn1=0 externalId=851795 cs4Label=link cs4=https://DEVCMS01.example.com/event_stream/events_for_bot?ev_id\=851795 dmac=00:1d:a2:af:32:a1 cs1Label=sname cs1=Trojan.Generic.DNS 
+<164>fenotify-851805.alert: CEF:0|FireEye|CMS|7.2.1.244420|DM|domain-match|1|rt=Feb 09 2015 03:23:14 UTC dvc=10.201.78.113 cn3Label=cncPort cn3=53 cn2Label=sid cn2=80494706 shost=dev001srv02.example.com proto=udp cs5Label=cncHost cs5=mfdclk001.org dvchost=DEVFEYE1 spt=50807 dvc=10.100.25.16 smac=00:00:0c:07:ac:00 cn1Label=vlan cn1=0 externalId=851805 cs4Label=link cs4=https://DEVCMS01.example.com/event_stream/events_for_bot?ev_id\=851805 dmac=00:1d:a2:af:32:a1 cs1Label=sname cs1=Trojan.Generic.DNS 
+<164>fenotify-851844.alert: CEF:0|FireEye|CMS|7.2.1.244420|DM|domain-match|1|rt=Feb 09 2015 06:19:41 UTC dvc=10.201.78.59 cn3Label=cncPort cn3=53 cn2Label=sid cn2=80494706 shost=dev001srv02.example.com proto=udp cs5Label=cncHost cs5=mfdclk001.org dvchost=DEVFEYE1 spt=50767 dvc=10.100.25.16 smac=00:00:0c:07:ac:00 cn1Label=vlan cn1=0 externalId=851844 cs4Label=link cs4=https://DEVCMS01.example.com/event_stream/events_for_bot?ev_id\=851844 dmac=00:1d:a2:af:32:a1 cs1Label=sname cs1=Trojan.Generic.DNS 
+<164>fenotify-851782.alert: CEF:0|FireEye|CMS|7.2.1.244420|DM|domain-match|1|rt=Feb 09 2015 01:18:22 UTC dvc=10.201.78.59 cn3Label=cncPort cn3=53 cn2Label=sid cn2=80494706 shost=dev001srv02.example.com proto=udp cs5Label=cncHost cs5=mfdclk001.org dvchost=DEVFEYE1 spt=50940 dvc=10.100.25.16 smac=00:00:0c:07:ac:00 cn1Label=vlan cn1=0 externalId=851782 cs4Label=link cs4=https://DEVCMS01.example.com/event_stream/events_for_bot?ev_id\=851782 dmac=00:1d:a2:af:32:a1 cs1Label=sname cs1=Trojan.Generic.DNS 
+<164>fenotify-851940.alert: CEF:0|FireEye|CMS|7.2.1.244420|DM|domain-match|1|rt=Feb 09 2015 10:57:19 UTC dvc=10.201.78.85 cn3Label=cncPort cn3=53 cn2Label=sid cn2=80494706 shost=dev001srv02.example.com proto=udp cs5Label=cncHost cs5=mfdclk001.org dvchost=DEVFEYE1 spt=50646 dvc=10.100.25.16 smac=00:00:0c:07:ac:c8 cn1Label=vlan cn1=0 externalId=851940 cs4Label=link cs4=https://DEVCMS01.example.com/event_stream/events_for_bot?ev_id\=851940 dmac=88:43:e1:95:13:29 cs1Label=sname cs1=Trojan.Generic.DNS 
+<164>fenotify-851881.alert: CEF:0|FireEye|CMS|7.2.1.244420|DM|domain-match|1|rt=Feb 09 2015 08:13:15 UTC dvc=10.201.78.84 cn3Label=cncPort cn3=53 cn2Label=sid cn2=80494706 shost=dev001srv02.example.com proto=udp cs5Label=cncHost cs5=mfdclk001.org dvchost=DEVFEYE1 spt=61237 dvc=10.100.25.16 smac=00:00:0c:07:ac:c8 cn1Label=vlan cn1=0 externalId=851881 cs4Label=link cs4=https://DEVCMS01.example.com/event_stream/events_for_bot?ev_id\=851881 dmac=88:43:e1:95:13:29 cs1Label=sname cs1=Trojan.Generic.DNS 
+<164>fenotify-851839.alert: CEF:0|FireEye|CMS|7.2.1.244420|DM|domain-match|1|rt=Feb 09 2015 05:33:19 UTC dvc=10.201.78.10 cn3Label=cncPort cn3=53 cn2Label=sid cn2=80494706 proto=udp cs5Label=cncHost cs5=mfdclk001.org dvchost=DEVFEYE1 spt=49186 dvc=10.100.25.5 smac=00:00:0c:07:ac:63 cn1Label=vlan cn1=143 externalId=851839 cs4Label=link cs4=https://DEVCMS01.example.com/event_stream/events_for_bot?ev_id\=851839 dmac=00:1b:17:00:09:01 cs1Label=sname cs1=Trojan.Generic.DNS 
+<164>fenotify-851983.alert: CEF:0|FireEye|CMS|7.2.1.244420|DM|domain-match|1|rt=Feb 09 2015 12:28:26 UTC dvc=10.201.78.57 cn3Label=cncPort cn3=53 cn2Label=sid cn2=80494706 shost=dev001srv02.example.com proto=udp cs5Label=cncHost cs5=mfdclk001.org dvchost=DEVFEYE1 spt=54527 dvc=10.100.25.16 smac=00:00:0c:07:ac:00 cn1Label=vlan cn1=0 externalId=851983 cs4Label=link cs4=https://DEVCMS01.example.com/event_stream/events_for_bot?ev_id\=851983 dmac=00:1d:a2:af:32:a1 cs1Label=sname cs1=Trojan.Generic.DNS 
+<164>fenotify-851987.alert: CEF:0|FireEye|CMS|7.2.1.244420|DM|domain-match|1|rt=Feb 09 2015 12:33:41 UTC dvc=10.201.78.113 cn3Label=cncPort cn3=53 cn2Label=sid cn2=80494706 shost=dev001srv02.example.com proto=udp cs5Label=cncHost cs5=mfdclk001.org dvchost=DEVFEYE1 spt=51218 dvc=10.100.25.16 smac=00:00:0c:07:ac:00 cn1Label=vlan cn1=0 externalId=851987 cs4Label=link cs4=https://DEVCMS01.example.com/event_stream/events_for_bot?ev_id\=851987 dmac=00:1d:a2:af:32:a1 cs1Label=sname cs1=Trojan.Generic.DNS 
+<164>fenotify-852010.alert: CEF:0|FireEye|CMS|7.2.1.244420|DM|domain-match|1|rt=Feb 09 2015 13:15:08 UTC dvc=10.201.78.12 cn3Label=cncPort cn3=53 cn2Label=sid cn2=80494706 shost=dev001srv02.example.com proto=udp cs5Label=cncHost cs5=mfdclk001.org dvchost=DEVFEYE1 spt=55203 dvc=10.100.25.16 smac=00:00:0c:07:ac:c8 cn1Label=vlan cn1=0 externalId=852010 cs4Label=link cs4=https://DEVCMS01.example.com/event_stream/events_for_bot?ev_id\=852010 dmac=88:43:e1:95:13:29 cs1Label=sname cs1=Trojan.Generic.DNS 
+<164>fenotify-852053.alert: CEF:0|FireEye|CMS|7.2.1.244420|DM|domain-match|1|rt=Feb 09 2015 14:16:45 UTC dvc=10.201.78.84 cn3Label=cncPort cn3=53 cn2Label=sid cn2=80494706 shost=dev001srv02.example.com proto=udp cs5Label=cncHost cs5=mfdclk001.org dvchost=DEVFEYE1 spt=62235 dvc=10.100.25.16 smac=00:00:0c:07:ac:c8 cn1Label=vlan cn1=0 externalId=852053 cs4Label=link cs4=https://DEVCMS01.example.com/event_stream/events_for_bot?ev_id\=852053 dmac=88:43:e1:95:13:29 cs1Label=sname cs1=Trojan.Generic.DNS 
+<164>fenotify-852455.alert: CEF:0|FireEye|CMS|7.2.1.244420|DM|domain-match|1|rt=Feb 09 2015 17:28:38 UTC dvc=10.201.78.34 cn3Label=cncPort cn3=53 cn2Label=sid cn2=80494706 shost=dev001srv02.example.com proto=udp cs5Label=cncHost cs5=mfdclk001.org dvchost=DEVFEYE1 spt=65175 dvc=10.100.25.16 smac=00:00:0c:07:ac:c8 cn1Label=vlan cn1=0 externalId=852455 cs4Label=link cs4=https://DEVCMS01.example.com/event_stream/events_for_bot?ev_id\=852455 dmac=88:43:e1:95:13:29 cs1Label=sname cs1=Trojan.Generic.DNS 
+<164>fenotify-851887.alert: CEF:0|FireEye|CMS|7.2.1.244420|DM|domain-match|1|rt=Feb 09 2015 08:24:54 UTC dvc=10.201.78.44 cn3Label=cncPort cn3=53 cn2Label=sid cn2=80494706 shost=dev001srv02.example.com proto=udp cs5Label=cncHost cs5=mfdclk001.org dvchost=DEVFEYE1 spt=56334 dvc=10.100.25.16 smac=00:00:0c:07:ac:c8 cn1Label=vlan cn1=0 externalId=851887 cs4Label=link cs4=https://DEVCMS01.example.com/event_stream/events_for_bot?ev_id\=851887 dmac=88:43:e1:95:13:29 cs1Label=sname cs1=Trojan.Generic.DNS 
+<164>fenotify-851822.alert: CEF:0|FireEye|CMS|7.2.1.244420|DM|domain-match|1|rt=Feb 09 2015 04:41:49 UTC dvc=10.201.78.54 cn3Label=cncPort cn3=53 cn2Label=sid cn2=80494706 shost=dev001srv02.example.com proto=udp cs5Label=cncHost cs5=mfdclk001.org dvchost=DEVFEYE1 spt=49732 dvc=10.100.25.16 smac=00:00:0c:07:ac:c8 cn1Label=vlan cn1=0 externalId=851822 cs4Label=link cs4=https://DEVCMS01.example.com/event_stream/events_for_bot?ev_id\=851822 dmac=88:43:e1:95:13:29 cs1Label=sname cs1=Trojan.Generic.DNS 
+<164>fenotify-851832.alert: CEF:0|FireEye|CMS|7.2.1.244420|DM|domain-match|1|rt=Feb 09 2015 05:19:15 UTC dvc=10.201.78.160 cn3Label=cncPort cn3=53 cn2Label=sid cn2=80494706 shost=dev001srv02.example.com proto=udp cs5Label=cncHost cs5=mfdclk001.org dvchost=DEVFEYE1 spt=62962 dvc=10.100.25.16 smac=00:00:0c:07:ac:00 cn1Label=vlan cn1=0 externalId=851832 cs4Label=link cs4=https://DEVCMS01.example.com/event_stream/events_for_bot?ev_id\=851832 dmac=00:1d:a2:af:32:a1 cs1Label=sname cs1=Trojan.Generic.DNS 
+<164>fenotify-851780.alert: CEF:0|FireEye|CMS|7.2.1.244420|DM|domain-match|1|rt=Feb 09 2015 00:56:46 UTC dvc=10.201.78.12 cn3Label=cncPort cn3=53 cn2Label=sid cn2=80494706 shost=dev001srv02.example.com proto=udp cs5Label=cncHost cs5=mfdclk001.org dvchost=DEVFEYE1 spt=54301 dvc=10.100.25.16 smac=00:00:0c:07:ac:c8 cn1Label=vlan cn1=0 externalId=851780 cs4Label=link cs4=https://DEVCMS01.example.com/event_stream/events_for_bot?ev_id\=851780 dmac=88:43:e1:95:13:29 cs1Label=sname cs1=Trojan.Generic.DNS 
+<164>fenotify-851792.alert: CEF:0|FireEye|CMS|7.2.1.244420|DM|domain-match|1|rt=Feb 09 2015 02:15:06 UTC dvc=10.201.78.194 cn3Label=cncPort cn3=53 cn2Label=sid cn2=80494706 shost=dev001srv02.example.com proto=udp cs5Label=cncHost cs5=mfdclk001.org dvchost=DEVFEYE1 spt=64831 dvc=10.100.25.16 smac=00:00:0c:07:ac:c8 cn1Label=vlan cn1=0 externalId=851792 cs4Label=link cs4=https://DEVCMS01.example.com/event_stream/events_for_bot?ev_id\=851792 dmac=88:43:e1:95:13:29 cs1Label=sname cs1=Trojan.Generic.DNS 
+<164>fenotify-851806.alert: CEF:0|FireEye|CMS|7.2.1.244420|DM|domain-match|1|rt=Feb 09 2015 03:24:05 UTC dvc=10.201.78.57 cn3Label=cncPort cn3=53 cn2Label=sid cn2=80494706 shost=dev001srv02.example.com proto=udp cs5Label=cncHost cs5=mfdclk001.org dvchost=DEVFEYE1 spt=53417 dvc=10.100.25.16 smac=00:00:0c:07:ac:00 cn1Label=vlan cn1=0 externalId=851806 cs4Label=link cs4=https://DEVCMS01.example.com/event_stream/events_for_bot?ev_id\=851806 dmac=00:1d:a2:af:32:a1 cs1Label=sname cs1=Trojan.Generic.DNS 
+<164>fenotify-851840.alert: CEF:0|FireEye|CMS|7.2.1.244420|DM|domain-match|1|rt=Feb 09 2015 06:00:58 UTC dvc=10.201.78.40 cn3Label=cncPort cn3=53 cn2Label=sid cn2=80494706 shost=dev001srv02.example.com proto=udp cs5Label=cncHost cs5=mfdclk001.org dvchost=DEVFEYE1 spt=50709 dvc=10.100.25.16 smac=00:00:0c:07:ac:c8 cn1Label=vlan cn1=0 externalId=851840 cs4Label=link cs4=https://DEVCMS01.example.com/event_stream/events_for_bot?ev_id\=851840 dmac=88:43:e1:95:13:29 cs1Label=sname cs1=Trojan.Generic.DNS 
+<164>fenotify-851929.alert: CEF:0|FireEye|CMS|7.2.1.244420|DM|domain-match|1|rt=Feb 09 2015 10:37:14 UTC dvc=10.201.78.87 cn3Label=cncPort cn3=53 cn2Label=sid cn2=80494706 shost=dev001srv02.example.com proto=udp cs5Label=cncHost cs5=mfdclk001.org dvchost=DEVFEYE1 spt=62909 dvc=10.100.25.16 smac=00:00:0c:07:ac:c8 cn1Label=vlan cn1=0 externalId=851929 cs4Label=link cs4=https://DEVCMS01.example.com/event_stream/events_for_bot?ev_id\=851929 dmac=88:43:e1:95:13:29 cs1Label=sname cs1=Trojan.Generic.DNS 
+<164>fenotify-851918.alert: CEF:0|FireEye|CMS|7.2.1.244420|DM|domain-match|1|rt=Feb 09 2015 10:17:41 UTC dvc=10.201.78.34 cn3Label=cncPort cn3=53 cn2Label=sid cn2=80494706 proto=udp cs5Label=cncHost cs5=mfdclk001.org dvchost=DEVFEYE1 spt=63483 dvc=10.100.25.5 smac=00:00:0c:07:ac:63 cn1Label=vlan cn1=143 externalId=851918 cs4Label=link cs4=https://DEVCMS01.example.com/event_stream/events_for_bot?ev_id\=851918 dmac=00:1b:17:00:09:01 cs1Label=sname cs1=Trojan.Generic.DNS 
+<164>fenotify-851842.alert: CEF:0|FireEye|CMS|7.2.1.244420|DM|domain-match|1|rt=Feb 09 2015 06:03:05 UTC dvc=10.201.78.68 cn3Label=cncPort cn3=53 cn2Label=sid cn2=80494706 proto=udp cs5Label=cncHost cs5=mfdclk001.org dvchost=DEVFEYE1 spt=59908 dvc=10.100.25.16 smac=00:00:0c:07:ac:5a cn1Label=vlan cn1=0 externalId=851842 cs4Label=link cs4=https://DEVCMS01.example.com/event_stream/events_for_bot?ev_id\=851842 dmac=00:09:0f:33:4f:48 cs1Label=sname cs1=Trojan.Generic.DNS 
+<164>fenotify-851948.alert: CEF:0|FireEye|CMS|7.2.1.244420|DM|domain-match|1|rt=Feb 09 2015 11:13:18 UTC dvc=10.201.78.86 cn3Label=cncPort cn3=53 cn2Label=sid cn2=80494706 shost=dev001srv02.example.com proto=udp cs5Label=cncHost cs5=mfdclk001.org dvchost=DEVFEYE1 spt=51327 dvc=10.100.25.16 smac=00:00:0c:07:ac:c8 cn1Label=vlan cn1=0 externalId=851948 cs4Label=link cs4=https://DEVCMS01.example.com/event_stream/events_for_bot?ev_id\=851948 dmac=88:43:e1:95:13:29 cs1Label=sname cs1=Trojan.Generic.DNS 
+<164>fenotify-852008.alert: CEF:0|FireEye|CMS|7.2.1.244420|DM|domain-match|1|rt=Feb 09 2015 13:13:25 UTC dvc=10.201.78.20 cn3Label=cncPort cn3=53 cn2Label=sid cn2=80494706 shost=dev001srv02.example.com proto=udp cs5Label=cncHost cs5=mfdclk001.org dvchost=DEVFEYE1 spt=63619 dvc=10.100.25.16 smac=00:00:0c:07:ac:c8 cn1Label=vlan cn1=0 externalId=852008 cs4Label=link cs4=https://DEVCMS01.example.com/event_stream/events_for_bot?ev_id\=852008 dmac=88:43:e1:95:13:29 cs1Label=sname cs1=Trojan.Generic.DNS 
+<164>fenotify-852072.alert: CEF:0|FireEye|CMS|7.2.1.244420|DM|domain-match|1|rt=Feb 09 2015 14:30:09 UTC dvc=10.201.78.37 cn3Label=cncPort cn3=53 cn2Label=sid cn2=80494706 shost=dev001srv02.example.com proto=udp cs5Label=cncHost cs5=mfdclk001.org dvchost=DEVFEYE1 spt=53467 dvc=10.100.25.16 smac=00:00:0c:07:ac:00 cn1Label=vlan cn1=0 externalId=852072 cs4Label=link cs4=https://DEVCMS01.example.com/event_stream/events_for_bot?ev_id\=852072 dmac=00:1d:a2:af:32:a1 cs1Label=sname cs1=Trojan.Generic.DNS 
+<164>fenotify-852077.alert: CEF:0|FireEye|CMS|7.2.1.244420|DM|domain-match|1|rt=Feb 09 2015 14:31:58 UTC dvc=10.201.78.11 cn3Label=cncPort cn3=53 cn2Label=sid cn2=80494706 proto=udp cs5Label=cncHost cs5=mfdclk001.org dvchost=DEVFEYE1 spt=58546 dvc=10.100.25.5 smac=00:00:0c:07:ac:63 cn1Label=vlan cn1=143 externalId=852077 cs4Label=link cs4=https://DEVCMS01.example.com/event_stream/events_for_bot?ev_id\=852077 dmac=00:1b:17:00:09:01 cs1Label=sname cs1=Trojan.Generic.DNS 
+<164>fenotify-852110.alert: CEF:0|FireEye|CMS|7.2.1.244420|DM|domain-match|1|rt=Feb 09 2015 14:56:32 UTC dvc=10.201.78.160 cn3Label=cncPort cn3=53 cn2Label=sid cn2=80494706 shost=dev001srv02.example.com proto=udp cs5Label=cncHost cs5=mfdclk001.org dvchost=DEVFEYE1 spt=61983 dvc=10.100.25.16 smac=00:00:0c:07:ac:00 cn1Label=vlan cn1=0 externalId=852110 cs4Label=link cs4=https://DEVCMS01.example.com/event_stream/events_for_bot?ev_id\=852110 dmac=00:1d:a2:af:32:a1 cs1Label=sname cs1=Trojan.Generic.DNS 
+<164>fenotify-852378.alert: CEF:0|FireEye|CMS|7.2.1.244420|DM|domain-match|1|rt=Feb 09 2015 17:03:31 UTC dvc=10.201.78.85 cn3Label=cncPort cn3=53 cn2Label=sid cn2=80494706 shost=dev001srv02.example.com proto=udp cs5Label=cncHost cs5=mfdclk001.org dvchost=DEVFEYE1 spt=49942 dvc=10.100.25.16 smac=00:00:0c:07:ac:c8 cn1Label=vlan cn1=0 externalId=852378 cs4Label=link cs4=https://DEVCMS01.example.com/event_stream/events_for_bot?ev_id\=852378 dmac=88:43:e1:95:13:29 cs1Label=sname cs1=Trojan.Generic.DNS 
+<164>fenotify-851787.alert: CEF:0|FireEye|CMS|7.2.1.244420|DM|domain-match|1|rt=Feb 09 2015 01:57:21 UTC dvc=10.201.78.44 cn3Label=cncPort cn3=53 cn2Label=sid cn2=80494706 shost=dev001srv02.example.com proto=udp cs5Label=cncHost cs5=mfdclk001.org dvchost=DEVFEYE1 spt=55199 dvc=10.100.25.16 smac=00:00:0c:07:ac:c8 cn1Label=vlan cn1=0 externalId=851787 cs4Label=link cs4=https://DEVCMS01.example.com/event_stream/events_for_bot?ev_id\=851787 dmac=88:43:e1:95:13:29 cs1Label=sname cs1=Trojan.Generic.DNS 
+<164>fenotify-851800.alert: CEF:0|FireEye|CMS|7.2.1.244420|DM|domain-match|1|rt=Feb 09 2015 02:54:32 UTC dvc=10.201.78.34 cn3Label=cncPort cn3=53 cn2Label=sid cn2=80494706 shost=dev001srv02.example.com proto=udp cs5Label=cncHost cs5=mfdclk001.org dvchost=DEVFEYE1 spt=50605 dvc=10.100.25.16 smac=00:00:0c:07:ac:c8 cn1Label=vlan cn1=0 externalId=851800 cs4Label=link cs4=https://DEVCMS01.example.com/event_stream/events_for_bot?ev_id\=851800 dmac=88:43:e1:95:13:29 cs1Label=sname cs1=Trojan.Generic.DNS 
+<164>fenotify-851941.alert: CEF:0|FireEye|CMS|7.2.1.244420|DM|domain-match|1|rt=Feb 09 2015 10:58:30 UTC dvc=10.201.78.54 cn3Label=cncPort cn3=53 cn2Label=sid cn2=80494706 shost=dev001srv02.example.com proto=udp cs5Label=cncHost cs5=mfdclk001.org dvchost=DEVFEYE1 spt=51721 dvc=10.100.25.16 smac=00:00:0c:07:ac:c8 cn1Label=vlan cn1=0 externalId=851941 cs4Label=link cs4=https://DEVCMS01.example.com/event_stream/events_for_bot?ev_id\=851941 dmac=88:43:e1:95:13:29 cs1Label=sname cs1=Trojan.Generic.DNS 
+<164>fenotify-851850.alert: CEF:0|FireEye|CMS|7.2.1.244420|DM|domain-match|1|rt=Feb 09 2015 06:29:59 UTC dvc=10.201.78.113 cn3Label=cncPort cn3=53 cn2Label=sid cn2=80494706 shost=dev001srv02.example.com proto=udp cs5Label=cncHost cs5=mfdclk001.org dvchost=DEVFEYE1 spt=50606 dvc=10.100.25.16 smac=00:00:0c:07:ac:00 cn1Label=vlan cn1=0 externalId=851850 cs4Label=link cs4=https://DEVCMS01.example.com/event_stream/events_for_bot?ev_id\=851850 dmac=00:1d:a2:af:32:a1 cs1Label=sname cs1=Trojan.Generic.DNS 
+<164>fenotify-851885.alert: CEF:0|FireEye|CMS|7.2.1.244420|DM|domain-match|1|rt=Feb 09 2015 08:22:40 UTC dvc=10.201.78.37 cn3Label=cncPort cn3=53 cn2Label=sid cn2=80494706 shost=dev001srv02.example.com proto=udp cs5Label=cncHost cs5=mfdclk001.org dvchost=DEVFEYE1 spt=53481 dvc=10.100.25.16 smac=00:00:0c:07:ac:00 cn1Label=vlan cn1=0 externalId=851885 cs4Label=link cs4=https://DEVCMS01.example.com/event_stream/events_for_bot?ev_id\=851885 dmac=00:1d:a2:af:32:a1 cs1Label=sname cs1=Trojan.Generic.DNS 
+<164>fenotify-851801.alert: CEF:0|FireEye|CMS|7.2.1.244420|DM|domain-match|1|rt=Feb 09 2015 02:55:09 UTC dvc=10.201.78.6 cn3Label=cncPort cn3=53 cn2Label=sid cn2=80494706 shost=dev001srv02.example.com proto=udp cs5Label=cncHost cs5=mfdclk001.org dvchost=DEVFEYE1 spt=59875 dvc=10.100.25.16 smac=00:00:0c:07:ac:c8 cn1Label=vlan cn1=0 externalId=851801 cs4Label=link cs4=https://DEVCMS01.example.com/event_stream/events_for_bot?ev_id\=851801 dmac=88:43:e1:95:13:29 cs1Label=sname cs1=Trojan.Generic.DNS 
+<164>fenotify-851884.alert: CEF:0|FireEye|CMS|7.2.1.244420|DM|domain-match|1|rt=Feb 09 2015 08:20:10 UTC dvc=10.201.78.194 cn3Label=cncPort cn3=53 cn2Label=sid cn2=80494706 shost=dev001srv02.example.com proto=udp cs5Label=cncHost cs5=mfdclk001.org dvchost=DEVFEYE1 spt=50039 dvc=10.100.25.16 smac=00:00:0c:07:ac:c8 cn1Label=vlan cn1=0 externalId=851884 cs4Label=link cs4=https://DEVCMS01.example.com/event_stream/events_for_bot?ev_id\=851884 dmac=88:43:e1:95:13:29 cs1Label=sname cs1=Trojan.Generic.DNS 
+<164>fenotify-851815.alert: CEF:0|FireEye|CMS|7.2.1.244420|DM|domain-match|1|rt=Feb 09 2015 04:06:05 UTC dvc=10.201.78.11 cn3Label=cncPort cn3=53 cn2Label=sid cn2=80494706 shost=dev001srv02.example.com proto=udp cs5Label=cncHost cs5=mfdclk001.org dvchost=DEVFEYE1 spt=53889 dvc=10.100.25.16 smac=00:00:0c:07:ac:c8 cn1Label=vlan cn1=0 externalId=851815 cs4Label=link cs4=https://DEVCMS01.example.com/event_stream/events_for_bot?ev_id\=851815 dmac=88:43:e1:95:13:29 cs1Label=sname cs1=Trojan.Generic.DNS 
+<164>fenotify-851825.alert: CEF:0|FireEye|CMS|7.2.1.244420|DM|domain-match|1|rt=Feb 09 2015 04:49:07 UTC dvc=10.201.78.85 cn3Label=cncPort cn3=53 cn2Label=sid cn2=80494706 shost=dev001srv02.example.com proto=udp cs5Label=cncHost cs5=mfdclk001.org dvchost=DEVFEYE1 spt=51906 dvc=10.100.25.16 smac=00:00:0c:07:ac:c8 cn1Label=vlan cn1=0 externalId=851825 cs4Label=link cs4=https://DEVCMS01.example.com/event_stream/events_for_bot?ev_id\=851825 dmac=88:43:e1:95:13:29 cs1Label=sname cs1=Trojan.Generic.DNS 
+<164>fenotify-851966.alert: CEF:0|FireEye|CMS|7.2.1.244420|DM|domain-match|1|rt=Feb 09 2015 11:50:43 UTC dvc=10.201.78.10 cn3Label=cncPort cn3=53 cn2Label=sid cn2=80494706 proto=udp cs5Label=cncHost cs5=mfdclk001.org dvchost=DEVFEYE1 spt=50758 dvc=10.100.25.5 smac=00:00:0c:07:ac:63 cn1Label=vlan cn1=143 externalId=851966 cs4Label=link cs4=https://DEVCMS01.example.com/event_stream/events_for_bot?ev_id\=851966 dmac=00:1b:17:00:09:01 cs1Label=sname cs1=Trojan.Generic.DNS 
+<164>fenotify-852112.alert: CEF:0|FireEye|CMS|7.2.1.244420|DM|domain-match|1|rt=Feb 09 2015 14:58:20 UTC dvc=10.201.78.6 cn3Label=cncPort cn3=53 cn2Label=sid cn2=80494706 shost=dev001srv02.example.com proto=udp cs5Label=cncHost cs5=mfdclk001.org dvchost=DEVFEYE1 spt=60631 dvc=10.100.25.16 smac=00:00:0c:07:ac:c8 cn1Label=vlan cn1=0 externalId=852112 cs4Label=link cs4=https://DEVCMS01.example.com/event_stream/events_for_bot?ev_id\=852112 dmac=88:43:e1:95:13:29 cs1Label=sname cs1=Trojan.Generic.DNS 
+<164>fenotify-852126.alert: CEF:0|FireEye|CMS|7.2.1.244420|DM|domain-match|1|rt=Feb 09 2015 15:03:43 UTC dvc=10.201.78.60 cn3Label=cncPort cn3=53 cn2Label=sid cn2=80494706 shost=dev001srv02.example.com proto=udp cs5Label=cncHost cs5=mfdclk001.org dvchost=DEVFEYE1 spt=65017 dvc=10.100.25.16 smac=00:00:0c:07:ac:00 cn1Label=vlan cn1=0 externalId=852126 cs4Label=link cs4=https://DEVCMS01.example.com/event_stream/events_for_bot?ev_id\=852126 dmac=00:1d:a2:af:32:a1 cs1Label=sname cs1=Trojan.Generic.DNS 
+<164>fenotify-852407.alert: CEF:0|FireEye|CMS|7.2.1.244420|DM|domain-match|1|rt=Feb 09 2015 17:15:10 UTC dvc=10.201.78.54 cn3Label=cncPort cn3=53 cn2Label=sid cn2=80494706 shost=dev001srv02.example.com proto=udp cs5Label=cncHost cs5=mfdclk001.org dvchost=DEVFEYE1 spt=49620 dvc=10.100.25.16 smac=00:00:0c:07:ac:c8 cn1Label=vlan cn1=0 externalId=852407 cs4Label=link cs4=https://DEVCMS01.example.com/event_stream/events_for_bot?ev_id\=852407 dmac=88:43:e1:95:13:29 cs1Label=sname cs1=Trojan.Generic.DNS 
+<164>fenotify-852417.alert: CEF:0|FireEye|CMS|7.2.1.244420|DM|domain-match|1|rt=Feb 09 2015 17:17:11 UTC dvc=10.201.78.86 cn3Label=cncPort cn3=53 cn2Label=sid cn2=80494706 shost=dev001srv02.example.com proto=udp cs5Label=cncHost cs5=mfdclk001.org dvchost=DEVFEYE1 spt=51333 dvc=10.100.25.16 smac=00:00:0c:07:ac:c8 cn1Label=vlan cn1=0 externalId=852417 cs4Label=link cs4=https://DEVCMS01.example.com/event_stream/events_for_bot?ev_id\=852417 dmac=88:43:e1:95:13:29 cs1Label=sname cs1=Trojan.Generic.DNS 
+<164>fenotify-852431.alert: CEF:0|FireEye|CMS|7.2.1.244420|DM|domain-match|1|rt=Feb 09 2015 17:20:08 UTC dvc=10.201.78.11 cn3Label=cncPort cn3=53 cn2Label=sid cn2=80494706 shost=dev001srv02.example.com proto=udp cs5Label=cncHost cs5=mfdclk001.org dvchost=DEVFEYE1 spt=53525 dvc=10.100.25.16 smac=00:00:0c:07:ac:c8 cn1Label=vlan cn1=0 externalId=852431 cs4Label=link cs4=https://DEVCMS01.example.com/event_stream/events_for_bot?ev_id\=852431 dmac=88:43:e1:95:13:29 cs1Label=sname cs1=Trojan.Generic.DNS 
+<164>fenotify-852438.alert: CEF:0|FireEye|CMS|7.2.1.244420|DM|domain-match|1|rt=Feb 09 2015 17:21:21 UTC dvc=10.201.78.84 cn3Label=cncPort cn3=53 cn2Label=sid cn2=80494706 shost=dev001srv02.example.com proto=udp cs5Label=cncHost cs5=mfdclk001.org dvchost=DEVFEYE1 spt=62464 dvc=10.100.25.16 smac=00:00:0c:07:ac:c8 cn1Label=vlan cn1=0 externalId=852438 cs4Label=link cs4=https://DEVCMS01.example.com/event_stream/events_for_bot?ev_id\=852438 dmac=88:43:e1:95:13:29 cs1Label=sname cs1=Trojan.Generic.DNS 
+<164>fenotify-3483822.1.alert: CEF:0|FireEye|CMS|7.2.1.244420|MO|malware-object|4|rt=Feb 09 2015 07:24:06 UTC dvc=10.201.78.216 cn3Label=cncPort cn3=80 dst=191.235.179.140 fileHash=6126d97e5bd4e6d93e3e3579cc5b3ce0 filePath=/analysis/191.235.179.140_80-10.220.55.216_56118--833719413_9204551_T.pcoff cs5Label=cncHost cs5=api.shamenchik.info cs3Label=osinfo cs3=Microsoft WindowsXP 32-bit 5.1 sp3 14.0528 proto=tcp dvchost=DEVFEYE1 dvc=10.100.25.16 cn1Label=vlan cn1=0 externalId=3483822 cs4Label=link cs4=https://DEVCMS01.example.com/event_stream/events_for_bot?ma_id\=3483822 cs6Label=channel cs6=POST /api/sdm HTTP/1.1::~~Content-Type: application/x-json::~~Accept: */*::~~User-Agent: Mozilla/4.0 (compatible; MSIE 7.0; Windows NT 5.1; Trident/4.0; .NET CLR 2.0.50727; .NET CLR 3.0.04506.648; .NET CLR 3.5.21022; .NET4.0C; .NET4.0E)::~~Host: api.shamenchik.info::~~Content-Length: 800::~~Connection: Keep-Alive::~~Cache-Control: no-cache::~~::~~g+3CouWsTcAym6cirpXcrPeCqh2q2xYh//aNKX15/lgvTM
+<164>fenotify-851890.alert: CEF:0|FireEye|CMS|7.2.1.244420|DM|domain-match|1|rt=Feb 09 2015 08:36:36 UTC dvc=10.201.78.160 cn3Label=cncPort cn3=53 cn2Label=sid cn2=80494706 shost=dev001srv02.example.com proto=udp cs5Label=cncHost cs5=mfdclk001.org dvchost=DEVFEYE1 spt=63018 dvc=10.100.25.16 smac=00:00:0c:07:ac:00 cn1Label=vlan cn1=0 externalId=851890 cs4Label=link cs4=https://DEVCMS01.example.com/event_stream/events_for_bot?ev_id\=851890 dmac=00:1d:a2:af:32:a1 cs1Label=sname cs1=Trojan.Generic.DNS 
+<164>fenotify-851861.alert: CEF:0|FireEye|CMS|7.2.1.244420|DM|domain-match|1|rt=Feb 09 2015 07:11:45 UTC dvc=10.201.78.20 cn3Label=cncPort cn3=53 cn2Label=sid cn2=80494706 shost=dev001srv02.example.com proto=udp cs5Label=cncHost cs5=mfdclk001.org dvchost=DEVFEYE1 spt=62660 dvc=10.100.25.16 smac=00:00:0c:07:ac:c8 cn1Label=vlan cn1=0 externalId=851861 cs4Label=link cs4=https://DEVCMS01.example.com/event_stream/events_for_bot?ev_id\=851861 dmac=88:43:e1:95:13:29 cs1Label=sname cs1=Trojan.Generic.DNS 
+<164>fenotify-851781.alert: CEF:0|FireEye|CMS|7.2.1.244420|DM|domain-match|1|rt=Feb 09 2015 01:10:09 UTC dvc=10.201.78.20 cn3Label=cncPort cn3=53 cn2Label=sid cn2=80494706 shost=dev001srv02.example.com proto=udp cs5Label=cncHost cs5=mfdclk001.org dvchost=DEVFEYE1 spt=63319 dvc=10.100.25.16 smac=00:00:0c:07:ac:c8 cn1Label=vlan cn1=0 externalId=851781 cs4Label=link cs4=https://DEVCMS01.example.com/event_stream/events_for_bot?ev_id\=851781 dmac=88:43:e1:95:13:29 cs1Label=sname cs1=Trojan.Generic.DNS 
+<164>fenotify-851837.alert: CEF:0|FireEye|CMS|7.2.1.244420|DM|domain-match|1|rt=Feb 09 2015 05:30:01 UTC dvc=10.201.78.60 cn3Label=cncPort cn3=53 cn2Label=sid cn2=80494706 shost=dev001srv02.example.com proto=udp cs5Label=cncHost cs5=mfdclk001.org dvchost=DEVFEYE1 spt=49533 dvc=10.100.25.16 smac=00:00:0c:07:ac:00 cn1Label=vlan cn1=0 externalId=851837 cs4Label=link cs4=https://DEVCMS01.example.com/event_stream/events_for_bot?ev_id\=851837 dmac=00:1d:a2:af:32:a1 cs1Label=sname cs1=Trojan.Generic.DNS 
+<164>fenotify-851846.alert: CEF:0|FireEye|CMS|7.2.1.244420|DM|domain-match|1|rt=Feb 09 2015 06:26:50 UTC dvc=10.201.78.57 cn3Label=cncPort cn3=53 cn2Label=sid cn2=80494706 shost=dev001srv02.example.com proto=udp cs5Label=cncHost cs5=mfdclk001.org dvchost=DEVFEYE1 spt=53933 dvc=10.100.25.16 smac=00:00:0c:07:ac:00 cn1Label=vlan cn1=0 externalId=851846 cs4Label=link cs4=https://DEVCMS01.example.com/event_stream/events_for_bot?ev_id\=851846 dmac=00:1d:a2:af:32:a1 cs1Label=sname cs1=Trojan.Generic.DNS 
+<164>fenotify-851920.alert: CEF:0|FireEye|CMS|7.2.1.244420|DM|domain-match|1|rt=Feb 09 2015 10:26:37 UTC dvc=10.201.78.51 cn3Label=cncPort cn3=53 cn2Label=sid cn2=80494706 shost=dev001srv02.example.com proto=udp cs5Label=cncHost cs5=mfdclk001.org dvchost=DEVFEYE1 spt=60410 dvc=10.100.25.16 smac=00:00:0c:07:ac:63 cn1Label=vlan cn1=0 externalId=851920 cs4Label=link cs4=https://DEVCMS01.example.com/event_stream/events_for_bot?ev_id\=851920 dmac=5c:5e:ab:eb:ab:0d cs1Label=sname cs1=Trojan.Generic.DNS 
+<164>fenotify-851818.alert: CEF:0|FireEye|CMS|7.2.1.244420|DM|domain-match|1|rt=Feb 09 2015 04:25:02 UTC dvc=10.201.78.51 cn3Label=cncPort cn3=53 cn2Label=sid cn2=80494706 shost=dev001srv02.example.com proto=udp cs5Label=cncHost cs5=mfdclk001.org dvchost=DEVFEYE1 spt=60319 dvc=10.100.25.16 smac=00:00:0c:07:ac:63 cn1Label=vlan cn1=0 externalId=851818 cs4Label=link cs4=https://DEVCMS01.example.com/event_stream/events_for_bot?ev_id\=851818 dmac=5c:5e:ab:eb:ab:0d cs1Label=sname cs1=Trojan.Generic.DNS 
+<164>fenotify-851866.alert: CEF:0|FireEye|CMS|7.2.1.244420|DM|domain-match|1|rt=Feb 09 2015 07:13:28 UTC dvc=10.201.78.12 cn3Label=cncPort cn3=53 cn2Label=sid cn2=80494706 shost=dev001srv02.example.com proto=udp cs5Label=cncHost cs5=mfdclk001.org dvchost=DEVFEYE1 spt=54836 dvc=10.100.25.16 smac=00:00:0c:07:ac:c8 cn1Label=vlan cn1=0 externalId=851866 cs4Label=link cs4=https://DEVCMS01.example.com/event_stream/events_for_bot?ev_id\=851866 dmac=88:43:e1:95:13:29 cs1Label=sname cs1=Trojan.Generic.DNS 
+<164>fenotify-851773.alert: CEF:0|FireEye|CMS|7.2.1.244420|DM|domain-match|1|rt=Feb 09 2015 00:01:29 UTC dvc=10.201.78.68 cn3Label=cncPort cn3=53 cn2Label=sid cn2=80494706 proto=udp cs5Label=cncHost cs5=mfdclk001.org dvchost=DEVFEYE1 spt=60239 dvc=10.100.25.16 smac=00:00:0c:07:ac:5a cn1Label=vlan cn1=0 externalId=851773 cs4Label=link cs4=https://DEVCMS01.example.com/event_stream/events_for_bot?ev_id\=851773 dmac=00:09:0f:33:4f:48 cs1Label=sname cs1=Trojan.Generic.DNS 
+<164>fenotify-851935.alert: CEF:0|FireEye|CMS|7.2.1.244420|DM|domain-match|1|rt=Feb 09 2015 10:48:18 UTC dvc=10.201.78.11 cn3Label=cncPort cn3=53 cn2Label=sid cn2=80494706 shost=dev001srv02.example.com proto=udp cs5Label=cncHost cs5=mfdclk001.org dvchost=DEVFEYE1 spt=54362 dvc=10.100.25.16 smac=00:00:0c:07:ac:c8 cn1Label=vlan cn1=0 externalId=851935 cs4Label=link cs4=https://DEVCMS01.example.com/event_stream/events_for_bot?ev_id\=851935 dmac=88:43:e1:95:13:29 cs1Label=sname cs1=Trojan.Generic.DNS 
+<164>fenotify-851970.alert: CEF:0|FireEye|CMS|7.2.1.244420|DM|domain-match|1|rt=Feb 09 2015 12:04:50 UTC dvc=10.201.78.40 cn3Label=cncPort cn3=53 cn2Label=sid cn2=80494706 shost=dev001srv02.example.com proto=udp cs5Label=cncHost cs5=mfdclk001.org dvchost=DEVFEYE1 spt=50327 dvc=10.100.25.16 smac=00:00:0c:07:ac:c8 cn1Label=vlan cn1=0 externalId=851970 cs4Label=link cs4=https://DEVCMS01.example.com/event_stream/events_for_bot?ev_id\=851970 dmac=88:43:e1:95:13:29 cs1Label=sname cs1=Trojan.Generic.DNS 
+<164>fenotify-851975.alert: CEF:0|FireEye|CMS|7.2.1.244420|DM|domain-match|1|rt=Feb 09 2015 12:21:18 UTC dvc=10.201.78.59 cn3Label=cncPort cn3=53 cn2Label=sid cn2=80494706 shost=dev001srv02.example.com proto=udp cs5Label=cncHost cs5=mfdclk001.org dvchost=DEVFEYE1 spt=51420 dvc=10.100.25.16 smac=00:00:0c:07:ac:00 cn1Label=vlan cn1=0 externalId=851975 cs4Label=link cs4=https://DEVCMS01.example.com/event_stream/events_for_bot?ev_id\=851975 dmac=00:1d:a2:af:32:a1 cs1Label=sname cs1=Trojan.Generic.DNS 
+<164>fenotify-852454.alert: CEF:0|FireEye|CMS|7.2.1.244420|DM|domain-match|1|rt=Feb 09 2015 17:28:34 UTC dvc=10.201.78.44 cn3Label=cncPort cn3=53 cn2Label=sid cn2=80494706 shost=dev001srv02.example.com proto=udp cs5Label=cncHost cs5=mfdclk001.org dvchost=DEVFEYE1 spt=55348 dvc=10.100.25.16 smac=00:00:0c:07:ac:c8 cn1Label=vlan cn1=0 externalId=852454 cs4Label=link cs4=https://DEVCMS01.example.com/event_stream/events_for_bot?ev_id\=852454 dmac=88:43:e1:95:13:29 cs1Label=sname cs1=Trojan.Generic.DNS 
+<164>fenotify-3483798.2.alert: act;Trojan.Kuloz;Trojan.Kuluoz 
+<164>fenotify-834781.2.alert: Connection: Keep-Alive::~~::~~ dmac=00:1d:a2:af:32:a1 cs1Label=sname cs1=Malicious.URL 
+<164>fenotify-3483794.3.alert: 0d3cc7cc055f8d686a1b5d5c30db85c5423620e6bd231d592266782cf5e1647ae575e77b HTTP/1.1::~~Accept: */*::~~Proxy-Authorization: Basic ::~~User-Agent: Mozilla/5.0 (Windows NT 6.1; WOW64) AppleWebKit/537.36 (KHTML, like Gecko) Chrome/35.0.1916.153 Safari/537.36::~~Host: 5aqobwcp1xuqztwht.0eq0w6k.com::~~Connection: Keep-Alive::~~::~~ cs2Label=anomaly cs2=misc-anomaly cs1Label=sname cs1=FE_Evasion_Sandboxie;FE_Evasion_VMDetect 
+<164>fenotify-3483796.2.alert: jan.Kuloz;Trojan.Kuluoz 
+<164>fenotify-851894.alert: CEF:0|FireEye|CMS|7.2.1.244420|DM|domain-match|1|rt=Feb 09 2015 08:45:48 UTC dvc=10.201.78.60 cn3Label=cncPort cn3=53 cn2Label=sid cn2=80494706 shost=dev001srv02.example.com proto=udp cs5Label=cncHost cs5=mfdclk001.org dvchost=DEVFEYE1 spt=49433 dvc=10.100.25.16 smac=00:00:0c:07:ac:00 cn1Label=vlan cn1=0 externalId=851894 cs4Label=link cs4=https://DEVCMS01.example.com/event_stream/events_for_bot?ev_id\=851894 dmac=00:1d:a2:af:32:a1 cs1Label=sname cs1=Trojan.Generic.DNS 
+<164>fenotify-851899.alert: CEF:0|FireEye|CMS|7.2.1.244420|DM|domain-match|1|rt=Feb 09 2015 08:54:50 UTC dvc=10.201.78.34 cn3Label=cncPort cn3=53 cn2Label=sid cn2=80494706 shost=dev001srv02.example.com proto=udp cs5Label=cncHost cs5=mfdclk001.org dvchost=DEVFEYE1 spt=50711 dvc=10.100.25.16 smac=00:00:0c:07:ac:c8 cn1Label=vlan cn1=0 externalId=851899 cs4Label=link cs4=https://DEVCMS01.example.com/event_stream/events_for_bot?ev_id\=851899 dmac=88:43:e1:95:13:29 cs1Label=sname cs1=Trojan.Generic.DNS 
+<164>fenotify-851851.alert: CEF:0|FireEye|CMS|7.2.1.244420|DM|domain-match|1|rt=Feb 09 2015 06:31:05 UTC dvc=10.201.78.190 cn3Label=cncPort cn3=53 cn2Label=sid cn2=80494706 shost=dev001srv02.example.com proto=udp cs5Label=cncHost cs5=mfdclk001.org dvchost=DEVFEYE1 spt=61134 dvc=10.100.25.16 smac=00:00:0c:07:ac:00 cn1Label=vlan cn1=0 externalId=851851 cs4Label=link cs4=https://DEVCMS01.example.com/event_stream/events_for_bot?ev_id\=851851 dmac=00:1d:a2:af:32:a1 cs1Label=sname cs1=Trojan.Generic.DNS 
+<164>fenotify-851845.alert: CEF:0|FireEye|CMS|7.2.1.244420|DM|domain-match|1|rt=Feb 09 2015 06:20:46 UTC dvc=10.201.78.20 cn3Label=cncPort cn3=53 cn2Label=sid cn2=80494706 shost=dev001srv02.example.com proto=udp cs5Label=cncHost cs5=mfdclk001.org dvchost=DEVFEYE1 spt=55294 dvc=10.100.25.16 smac=00:00:0c:07:ac:00 cn1Label=vlan cn1=0 externalId=851845 cs4Label=link cs4=https://DEVCMS01.example.com/event_stream/events_for_bot?ev_id\=851845 dmac=00:1d:a2:af:32:a1 cs1Label=sname cs1=Trojan.Generic.DNS 
+<164>fenotify-851789.alert: CEF:0|FireEye|CMS|7.2.1.244420|DM|domain-match|1|rt=Feb 09 2015 02:03:48 UTC dvc=10.201.78.84 cn3Label=cncPort cn3=53 cn2Label=sid cn2=80494706 shost=dev001srv02.example.com proto=udp cs5Label=cncHost cs5=mfdclk001.org dvchost=DEVFEYE1 spt=62782 dvc=10.100.25.16 smac=00:00:0c:07:ac:c8 cn1Label=vlan cn1=0 externalId=851789 cs4Label=link cs4=https://DEVCMS01.example.com/event_stream/events_for_bot?ev_id\=851789 dmac=88:43:e1:95:13:29 cs1Label=sname cs1=Trojan.Generic.DNS 
+<164>fenotify-851820.alert: CEF:0|FireEye|CMS|7.2.1.244420|DM|domain-match|1|rt=Feb 09 2015 04:33:45 UTC dvc=10.201.78.87 cn3Label=cncPort cn3=53 cn2Label=sid cn2=80494706 shost=dev001srv02.example.com proto=udp cs5Label=cncHost cs5=mfdclk001.org dvchost=DEVFEYE1 spt=63559 dvc=10.100.25.16 smac=00:00:0c:07:ac:c8 cn1Label=vlan cn1=0 externalId=851820 cs4Label=link cs4=https://DEVCMS01.example.com/event_stream/events_for_bot?ev_id\=851820 dmac=88:43:e1:95:13:29 cs1Label=sname cs1=Trojan.Generic.DNS 
+<164>fenotify-851828.alert: CEF:0|FireEye|CMS|7.2.1.244420|DM|domain-match|1|rt=Feb 09 2015 05:09:07 UTC dvc=10.201.78.86 cn3Label=cncPort cn3=53 cn2Label=sid cn2=80494706 shost=dev001srv02.example.com proto=udp cs5Label=cncHost cs5=mfdclk001.org dvchost=DEVFEYE1 spt=52967 dvc=10.100.25.16 smac=00:00:0c:07:ac:c8 cn1Label=vlan cn1=0 externalId=851828 cs4Label=link cs4=https://DEVCMS01.example.com/event_stream/events_for_bot?ev_id\=851828 dmac=88:43:e1:95:13:29 cs1Label=sname cs1=Trojan.Generic.DNS 
+<164>fenotify-851816.alert: CEF:0|FireEye|CMS|7.2.1.244420|DM|domain-match|1|rt=Feb 09 2015 04:16:05 UTC dvc=10.201.78.34 cn3Label=cncPort cn3=53 cn2Label=sid cn2=80494706 proto=udp cs5Label=cncHost cs5=mfdclk001.org dvchost=DEVFEYE1 spt=61806 dvc=10.100.25.5 smac=00:00:0c:07:ac:63 cn1Label=vlan cn1=143 externalId=851816 cs4Label=link cs4=https://DEVCMS01.example.com/event_stream/events_for_bot?ev_id\=851816 dmac=00:1b:17:00:09:01 cs1Label=sname cs1=Trojan.Generic.DNS 
+<164>fenotify-851831.alert: CEF:0|FireEye|CMS|7.2.1.244420|DM|domain-match|1|rt=Feb 09 2015 05:14:31 UTC dvc=10.201.78.11 cn3Label=cncPort cn3=53 cn2Label=sid cn2=80494706 proto=udp cs5Label=cncHost cs5=mfdclk001.org dvchost=DEVFEYE1 spt=58655 dvc=10.100.25.5 smac=00:00:0c:07:ac:63 cn1Label=vlan cn1=143 externalId=851831 cs4Label=link cs4=https://DEVCMS01.example.com/event_stream/events_for_bot?ev_id\=851831 dmac=00:1b:17:00:09:01 cs1Label=sname cs1=Trojan.Generic.DNS 
+<164>fenotify-851950.alert: CEF:0|FireEye|CMS|7.2.1.244420|DM|domain-match|1|rt=Feb 09 2015 11:16:07 UTC dvc=10.201.78.11 cn3Label=cncPort cn3=53 cn2Label=sid cn2=80494706 proto=udp cs5Label=cncHost cs5=mfdclk001.org dvchost=DEVFEYE1 spt=58855 dvc=10.100.25.5 smac=00:00:0c:07:ac:63 cn1Label=vlan cn1=143 externalId=851950 cs4Label=link cs4=https://DEVCMS01.example.com/event_stream/events_for_bot?ev_id\=851950 dmac=00:1b:17:00:09:01 cs1Label=sname cs1=Trojan.Generic.DNS 
+<164>fenotify-851988.alert: CEF:0|FireEye|CMS|7.2.1.244420|DM|domain-match|1|rt=Feb 09 2015 12:35:26 UTC dvc=10.201.78.190 cn3Label=cncPort cn3=53 cn2Label=sid cn2=80494706 shost=dev001srv02.example.com proto=udp cs5Label=cncHost cs5=mfdclk001.org dvchost=DEVFEYE1 spt=61427 dvc=10.100.25.16 smac=00:00:0c:07:ac:00 cn1Label=vlan cn1=0 externalId=851988 cs4Label=link cs4=https://DEVCMS01.example.com/event_stream/events_for_bot?ev_id\=851988 dmac=00:1d:a2:af:32:a1 cs1Label=sname cs1=Trojan.Generic.DNS 
+<164>fenotify-852013.alert: CEF:0|FireEye|CMS|7.2.1.244420|DM|domain-match|1|rt=Feb 09 2015 13:18:29 UTC dvc=10.201.78.34 cn3Label=cncPort cn3=53 cn2Label=sid cn2=80494706 proto=udp cs5Label=cncHost cs5=mfdclk001.org dvchost=DEVFEYE1 spt=61630 dvc=10.100.25.5 smac=00:00:0c:07:ac:63 cn1Label=vlan cn1=143 externalId=852013 cs4Label=link cs4=https://DEVCMS01.example.com/event_stream/events_for_bot?ev_id\=852013 dmac=00:1b:17:00:09:01 cs1Label=sname cs1=Trojan.Generic.DNS 
+<164>fenotify-852070.alert: CEF:0|FireEye|CMS|7.2.1.244420|DM|domain-match|1|rt=Feb 09 2015 14:27:45 UTC dvc=10.201.78.44 cn3Label=cncPort cn3=53 cn2Label=sid cn2=80494706 shost=dev001srv02.example.com proto=udp cs5Label=cncHost cs5=mfdclk001.org dvchost=DEVFEYE1 spt=54769 dvc=10.100.25.16 smac=00:00:0c:07:ac:c8 cn1Label=vlan cn1=0 externalId=852070 cs4Label=link cs4=https://DEVCMS01.example.com/event_stream/events_for_bot?ev_id\=852070 dmac=88:43:e1:95:13:29 cs1Label=sname cs1=Trojan.Generic.DNS 
+<164>fenotify-852082.alert: CEF:0|FireEye|CMS|7.2.1.244420|DM|domain-match|1|rt=Feb 09 2015 14:35:15 UTC dvc=10.201.78.68 cn3Label=cncPort cn3=53 cn2Label=sid cn2=80494706 proto=udp cs5Label=cncHost cs5=mfdclk001.org dvchost=DEVFEYE1 spt=60922 dvc=10.100.25.16 smac=00:00:0c:07:ac:5a cn1Label=vlan cn1=0 externalId=852082 cs4Label=link cs4=https://DEVCMS01.example.com/event_stream/events_for_bot?ev_id\=852082 dmac=00:09:0f:33:4f:48 cs1Label=sname cs1=Trojan.Generic.DNS 
+<164>fenotify-852114.alert: CEF:0|FireEye|CMS|7.2.1.244420|DM|domain-match|1|rt=Feb 09 2015 14:59:08 UTC dvc=10.201.78.194 cn3Label=cncPort cn3=53 cn2Label=sid cn2=80494706 shost=dev001srv02.example.com proto=udp cs5Label=cncHost cs5=mfdclk001.org dvchost=DEVFEYE1 spt=50396 dvc=10.100.25.16 smac=00:00:0c:07:ac:c8 cn1Label=vlan cn1=0 externalId=852114 cs4Label=link cs4=https://DEVCMS01.example.com/event_stream/events_for_bot?ev_id\=852114 dmac=88:43:e1:95:13:29 cs1Label=sname cs1=Trojan.Generic.DNS 
+<164>fenotify-852295.alert: CEF:0|FireEye|CMS|7.2.1.244420|DM|domain-match|1|rt=Feb 09 2015 16:30:40 UTC dvc=10.201.78.51 cn3Label=cncPort cn3=53 cn2Label=sid cn2=80494706 shost=dev001srv02.example.com proto=udp cs5Label=cncHost cs5=mfdclk001.org dvchost=DEVFEYE1 spt=60266 dvc=10.100.25.16 smac=00:00:0c:07:ac:63 cn1Label=vlan cn1=0 externalId=852295 cs4Label=link cs4=https://DEVCMS01.example.com/event_stream/events_for_bot?ev_id\=852295 dmac=5c:5e:ab:eb:ab:0d cs1Label=sname cs1=Trojan.Generic.DNS 
+<164>fenotify-3483807.2.alert: z0Q6RNzwu2BoLSVUhiBihE4z0mlPDacuE1Waqs86Z9VVYg6iM2MlFH8GZgagnlOuzfB2JHdKPc/GwnzFk5DPfUPJAe8DH9Y6hwohv0t6XFVWx5UDSGARW8w3GAop9R+9iaSCuomuLU26/gaqL4gfjZqjLHzoDx+vhOCiOP6RnGMio5v2kcKxitPL7pPVu5FJ6MwUG7QOLecwONRzQsFh/jXFT4gyR2iS/EbufAeRofQVZHsj9dhgHZKNLcsFPnw/8lWlvgku7s28l57lAGxtp99jkzYD58jPgBm1nGbJPubFTL47ZmBkPPNsc1XjRcNvPz5/nzb0eWctXZ2cKocAQnT2zHOgBxRM6my9QW/Lg0JWaQyqBO2EOyTfej6KgVlHoIf0E3bv6C5PgVrJunAIqqlO6EvKvILlDYk2yoklvP3Fry5p4Nrw2isE95Used9Zqsoxx0bWInNcfyQhoqqlmYKiZZb+aBwGvJEL634pmoTMEBMdn4s3gz2a7aLV+vOVULQbgR15PygsYQdOnymv7uWZtdKOp7ut21GwNu9ZxJGMrssW0gzvaZiZDs7FSordVPUiUqcfS6ciU1cl29fNTWnmRkq4vk+vBgvUQLxTTAleV9k5svtB237GvvolWE72ugJQXUun51WxAqOAZpV0c6tEbK5qd6Z55z8Rs/LpN8VM4/nbZmfB5XY+eCCLfULjisVoStUUeH67&report\=p509XA27GEFLLes0RJ8pJJdIkbJ+3YkVUv2qjhuxlRPlVrrEZckeXFIaD+4/a1xulR8kKMx9GrPD2uc/wC+NxgKg/ok/kttHH45shX4YjPLsS4QtXUHugcE5Rr1238CYegHwOKWzAp3g5Mpt7loabRTBtmzXXeLBV4cFKv3zWpxQ7+CBGpsDfsvkD2Qgst3FX05VQHBpnJfXgRqdRrLyUjezF1tlIgvvNCv6hQ+zffxKk0WcD
 oUe8
+<164>fenotify-3483794.2.alert: 53 Safari/537.36::~~Host: 5aqobwcp1xuqztwht.0eq0w6k.com::~~Connection: Keep-Alive::~~::~~GET /93ea73bcdaf32d5074e62be84ee83a84cacefa8dcf855c265457842d6b05f469863ca7110d3cc7cc055f8d686a1b5d5c30db85c5423620e6bd231d592266782cf5e1647ae575e77b HTTP/1.1::~~Accept: */*::~~Proxy-Authorization: Basic ::~~User-Agent: Mozilla/5.0 (Windows NT 6.1; WOW64) AppleWebKit/537.36 (KHTML, like Gecko) Chrome/35.0.1916.153 Safari/537.36::~~Host: 5aqobwcp1xuqztwht.0eq0w6k.com::~~Connection: Keep-Alive::~~::~~GET /93ea73bcdaf32d5074e62be84ee83a84cacefa8dcf855c265457842d6b05f469863ca7110d3cc7cc055f8d686a1b5d5c30db85c5423620e6bd231d592266782cf5e1647ae575e77b HTTP/1.1::~~Accept: */*::~~Proxy-Authorization: Basic ::~~User-Agent: Mozilla/5.0 (Windows NT 6.1; WOW64) AppleWebKit/537.36 (KHTML, like Gecko) Chrome/35.0.1916.153 Safari/537.36::~~Host: 5aqobwcp1xuqztwht.0eq0w6k.com::~~Connection: Keep-Alive::~~::~~GET /93ea73bcdaf32d5074e62be84ee83a84cacefa8dcf855c265457842d6b05f469863c
 a711
+<164>fenotify-3483799.2.alert: L, like Gecko) Chrome/35.0.1916.153 Safari/537.36::~~Host: pkeyqcot5gzamu.5t9dyvo2.com::~~Connection: Keep-Alive::~~::~~ cs2Label=anomaly cs2=misc-anomaly cs1Label=sname cs1=FE_PUP_Softpulse;FE_Evasion_VMDetect;FE_Evasion_DBGDetect_Files;FE_Evasion_Sandboxie 
+<164>fenotify-3483807.3.alert: n6o4JWRQX2V1jsLkx8LFQz3nXe7Bbiuuc1sMcdS/lEv7f9zpw09qs0LvVpRJe4tZjE4Gsghh7Xh5OAxE2A7HBLnWjloIazv6jvun+R1BpF1vuujyEdDgKWIv4BeMmQQJ6p66O/U0jHvWelTBMT+RTVFERsryrpWE+g7AHeRyzDIERgWxHxzA9y6cQ9JYp2/JOPdUzWnLWM24Be6fWmlJ37J90GuEvHh+WXWsaewcBg8xUAhlQBfEHP01PGcuX2yJin2rQ8/GhkiF210HCJUCIbxxz6rZuf6CaksKSXPIeXf1Iifha58Rtm cs2Label=anomaly cs2=misc-anomaly cs1Label=sname cs1=Malware.Binary 
\ No newline at end of file


[06/15] incubator-metron git commit: METRON-142 Simplify Parser configuration (merrimanr via cestella) closes apache/incubator-metron#120

Posted by ce...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/df8d682e/metron-platform/metron-integration-test/src/main/sample/data/SampleInput/ISESampleOutput
----------------------------------------------------------------------
diff --git a/metron-platform/metron-integration-test/src/main/sample/data/SampleInput/ISESampleOutput b/metron-platform/metron-integration-test/src/main/sample/data/SampleInput/ISESampleOutput
new file mode 100644
index 0000000..1cb0678
--- /dev/null
+++ b/metron-platform/metron-integration-test/src/main/sample/data/SampleInput/ISESampleOutput
@@ -0,0 +1,308 @@
+Aug  6 17:26:31 10.34.84.145 Aug  7 00:45:43 stage-pdp01 CISE_Profiler 0000024855 1 0 2014-08-07 00:45:43.741 -07:00 0000288542 80002 INFO  Profiler: Profiler EndPoint profiling event occurred, ConfigVersionId=113, EndpointCertainityMetric=10, EndpointIPAddress=10.56.111.14, EndpointMacAddress=3C:97:0E:C3:F8:F1, EndpointMatchedPolicy=Nortel-Device, EndpointNADAddress=10.56.72.127, EndpointOUI=Wistron InfoComm(Kunshan)Co.\,Ltd., EndpointPolicy=Nortel-Device, EndpointProperty=StaticAssignment=false\,PostureApplicable=Yes\,PolicyVersion=402\,IdentityGroupID=0c1d9270-68a6-11e1-bc72-0050568e013c\,Total Certainty Factor=10\,BYODRegistration=Unknown\,FeedService=false\,EndPointPolicyID=49054ed0-68a6-11e1-bc72-0050568e013c\,FirstCollection=1407397543718\,MatchedPolicyID=49054ed0-68a6-11e1-bc72-0050568e013c\,TimeToProfile=19\,StaticGroupAssignment=false\,NmapSubnetScanID=0\,DeviceRegistrationStatus=NotRegistered\,PortalUser=, EndpointSourceEvent=SNMPQuery Probe, EndpointIdentityGroup=Profile
 d, ProfilerServer=stage-pdp01.cisco.com, 
+Aug  6 17:26:31 10.34.84.145 Aug  7 00:45:43 stage-pdp01 CISE_Profiler 0000024856 1 0 2014-08-07 00:45:43.786 -07:00 0000288543 80002 INFO  Profiler: Profiler EndPoint profiling event occurred, ConfigVersionId=113, EndpointCertainityMetric=10, EndpointIPAddress=10.56.111.14, EndpointMacAddress=3C:97:0E:C3:F8:F1, EndpointMatchedPolicy=Nortel-Device, EndpointNADAddress=10.56.72.127, EndpointOUI=Wistron InfoComm(Kunshan)Co.\,Ltd., EndpointPolicy=Nortel-Device, EndpointProperty=StaticAssignment=false\,PostureApplicable=Yes\,BYODRegistration=Unknown\,EndPointPolicyID=49054ed0-68a6-11e1-bc72-0050568e013c\,FirstCollection=1407397543718\,TimeToProfile=19\,LastNmapScanTime=0\,StaticGroupAssignment=false\,DeviceRegistrationStatus=NotRegistered\,UpdateTime=0\,PolicyVersion=402\,IdentityGroupID=0c1d9270-68a6-11e1-bc72-0050568e013c\,Total Certainty Factor=10\,FeedService=false\,MatchedPolicyID=49054ed0-68a6-11e1-bc72-0050568e013c\,NmapScanCount=0\,NmapSubnetScanID=0\,PortalUser=, EndpointSourceE
 vent=SNMPQuery Probe, EndpointIdentityGroup=Profiled, ProfilerServer=stage-pdp01.cisco.com, 
+Aug  6 20:00:52 10.42.7.64 Aug  7 03:20:05 npf-sjca-pdp02 CISE_Profiler 0000373185 1 0 2014-08-07 03:20:05.549 -07:00 0011310202 80002 INFO  Profiler: Profiler EndPoint profiling event occurred, ConfigVersionId=241, EndpointCertainityMetric=90, EndpointIPAddress=10.56.129.142, EndpointMacAddress=3C:A9:F4:46:75:CC, EndpointMatchedPolicy=Windows7-Workstation, EndpointNADAddress=10.56.129.4, EndpointOUI=Intel Corporate, EndpointPolicy=Windows7-Workstation, EndpointProperty=StaticAssignment=false\,Calling-Station-ID=3c-a9-f4-46-75-cc\,Device Identifier=\,PostureApplicable=Yes\,dhcp-class-identifier=MSFT 5.0\,host-name=HASSI-WS03\,BYODRegistration=Unknown\,EndPointPolicyID=615ed410-68a6-11e1-bc72-0050568e013c\,FirstCollection=1406112353750\,TimeToProfile=11\,Framed-IP-Address=10.56.129.142\,LastNmapScanTime=0\,StaticGroupAssignment=false\,DeviceRegistrationStatus=NotRegistered\,NAS-Port-Type=Wireless - IEEE 802.11\,RegistrationTimeStamp=0\,UpdateTime=1407394245820\,PolicyVersion=403\,Ide
 ntityGroupID=5cb39b80-68a6-11e1-bc72-0050568e013c\,Total Certainty Factor=90\,FeedService=false\,MatchedPolicyID=615ed410-68a6-11e1-bc72-0050568e013c\,DestinationIPAddress=10.42.7.64\,CreateTime=1394526689397\,NmapScanCount=0\,NmapSubnetScanID=0\,AAA-Server=npf-sjca-pdp02\,PortalUser=, EndpointSourceEvent=RADIUS Probe, EndpointUserAgent=Mozilla/5.0 (Windows NT 6.1\\\; WOW64\\ rv:30.0) Gecko/20100101 Firefox/30.0, EndpointIdentityGroup=Workstation, ProfilerServer=npf-sjca-pdp02.cisco.com, 
+Aug  6 21:00:48 10.42.7.64 Aug  7 04:20:00 npf-sjca-pdp02 CISE_Profiler 0000373902 1 0 2014-08-07 04:20:00.983 -07:00 0011322557 80002 INFO  Profiler: Profiler EndPoint profiling event occurred, ConfigVersionId=241, EndpointCertainityMetric=30, EndpointIPAddress=10.56.129.142, EndpointMacAddress=3C:A9:F4:46:75:CC, EndpointMatchedPolicy=Microsoft-Workstation, EndpointNADAddress=10.56.129.4, EndpointOUI=Intel Corporate, EndpointPolicy=Microsoft-Workstation, EndpointProperty=StaticAssignment=false\,Calling-Station-ID=3c-a9-f4-46-75-cc\,Device Identifier=\,PostureApplicable=Yes\,dhcp-class-identifier=MSFT 5.0\,host-name=HASSI-WS03\,BYODRegistration=Unknown\,EndPointPolicyID=5f4a24e0-68a6-11e1-bc72-0050568e013c\,FirstCollection=1406112353750\,TimeToProfile=11\,Framed-IP-Address=10.56.129.142\,LastNmapScanTime=0\,StaticGroupAssignment=false\,DeviceRegistrationStatus=NotRegistered\,NAS-Port-Type=Wireless - IEEE 802.11\,RegistrationTimeStamp=0\,UpdateTime=1407406806572\,PolicyVersion=403\,I
 dentityGroupID=5cb39b80-68a6-11e1-bc72-0050568e013c\,Total Certainty Factor=30\,FeedService=false\,MatchedPolicyID=5f4a24e0-68a6-11e1-bc72-0050568e013c\,DestinationIPAddress=10.42.7.64\,CreateTime=1394526689397\,NmapScanCount=0\,NmapSubnetScanID=0\,AAA-Server=npf-sjca-pdp02\,PortalUser=, EndpointSourceEvent=RADIUS Probe, EndpointUserAgent=MS-WebServices/1.0, EndpointIdentityGroup=Workstation, ProfilerServer=npf-sjca-pdp02.cisco.com, 
+Aug  6 22:22:50 10.42.7.64 Aug  7 05:42:03 npf-sjca-pdp02 CISE_Profiler 0000374846 1 0 2014-08-07 05:42:03.617 -07:00 0011340138 80002 INFO  Profiler: Profiler EndPoint profiling event occurred, ConfigVersionId=241, EndpointCertainityMetric=10, EndpointMacAddress=68:A8:6D:4E:0D:86, EndpointMatchedPolicy=Apple-Device, EndpointOUI=Apple, EndpointPolicy=Apple-Device, EndpointProperty=StaticAssignment=false\,PostureApplicable=Yes\,host-name=PGIANG-M-306R\,BYODRegistration=Unknown\,EndPointPolicyID=377d8ba0-68a6-11e1-bc72-0050568e013c\,FirstCollection=1407415322895\,TimeToProfile=717\,StaticGroupAssignment=false\,DeviceRegistrationStatus=NotRegistered\,PolicyVersion=403\,IdentityGroupID=abbbcac0-89e6-11e1-bf14-005056aa4dd7\,Total Certainty Factor=10\,ciaddr=0.0.0.0\,FeedService=false\,dhcp-parameter-request-list=1\, 3\, 6\, 15\, 119\, 95\, 252\, 44\, 46\,MatchedPolicyID=377d8ba0-68a6-11e1-bc72-0050568e013c\,NmapSubnetScanID=0\,PortalUser=, EndpointSourceEvent=DHCP Probe, EndpointIdentity
 Group=Apple-Device, ProfilerServer=npf-sjca-pdp02.cisco.com, 
+Aug  6 23:30:10 10.42.7.64 Aug  7 06:49:23 npf-sjca-pdp02 CISE_Profiler 0000375603 1 0 2014-08-07 06:49:23.920 -07:00 0011353768 80002 INFO  Profiler: Profiler EndPoint profiling event occurred, ConfigVersionId=241, EndpointCertainityMetric=90, EndpointIPAddress=10.56.129.142, EndpointMacAddress=3C:A9:F4:46:75:CC, EndpointMatchedPolicy=Windows7-Workstation, EndpointNADAddress=10.56.129.4, EndpointOUI=Intel Corporate, EndpointPolicy=Windows7-Workstation, EndpointProperty=StaticAssignment=false\,Calling-Station-ID=3c-a9-f4-46-75-cc\,Device Identifier=\,PostureApplicable=Yes\,dhcp-class-identifier=MSFT 5.0\,host-name=HASSI-WS03\,BYODRegistration=Unknown\,EndPointPolicyID=615ed410-68a6-11e1-bc72-0050568e013c\,FirstCollection=1406112353750\,TimeToProfile=11\,Framed-IP-Address=10.56.129.142\,LastNmapScanTime=0\,StaticGroupAssignment=false\,DeviceRegistrationStatus=NotRegistered\,NAS-Port-Type=Wireless - IEEE 802.11\,RegistrationTimeStamp=0\,UpdateTime=1407410402099\,PolicyVersion=403\,Ide
 ntityGroupID=5cb39b80-68a6-11e1-bc72-0050568e013c\,Total Certainty Factor=90\,FeedService=false\,MatchedPolicyID=615ed410-68a6-11e1-bc72-0050568e013c\,DestinationIPAddress=10.42.7.64\,CreateTime=1394526689397\,NmapScanCount=0\,NmapSubnetScanID=0\,AAA-Server=npf-sjca-pdp02\,PortalUser=, EndpointSourceEvent=RADIUS Probe, EndpointUserAgent=Mozilla/5.0 (Windows NT 6.1\\\; WOW64\\ rv:30.0) Gecko/20100101 Firefox/30.0, EndpointIdentityGroup=Workstation, ProfilerServer=npf-sjca-pdp02.cisco.com, 
+Aug  6 23:30:48 10.42.7.64 Aug  7 06:50:01 npf-sjca-pdp02 CISE_Profiler 0000375611 1 0 2014-08-07 06:50:01.377 -07:00 0011353875 80002 INFO  Profiler: Profiler EndPoint profiling event occurred, ConfigVersionId=241, EndpointCertainityMetric=50, EndpointIPAddress=10.34.92.103, EndpointMacAddress=3C:A9:F4:29:FC:3C, EndpointMatchedPolicy=Microsoft-Workstation, EndpointNADAddress=10.34.76.212, EndpointOUI=Intel Corporate, EndpointPolicy=Microsoft-Workstation, EndpointProperty=StaticAssignment=false\,Calling-Station-ID=3c-a9-f4-29-fc-3c\,Device Identifier=\,PostureApplicable=Yes\,dhcp-class-identifier=MSFT 5.0\,host-name=AMIBASU-WS01\,BYODRegistration=Unknown\,EndPointPolicyID=5f4a24e0-68a6-11e1-bc72-0050568e013c\,FirstCollection=1406109860322\,L4_DST_PORT=50428\,TimeToProfile=7\,Framed-IP-Address=10.34.92.103\,LastNmapScanTime=1380758278898\,StaticGroupAssignment=false\,DeviceRegistrationStatus=NotRegistered\,NAS-Port-Type=Wireless - IEEE 802.11\,RegistrationTimeStamp=0\,UpdateTime=1406
 686034558\,PolicyVersion=403\,IdentityGroupID=5cb39b80-68a6-11e1-bc72-0050568e013c\,Total Certainty Factor=50\,operating-system=Microsoft Windows Vista SP0 - SP2\, Server 2008\, or Windows 7 Ultimate\,FeedService=false\,MatchedPolicyID=5f4a24e0-68a6-11e1-bc72-0050568e013c\,DestinationIPAddress=10.42.7.64\,CreateTime=1373657280926\,NmapScanCount=3\,NmapSubnetScanID=0\,AAA-Server=npf-sjca-pdp02\,PortalUser=, EndpointSourceEvent=RADIUS Probe, EndpointUserAgent=MS-WebServices/1.0, EndpointIdentityGroup=Workstation, ProfilerServer=npf-sjca-pdp02.cisco.com, 
+Aug  6 23:32:52 10.42.7.64 Aug  7 06:52:05 npf-sjca-pdp02 CISE_Profiler 0000375636 1 0 2014-08-07 06:52:05.272 -07:00 0011354313 80002 INFO  Profiler: Profiler EndPoint profiling event occurred, ConfigVersionId=241, EndpointCertainityMetric=30, EndpointIPAddress=10.56.129.143, EndpointMacAddress=E8:2A:EA:23:5E:3D, EndpointMatchedPolicy=Microsoft-Workstation, EndpointNADAddress=10.56.129.4, EndpointOUI=Intel Corporate, EndpointPolicy=Microsoft-Workstation, EndpointProperty=StaticAssignment=false\,Calling-Station-ID=e8-2a-ea-23-5e-3d\,Device Identifier=\,PostureApplicable=Yes\,dhcp-class-identifier=MSFT 5.0\,host-name=ANOY-WS01\,BYODRegistration=Unknown\,EndPointPolicyID=5f4a24e0-68a6-11e1-bc72-0050568e013c\,FirstCollection=1406114784910\,TimeToProfile=7\,Framed-IP-Address=10.56.129.143\,LastNmapScanTime=0\,StaticGroupAssignment=false\,DeviceRegistrationStatus=NotRegistered\,NAS-Port-Type=Wireless - IEEE 802.11\,RegistrationTimeStamp=0\,UpdateTime=1407395211208\,PolicyVersion=403\,Ide
 ntityGroupID=5cb39b80-68a6-11e1-bc72-0050568e013c\,Total Certainty Factor=30\,FeedService=false\,MatchedPolicyID=5f4a24e0-68a6-11e1-bc72-0050568e013c\,DestinationIPAddress=10.42.7.64\,CreateTime=1405408515121\,NmapScanCount=0\,NmapSubnetScanID=0\,AAA-Server=npf-sjca-pdp02\,PortalUser=, EndpointSourceEvent=RADIUS Probe, EndpointUserAgent=MS-WebServices/1.0, EndpointIdentityGroup=Workstation, ProfilerServer=npf-sjca-pdp02.cisco.com, 
+Aug  6 16:40:52 10.42.7.64 Aug  7 00:00:04 npf-sjca-pdp02 CISE_Failed_Attempts 0000370855 1 0 2014-08-07 00:00:04.527 -07:00 0011266584 5405 NOTICE Failed-Attempt: RADIUS Request dropped, ConfigVersionId=240, Device IP Address=10.56.72.126, Device Port=1645, DestinationIPAddress=10.42.7.64, DestinationPort=1812, RadiusPacketType=AccessRequest, UserName=#CTSREQUEST#, Protocol=Radius, RequestLatency=5, NetworkDeviceName=ntn01-11a-sw3, User-Name=#CTSREQUEST#, NAS-IP-Address=10.56.72.126, Service-Type=Outbound, AcsSessionID=npf-sjca-pdp02/195481465/270932, SelectedAccessService=NDAC_SGT_Service, FailureReason=11302 Received Secure RADIUS request without a cts-pac-opaque cisco-av-pair attribute, Step=11001, Step=11017, Step=15012, Step=11302, NetworkDeviceGroups=Location#All Locations#NTN, NetworkDeviceGroups=Device Type#All Device Types#Wired, CPMSessionID=0a2a0740000056EF53E323F4, Model Name=4503, Location=Location#All Locations#NTN, Device Type=Device Type#All Device Types#Wired, Resp
 onse={RadiusPacketType=Drop; },
+Aug  6 16:40:57 10.42.7.63 Aug  7 00:00:09 npf-sjca-pdp01 CISE_Failed_Attempts 0001969834 1 0 2014-08-07 00:00:09.568 -07:00 0098648519 5405 NOTICE Failed-Attempt: RADIUS Request dropped, ConfigVersionId=133, Device IP Address=10.56.72.126, Device Port=1645, DestinationIPAddress=10.42.7.63, DestinationPort=1812, RadiusPacketType=AccessRequest, UserName=#CTSREQUEST#, Protocol=Radius, RequestLatency=2, NetworkDeviceName=ntn01-11a-sw3, User-Name=#CTSREQUEST#, NAS-IP-Address=10.56.72.126, Service-Type=Outbound, AcsSessionID=npf-sjca-pdp01/195491152/2084839, SelectedAccessService=NDAC_SGT_Service, FailureReason=11302 Received Secure RADIUS request without a cts-pac-opaque cisco-av-pair attribute, Step=11001, Step=11017, Step=15012, Step=11302, NetworkDeviceGroups=Location#All Locations#NTN, NetworkDeviceGroups=Device Type#All Device Types#Wired, CPMSessionID=0a2a073f00005D4A53E323F9, Model Name=4503, Location=Location#All Locations#NTN, Device Type=Device Type#All Device Types#Wired, Res
 ponse={RadiusPacketType=Drop; },
+Aug  6 16:41:24 10.34.84.145 Aug  7 00:00:36 stage-pdp01 CISE_Failed_Attempts 0000024616 1 0 2014-08-07 00:00:36.332 -07:00 0000287007 5405 NOTICE Failed-Attempt: RADIUS Request dropped, ConfigVersionId=113, Device IP Address=172.23.91.132, Device Port=32769, DestinationIPAddress=10.34.84.145, DestinationPort=1812, Protocol=Radius, User-Name=test, NAS-IP-Address=192.168.30.11, Service-Type=NAS Prompt, NAS-Identifier=Cisco_1b:e0:84, AcsSessionID=stage-pdp01/196593288/19317, FailureReason=11007 Could not locate Network Device or AAA Client, Step=11001, Step=11017, Step=11007, Step=5405, 
+Aug  6 16:41:26 10.34.84.145 Aug  7 00:00:38 stage-pdp01 CISE_Failed_Attempts 0000024617 1 0 2014-08-07 00:00:38.336 -07:00 0000287011 5405 NOTICE Failed-Attempt: RADIUS Request dropped, ConfigVersionId=113, Device IP Address=172.23.91.132, Device Port=32769, DestinationIPAddress=10.34.84.145, DestinationPort=1812, Protocol=Radius, User-Name=test, NAS-IP-Address=192.168.30.11, Service-Type=NAS Prompt, NAS-Identifier=Cisco_1b:e0:84, AcsSessionID=stage-pdp01/196593288/19318, FailureReason=11007 Could not locate Network Device or AAA Client, Step=11001, Step=11017, Step=11007, Step=5405, 
+Aug  6 16:41:28 10.34.84.145 Aug  7 00:00:40 stage-pdp01 CISE_Failed_Attempts 0000024618 1 0 2014-08-07 00:00:40.336 -07:00 0000287015 5405 NOTICE Failed-Attempt: RADIUS Request dropped, ConfigVersionId=113, Device IP Address=172.23.91.132, Device Port=32769, DestinationIPAddress=10.34.84.145, DestinationPort=1812, Protocol=Radius, User-Name=test, NAS-IP-Address=192.168.30.11, Service-Type=NAS Prompt, NAS-Identifier=Cisco_1b:e0:84, AcsSessionID=stage-pdp01/196593288/19319, FailureReason=11007 Could not locate Network Device or AAA Client, Step=11001, Step=11017, Step=11007, Step=5405, 
+Aug  6 16:41:30 10.34.84.145 Aug  7 00:00:42 stage-pdp01 CISE_Failed_Attempts 0000024619 1 0 2014-08-07 00:00:42.340 -07:00 0000287019 5405 NOTICE Failed-Attempt: RADIUS Request dropped, ConfigVersionId=113, Device IP Address=172.23.91.132, Device Port=32769, DestinationIPAddress=10.34.84.145, DestinationPort=1812, Protocol=Radius, User-Name=test, NAS-IP-Address=192.168.30.11, Service-Type=NAS Prompt, NAS-Identifier=Cisco_1b:e0:84, AcsSessionID=stage-pdp01/196593288/19320, FailureReason=11007 Could not locate Network Device or AAA Client, Step=11001, Step=11017, Step=11007, Step=5405, 
+Aug  6 16:41:32 10.34.84.145 Aug  7 00:00:44 stage-pdp01 CISE_Failed_Attempts 0000024620 1 0 2014-08-07 00:00:44.340 -07:00 0000287023 5405 NOTICE Failed-Attempt: RADIUS Request dropped, ConfigVersionId=113, Device IP Address=172.23.91.132, Device Port=32769, DestinationIPAddress=10.34.84.145, DestinationPort=1812, Protocol=Radius, User-Name=test, NAS-IP-Address=192.168.30.11, Service-Type=NAS Prompt, NAS-Identifier=Cisco_1b:e0:84, AcsSessionID=stage-pdp01/196593288/19321, FailureReason=11007 Could not locate Network Device or AAA Client, Step=11001, Step=11017, Step=11007, Step=5405, 
+Aug  6 16:41:34 10.34.84.145 Aug  7 00:00:46 stage-pdp01 CISE_Failed_Attempts 0000024621 1 0 2014-08-07 00:00:46.344 -07:00 0000287027 5405 NOTICE Failed-Attempt: RADIUS Request dropped, ConfigVersionId=113, Device IP Address=172.23.91.132, Device Port=32769, DestinationIPAddress=10.34.84.145, DestinationPort=1812, Protocol=Radius, User-Name=test, NAS-IP-Address=192.168.30.11, Service-Type=NAS Prompt, NAS-Identifier=Cisco_1b:e0:84, AcsSessionID=stage-pdp01/196593288/19322, FailureReason=11007 Could not locate Network Device or AAA Client, Step=11001, Step=11017, Step=11007, Step=5405, 
+Aug  6 16:42:02 10.42.7.64 Aug  7 00:01:14 npf-sjca-pdp02 CISE_Failed_Attempts 0000370865 1 0 2014-08-07 00:01:14.610 -07:00 0011266810 5405 NOTICE Failed-Attempt: RADIUS Request dropped, ConfigVersionId=240, Device IP Address=10.56.72.126, Device Port=1645, DestinationIPAddress=10.42.7.64, DestinationPort=1812, RadiusPacketType=AccessRequest, UserName=#CTSREQUEST#, Protocol=Radius, RequestLatency=7, NetworkDeviceName=ntn01-11a-sw3, User-Name=#CTSREQUEST#, NAS-IP-Address=10.56.72.126, Service-Type=Outbound, AcsSessionID=npf-sjca-pdp02/195481465/270940, SelectedAccessService=NDAC_SGT_Service, FailureReason=11302 Received Secure RADIUS request without a cts-pac-opaque cisco-av-pair attribute, Step=11001, Step=11017, Step=15012, Step=11302, NetworkDeviceGroups=Location#All Locations#NTN, NetworkDeviceGroups=Device Type#All Device Types#Wired, CPMSessionID=0a2a0740000056F053E3243A, Model Name=4503, Location=Location#All Locations#NTN, Device Type=Device Type#All Device Types#Wired, Resp
 onse={RadiusPacketType=Drop; },
+Aug  6 16:42:07 10.42.7.63 Aug  7 00:01:19 npf-sjca-pdp01 CISE_Failed_Attempts 0001969923 1 0 2014-08-07 00:01:19.665 -07:00 0098652715 5405 NOTICE Failed-Attempt: RADIUS Request dropped, ConfigVersionId=133, Device IP Address=10.56.72.126, Device Port=1645, DestinationIPAddress=10.42.7.63, DestinationPort=1812, RadiusPacketType=AccessRequest, UserName=#CTSREQUEST#, Protocol=Radius, RequestLatency=2, NetworkDeviceName=ntn01-11a-sw3, User-Name=#CTSREQUEST#, NAS-IP-Address=10.56.72.126, Service-Type=Outbound, AcsSessionID=npf-sjca-pdp01/195491152/2084986, SelectedAccessService=NDAC_SGT_Service, FailureReason=11302 Received Secure RADIUS request without a cts-pac-opaque cisco-av-pair attribute, Step=11001, Step=11017, Step=15012, Step=11302, NetworkDeviceGroups=Location#All Locations#NTN, NetworkDeviceGroups=Device Type#All Device Types#Wired, CPMSessionID=0a2a073f00005D4B53E3243F, Model Name=4503, Location=Location#All Locations#NTN, Device Type=Device Type#All Device Types#Wired, Res
 ponse={RadiusPacketType=Drop; },
+Aug  6 16:42:12 10.42.7.64 Aug  7 00:01:24 npf-sjca-pdp02 CISE_Failed_Attempts 0000370867 1 0 2014-08-07 00:01:24.701 -07:00 0011266815 5405 NOTICE Failed-Attempt: RADIUS Request dropped, ConfigVersionId=240, Device IP Address=10.56.72.126, Device Port=1645, DestinationIPAddress=10.42.7.64, DestinationPort=1812, RadiusPacketType=AccessRequest, UserName=#CTSREQUEST#, Protocol=Radius, RequestLatency=5, NetworkDeviceName=ntn01-11a-sw3, User-Name=#CTSREQUEST#, NAS-IP-Address=10.56.72.126, Service-Type=Outbound, AcsSessionID=npf-sjca-pdp02/195481465/270941, SelectedAccessService=NDAC_SGT_Service, FailureReason=11302 Received Secure RADIUS request without a cts-pac-opaque cisco-av-pair attribute, Step=11001, Step=11017, Step=15012, Step=11302, NetworkDeviceGroups=Location#All Locations#NTN, NetworkDeviceGroups=Device Type#All Device Types#Wired, CPMSessionID=0a2a0740000056F153E32444, Model Name=4503, Location=Location#All Locations#NTN, Device Type=Device Type#All Device Types#Wired, Resp
 onse={RadiusPacketType=Drop; },
+Aug  6 16:42:17 10.42.7.63 Aug  7 00:01:29 npf-sjca-pdp01 CISE_Failed_Attempts 0001969935 1 0 2014-08-07 00:01:29.746 -07:00 0098653362 5405 NOTICE Failed-Attempt: RADIUS Request dropped, ConfigVersionId=133, Device IP Address=10.56.72.126, Device Port=1645, DestinationIPAddress=10.42.7.63, DestinationPort=1812, RadiusPacketType=AccessRequest, UserName=#CTSREQUEST#, Protocol=Radius, RequestLatency=1, NetworkDeviceName=ntn01-11a-sw3, User-Name=#CTSREQUEST#, NAS-IP-Address=10.56.72.126, Service-Type=Outbound, AcsSessionID=npf-sjca-pdp01/195491152/2085007, SelectedAccessService=NDAC_SGT_Service, FailureReason=11302 Received Secure RADIUS request without a cts-pac-opaque cisco-av-pair attribute, Step=11001, Step=11017, Step=15012, Step=11302, NetworkDeviceGroups=Location#All Locations#NTN, NetworkDeviceGroups=Device Type#All Device Types#Wired, CPMSessionID=0a2a073f00005D4C53E32449, Model Name=4503, Location=Location#All Locations#NTN, Device Type=Device Type#All Device Types#Wired, Res
 ponse={RadiusPacketType=Drop; },
+Aug  6 16:43:22 10.42.7.64 Aug  7 00:02:34 npf-sjca-pdp02 CISE_Failed_Attempts 0000370885 1 0 2014-08-07 00:02:34.792 -07:00 0011267367 5405 NOTICE Failed-Attempt: RADIUS Request dropped, ConfigVersionId=240, Device IP Address=10.56.72.126, Device Port=1645, DestinationIPAddress=10.42.7.64, DestinationPort=1812, RadiusPacketType=AccessRequest, UserName=#CTSREQUEST#, Protocol=Radius, RequestLatency=4, NetworkDeviceName=ntn01-11a-sw3, User-Name=#CTSREQUEST#, NAS-IP-Address=10.56.72.126, Service-Type=Outbound, AcsSessionID=npf-sjca-pdp02/195481465/270956, SelectedAccessService=NDAC_SGT_Service, FailureReason=11302 Received Secure RADIUS request without a cts-pac-opaque cisco-av-pair attribute, Step=11001, Step=11017, Step=15012, Step=11302, NetworkDeviceGroups=Location#All Locations#NTN, NetworkDeviceGroups=Device Type#All Device Types#Wired, CPMSessionID=0a2a0740000056F353E3248A, Model Name=4503, Location=Location#All Locations#NTN, Device Type=Device Type#All Device Types#Wired, Resp
 onse={RadiusPacketType=Drop; },
+Aug  6 16:43:27 10.42.7.63 Aug  7 00:02:39 npf-sjca-pdp01 CISE_Failed_Attempts 0001970043 1 0 2014-08-07 00:02:39.808 -07:00 0098657578 5405 NOTICE Failed-Attempt: RADIUS Request dropped, ConfigVersionId=133, Device IP Address=10.56.72.126, Device Port=1645, DestinationIPAddress=10.42.7.63, DestinationPort=1812, RadiusPacketType=AccessRequest, UserName=#CTSREQUEST#, Protocol=Radius, RequestLatency=1, NetworkDeviceName=ntn01-11a-sw3, User-Name=#CTSREQUEST#, NAS-IP-Address=10.56.72.126, Service-Type=Outbound, AcsSessionID=npf-sjca-pdp01/195491152/2085161, SelectedAccessService=NDAC_SGT_Service, FailureReason=11302 Received Secure RADIUS request without a cts-pac-opaque cisco-av-pair attribute, Step=11001, Step=11017, Step=15012, Step=11302, NetworkDeviceGroups=Location#All Locations#NTN, NetworkDeviceGroups=Device Type#All Device Types#Wired, CPMSessionID=0a2a073f00005D4D53E3248F, Model Name=4503, Location=Location#All Locations#NTN, Device Type=Device Type#All Device Types#Wired, Res
 ponse={RadiusPacketType=Drop; },
+Aug  6 16:43:56 10.42.7.64 Aug  7 00:03:08 npf-sjca-pdp02 CISE_Failed_Attempts 0000370897 1 0 2014-08-07 00:03:08.902 -07:00 0011267657 5435 NOTICE RADIUS: NAS conducted several failed authentications of the same scenario, ConfigVersionId=240, Device IP Address=10.56.129.4, Device Port=32770, DestinationIPAddress=10.42.7.64, DestinationPort=1813, RadiusPacketType=AccountingRequest, UserName=yshchory, Protocol=Radius, RequestLatency=49, NetworkDeviceName=NTN-WLC1, User-Name=yshchory, NAS-IP-Address=10.56.129.4, NAS-Port=1, Framed-IP-Address=10.56.129.141, Class=CACS:0a388104000045cd53e2be75:npf-sjca-pdp02/195481465/270958, Called-Station-ID=6c-41-6a-5f-6e-c0, Calling-Station-ID=90-18-7c-7b-59-01, NAS-Identifier=ntn01-11a-wlc1, Acct-Status-Type=Interim-Update, Acct-Delay-Time=0, Acct-Input-Octets=2359603, Acct-Output-Octets=26928466, Acct-Session-Id=53e2be78/90:18:7c:7b:59:01/13844, Acct-Authentic=RADIUS, Acct-Session-Time=1466, Acct-Input-Packets=14866, Acct-Output-Packets=23043, und
 efined-52=
+Aug  6 16:44:01 10.42.7.63 Aug  7 00:03:13 npf-sjca-pdp01 CISE_Failed_Attempts 0001970072 1 0 2014-08-07 00:03:13.112 -07:00 0098658804 5435 NOTICE RADIUS: NAS conducted several failed authentications of the same scenario, ConfigVersionId=133, Device IP Address=10.56.72.127, Device Port=1646, DestinationIPAddress=10.42.7.63, DestinationPort=1813, Protocol=Radius, NetworkDeviceName=ntn01-11a-sw4, User-Name=host/salfi-pc.cisco.com, NAS-IP-Address=10.56.72.127, NAS-Port=50212, Service-Type=Framed, Framed-IP-Address=10.56.111.14, Class=CACS:0A38487F00000397BDA7BCAC:npf-sjca-pdp02/195481465/270957, Called-Station-ID=00-26-99-28-5E-BB, Calling-Station-ID=3C-97-0E-C3-F8-F1, Acct-Status-Type=Interim-Update, Acct-Delay-Time=4, Acct-Input-Octets=225395, Acct-Output-Octets=761436, Acct-Session-Id=00000560, Acct-Authentic=RADIUS, Acct-Session-Time=43, Acct-Input-Packets=1163, Acct-Output-Packets=1080, NAS-Port-Type=Ethernet, NAS-Port-Id=GigabitEthernet2/12, undefined-151=F54C88B0, cisco-av-pair
 =audit-session-id=0A38487F00000397BDA7BCAC, cisco-av-pair=connect-progress=Auth Open, AcsSessionID=npf-sjca-pdp01/195491152/2085221, FailureReason=11038 RADIUS Accounting-Request header contains invalid Authenticator field, Step=11004, Step=11017, Step=11038, Step=5435, NetworkDeviceGroups=Location#All Locations#NTN, NetworkDeviceGroups=Device Type#All Device Types#Wired, CPMSessionID=0A38487F00000397BDA7BCAC, TotalFailedAttempts=2, TotalFailedTime=42, Model Name=4503, Location=Location#All Locations#NTN, Device Type=Device Type#All Device Types#Wired, 
+Aug  6 16:44:32 10.42.7.64 Aug  7 00:03:44 npf-sjca-pdp02 CISE_Failed_Attempts 0000370899 1 0 2014-08-07 00:03:44.851 -07:00 0011267663 5405 NOTICE Failed-Attempt: RADIUS Request dropped, ConfigVersionId=240, Device IP Address=10.56.72.126, Device Port=1645, DestinationIPAddress=10.42.7.64, DestinationPort=1812, RadiusPacketType=AccessRequest, UserName=#CTSREQUEST#, Protocol=Radius, RequestLatency=7, NetworkDeviceName=ntn01-11a-sw3, User-Name=#CTSREQUEST#, NAS-IP-Address=10.56.72.126, Service-Type=Outbound, AcsSessionID=npf-sjca-pdp02/195481465/270963, SelectedAccessService=NDAC_SGT_Service, FailureReason=11302 Received Secure RADIUS request without a cts-pac-opaque cisco-av-pair attribute, Step=11001, Step=11017, Step=15012, Step=11302, NetworkDeviceGroups=Location#All Locations#NTN, NetworkDeviceGroups=Device Type#All Device Types#Wired, CPMSessionID=0a2a0740000056F453E324D0, Model Name=4503, Location=Location#All Locations#NTN, Device Type=Device Type#All Device Types#Wired, Resp
 onse={RadiusPacketType=Drop; },
+Aug  6 16:44:36 10.34.84.145 Aug  7 00:03:48 stage-pdp01 CISE_Failed_Attempts 0000024632 1 0 2014-08-07 00:03:48.375 -07:00 0000287084 5405 NOTICE Failed-Attempt: RADIUS Request dropped, ConfigVersionId=113, Device IP Address=172.23.91.132, Device Port=32769, DestinationIPAddress=10.34.84.145, DestinationPort=1812, Protocol=Radius, User-Name=test, NAS-IP-Address=192.168.30.11, Service-Type=NAS Prompt, NAS-Identifier=Cisco_1b:e0:84, AcsSessionID=stage-pdp01/196593288/19329, FailureReason=11007 Could not locate Network Device or AAA Client, Step=11001, Step=11017, Step=11007, Step=5405, 
+Aug  6 16:44:37 10.42.7.63 Aug  7 00:03:49 npf-sjca-pdp01 CISE_Failed_Attempts 0001970128 1 0 2014-08-07 00:03:49.893 -07:00 0098661643 5405 NOTICE Failed-Attempt: RADIUS Request dropped, ConfigVersionId=133, Device IP Address=10.56.72.126, Device Port=1645, DestinationIPAddress=10.42.7.63, DestinationPort=1812, RadiusPacketType=AccessRequest, UserName=#CTSREQUEST#, Protocol=Radius, RequestLatency=1, NetworkDeviceName=ntn01-11a-sw3, User-Name=#CTSREQUEST#, NAS-IP-Address=10.56.72.126, Service-Type=Outbound, AcsSessionID=npf-sjca-pdp01/195491152/2085307, SelectedAccessService=NDAC_SGT_Service, FailureReason=11302 Received Secure RADIUS request without a cts-pac-opaque cisco-av-pair attribute, Step=11001, Step=11017, Step=15012, Step=11302, NetworkDeviceGroups=Location#All Locations#NTN, NetworkDeviceGroups=Device Type#All Device Types#Wired, CPMSessionID=0a2a073f00005D4E53E324D5, Model Name=4503, Location=Location#All Locations#NTN, Device Type=Device Type#All Device Types#Wired, Res
 ponse={RadiusPacketType=Drop; },
+Aug  6 16:44:38 10.34.84.145 Aug  7 00:03:50 stage-pdp01 CISE_Failed_Attempts 0000024633 1 0 2014-08-07 00:03:50.379 -07:00 0000287088 5405 NOTICE Failed-Attempt: RADIUS Request dropped, ConfigVersionId=113, Device IP Address=172.23.91.132, Device Port=32769, DestinationIPAddress=10.34.84.145, DestinationPort=1812, Protocol=Radius, User-Name=test, NAS-IP-Address=192.168.30.11, Service-Type=NAS Prompt, NAS-Identifier=Cisco_1b:e0:84, AcsSessionID=stage-pdp01/196593288/19330, FailureReason=11007 Could not locate Network Device or AAA Client, Step=11001, Step=11017, Step=11007, Step=5405, 
+Aug  6 16:44:40 10.34.84.145 Aug  7 00:03:52 stage-pdp01 CISE_Failed_Attempts 0000024634 1 0 2014-08-07 00:03:52.379 -07:00 0000287092 5405 NOTICE Failed-Attempt: RADIUS Request dropped, ConfigVersionId=113, Device IP Address=172.23.91.132, Device Port=32769, DestinationIPAddress=10.34.84.145, DestinationPort=1812, Protocol=Radius, User-Name=test, NAS-IP-Address=192.168.30.11, Service-Type=NAS Prompt, NAS-Identifier=Cisco_1b:e0:84, AcsSessionID=stage-pdp01/196593288/19331, FailureReason=11007 Could not locate Network Device or AAA Client, Step=11001, Step=11017, Step=11007, Step=5405, 
+Aug  6 16:44:42 10.34.84.145 Aug  7 00:03:54 stage-pdp01 CISE_Failed_Attempts 0000024635 1 0 2014-08-07 00:03:54.387 -07:00 0000287096 5405 NOTICE Failed-Attempt: RADIUS Request dropped, ConfigVersionId=113, Device IP Address=172.23.91.132, Device Port=32769, DestinationIPAddress=10.34.84.145, DestinationPort=1812, Protocol=Radius, User-Name=test, NAS-IP-Address=192.168.30.11, Service-Type=NAS Prompt, NAS-Identifier=Cisco_1b:e0:84, AcsSessionID=stage-pdp01/196593288/19332, FailureReason=11007 Could not locate Network Device or AAA Client, Step=11001, Step=11017, Step=11007, Step=5405, 
+Aug  6 16:44:42 10.42.7.64 Aug  7 00:03:54 npf-sjca-pdp02 CISE_Failed_Attempts 0000370903 1 0 2014-08-07 00:03:54.924 -07:00 0011267670 5405 NOTICE Failed-Attempt: RADIUS Request dropped, ConfigVersionId=240, Device IP Address=10.56.72.126, Device Port=1645, DestinationIPAddress=10.42.7.64, DestinationPort=1812, RadiusPacketType=AccessRequest, UserName=#CTSREQUEST#, Protocol=Radius, RequestLatency=4, NetworkDeviceName=ntn01-11a-sw3, User-Name=#CTSREQUEST#, NAS-IP-Address=10.56.72.126, Service-Type=Outbound, AcsSessionID=npf-sjca-pdp02/195481465/270964, SelectedAccessService=NDAC_SGT_Service, FailureReason=11302 Received Secure RADIUS request without a cts-pac-opaque cisco-av-pair attribute, Step=11001, Step=11017, Step=15012, Step=11302, NetworkDeviceGroups=Location#All Locations#NTN, NetworkDeviceGroups=Device Type#All Device Types#Wired, CPMSessionID=0a2a0740000056F553E324DA, Model Name=4503, Location=Location#All Locations#NTN, Device Type=Device Type#All Device Types#Wired, Resp
 onse={RadiusPacketType=Drop; },
+Aug  6 16:44:44 10.34.84.145 Aug  7 00:03:56 stage-pdp01 CISE_Failed_Attempts 0000024636 1 0 2014-08-07 00:03:56.386 -07:00 0000287100 5405 NOTICE Failed-Attempt: RADIUS Request dropped, ConfigVersionId=113, Device IP Address=172.23.91.132, Device Port=32769, DestinationIPAddress=10.34.84.145, DestinationPort=1812, Protocol=Radius, User-Name=test, NAS-IP-Address=192.168.30.11, Service-Type=NAS Prompt, NAS-Identifier=Cisco_1b:e0:84, AcsSessionID=stage-pdp01/196593288/19333, FailureReason=11007 Could not locate Network Device or AAA Client, Step=11001, Step=11017, Step=11007, Step=5405, 
+Aug  6 16:44:46 10.34.84.145 Aug  7 00:03:58 stage-pdp01 CISE_Failed_Attempts 0000024637 1 0 2014-08-07 00:03:58.390 -07:00 0000287104 5405 NOTICE Failed-Attempt: RADIUS Request dropped, ConfigVersionId=113, Device IP Address=172.23.91.132, Device Port=32769, DestinationIPAddress=10.34.84.145, DestinationPort=1812, Protocol=Radius, User-Name=test, NAS-IP-Address=192.168.30.11, Service-Type=NAS Prompt, NAS-Identifier=Cisco_1b:e0:84, AcsSessionID=stage-pdp01/196593288/19334, FailureReason=11007 Could not locate Network Device or AAA Client, Step=11001, Step=11017, Step=11007, Step=5405, 
+Aug  6 16:44:47 10.42.7.63 Aug  7 00:03:59 npf-sjca-pdp01 CISE_Failed_Attempts 0001970140 1 0 2014-08-07 00:03:59.951 -07:00 0098662310 5405 NOTICE Failed-Attempt: RADIUS Request dropped, ConfigVersionId=133, Device IP Address=10.56.72.126, Device Port=1645, DestinationIPAddress=10.42.7.63, DestinationPort=1812, RadiusPacketType=AccessRequest, UserName=#CTSREQUEST#, Protocol=Radius, RequestLatency=1, NetworkDeviceName=ntn01-11a-sw3, User-Name=#CTSREQUEST#, NAS-IP-Address=10.56.72.126, Service-Type=Outbound, AcsSessionID=npf-sjca-pdp01/195491152/2085331, SelectedAccessService=NDAC_SGT_Service, FailureReason=11302 Received Secure RADIUS request without a cts-pac-opaque cisco-av-pair attribute, Step=11001, Step=11017, Step=15012, Step=11302, NetworkDeviceGroups=Location#All Locations#NTN, NetworkDeviceGroups=Device Type#All Device Types#Wired, CPMSessionID=0a2a073f00005D4F53E324DF, Model Name=4503, Location=Location#All Locations#NTN, Device Type=Device Type#All Device Types#Wired, Res
 ponse={RadiusPacketType=Drop; },
+Aug  6 16:44:48 10.42.7.64 Aug  7 00:04:00 npf-sjca-pdp02 CISE_Failed_Attempts 0000370905 1 0 2014-08-07 00:04:00.526 -07:00 0011267674 5435 NOTICE RADIUS: NAS conducted several failed authentications of the same scenario, ConfigVersionId=240, Device IP Address=10.56.72.127, Device Port=1646, DestinationIPAddress=10.42.7.64, DestinationPort=1813, Protocol=Radius, NetworkDeviceName=ntn01-11a-sw4, User-Name=host/salfi-pc.cisco.com, NAS-IP-Address=10.56.72.127, NAS-Port=50212, Service-Type=Framed, Framed-IP-Address=169.254.53.87, Class=CACS:0A38487F00000397BDA7BCAC:npf-sjca-pdp02/195481465/270957, Called-Station-ID=00-26-99-28-5E-BB, Calling-Station-ID=3C-97-0E-C3-F8-F1, Acct-Status-Type=Interim-Update, Acct-Delay-Time=0, Acct-Input-Octets=1458615, Acct-Output-Octets=3836368, Acct-Session-Id=00000560, Acct-Authentic=RADIUS, Acct-Session-Time=95, Acct-Input-Packets=4505, Acct-Output-Packets=5619, NAS-Port-Type=Ethernet, NAS-Port-Id=GigabitEthernet2/12, undefined-151=F54C88B0, cisco-av-p
 air=audit-session-id=0A38487F00000397BDA7BCAC, cisco-av-pair=connect-progress=Auth Open, AcsSessionID=npf-sjca-pdp02/195481465/270965, FailureReason=11038 RADIUS Accounting-Request header contains invalid Authenticator field, Step=11004, Step=11017, Step=11038, Step=5435, NetworkDeviceGroups=Location#All Locations#NTN, NetworkDeviceGroups=Device Type#All Device Types#Wired, CPMSessionID=0A38487F00000397BDA7BCAC, TotalFailedAttempts=2, TotalFailedTime=52, Model Name=4503, Location=Location#All Locations#NTN, Device Type=Device Type#All Device Types#Wired, 
+Aug  6 16:45:52 10.42.7.64 Aug  7 00:05:04 npf-sjca-pdp02 CISE_Failed_Attempts 0000370920 1 0 2014-08-07 00:05:04.969 -07:00 0011267987 5405 NOTICE Failed-Attempt: RADIUS Request dropped, ConfigVersionId=240, Device IP Address=10.56.72.126, Device Port=1645, DestinationIPAddress=10.42.7.64, DestinationPort=1812, RadiusPacketType=AccessRequest, UserName=#CTSREQUEST#, Protocol=Radius, RequestLatency=6, NetworkDeviceName=ntn01-11a-sw3, User-Name=#CTSREQUEST#, NAS-IP-Address=10.56.72.126, Service-Type=Outbound, AcsSessionID=npf-sjca-pdp02/195481465/270977, SelectedAccessService=NDAC_SGT_Service, FailureReason=11302 Received Secure RADIUS request without a cts-pac-opaque cisco-av-pair attribute, Step=11001, Step=11017, Step=15012, Step=11302, NetworkDeviceGroups=Location#All Locations#NTN, NetworkDeviceGroups=Device Type#All Device Types#Wired, CPMSessionID=0a2a0740000056F653E32520, Model Name=4503, Location=Location#All Locations#NTN, Device Type=Device Type#All Device Types#Wired, Resp
 onse={RadiusPacketType=Drop; },
+Aug  6 16:45:58 10.42.7.63 Aug  7 00:05:09 npf-sjca-pdp01 CISE_Failed_Attempts 0001970212 1 0 2014-08-07 00:05:09.998 -07:00 0098665518 5405 NOTICE Failed-Attempt: RADIUS Request dropped, ConfigVersionId=133, Device IP Address=10.56.72.126, Device Port=1645, DestinationIPAddress=10.42.7.63, DestinationPort=1812, RadiusPacketType=AccessRequest, UserName=#CTSREQUEST#, Protocol=Radius, RequestLatency=1, NetworkDeviceName=ntn01-11a-sw3, User-Name=#CTSREQUEST#, NAS-IP-Address=10.56.72.126, Service-Type=Outbound, AcsSessionID=npf-sjca-pdp01/195491152/2085460, SelectedAccessService=NDAC_SGT_Service, FailureReason=11302 Received Secure RADIUS request without a cts-pac-opaque cisco-av-pair attribute, Step=11001, Step=11017, Step=15012, Step=11302, NetworkDeviceGroups=Location#All Locations#NTN, NetworkDeviceGroups=Device Type#All Device Types#Wired, CPMSessionID=0a2a073f00005D5053E32525, Model Name=4503, Location=Location#All Locations#NTN, Device Type=Device Type#All Device Types#Wired, Res
 ponse={RadiusPacketType=Drop; },
+Aug  6 16:47:03 10.42.7.64 Aug  7 00:06:15 npf-sjca-pdp02 CISE_Failed_Attempts 0000370931 1 0 2014-08-07 00:06:15.016 -07:00 0011268196 5405 NOTICE Failed-Attempt: RADIUS Request dropped, ConfigVersionId=240, Device IP Address=10.56.72.126, Device Port=1645, DestinationIPAddress=10.42.7.64, DestinationPort=1812, RadiusPacketType=AccessRequest, UserName=#CTSREQUEST#, Protocol=Radius, RequestLatency=5, NetworkDeviceName=ntn01-11a-sw3, User-Name=#CTSREQUEST#, NAS-IP-Address=10.56.72.126, Service-Type=Outbound, AcsSessionID=npf-sjca-pdp02/195481465/270985, SelectedAccessService=NDAC_SGT_Service, FailureReason=11302 Received Secure RADIUS request without a cts-pac-opaque cisco-av-pair attribute, Step=11001, Step=11017, Step=15012, Step=11302, NetworkDeviceGroups=Location#All Locations#NTN, NetworkDeviceGroups=Device Type#All Device Types#Wired, CPMSessionID=0a2a0740000056F753E32567, Model Name=4503, Location=Location#All Locations#NTN, Device Type=Device Type#All Device Types#Wired, Resp
 onse={RadiusPacketType=Drop; },
+Aug  6 16:47:08 10.42.7.63 Aug  7 00:06:20 npf-sjca-pdp01 CISE_Failed_Attempts 0001970324 1 0 2014-08-07 00:06:20.055 -07:00 0098669942 5405 NOTICE Failed-Attempt: RADIUS Request dropped, ConfigVersionId=133, Device IP Address=10.56.72.126, Device Port=1645, DestinationIPAddress=10.42.7.63, DestinationPort=1812, RadiusPacketType=AccessRequest, UserName=#CTSREQUEST#, Protocol=Radius, RequestLatency=2, NetworkDeviceName=ntn01-11a-sw3, User-Name=#CTSREQUEST#, NAS-IP-Address=10.56.72.126, Service-Type=Outbound, AcsSessionID=npf-sjca-pdp01/195491152/2085599, SelectedAccessService=NDAC_SGT_Service, FailureReason=11302 Received Secure RADIUS request without a cts-pac-opaque cisco-av-pair attribute, Step=11001, Step=11017, Step=15012, Step=11302, NetworkDeviceGroups=Location#All Locations#NTN, NetworkDeviceGroups=Device Type#All Device Types#Wired, CPMSessionID=0a2a073f00005D5153E3256C, Model Name=4503, Location=Location#All Locations#NTN, Device Type=Device Type#All Device Types#Wired, Res
 ponse={RadiusPacketType=Drop; },
+Aug  6 16:47:13 10.42.7.64 Aug  7 00:06:25 npf-sjca-pdp02 CISE_Failed_Attempts 0000370934 1 0 2014-08-07 00:06:25.097 -07:00 0011268209 5405 NOTICE Failed-Attempt: RADIUS Request dropped, ConfigVersionId=240, Device IP Address=10.56.72.126, Device Port=1645, DestinationIPAddress=10.42.7.64, DestinationPort=1812, RadiusPacketType=AccessRequest, UserName=#CTSREQUEST#, Protocol=Radius, RequestLatency=5, NetworkDeviceName=ntn01-11a-sw3, User-Name=#CTSREQUEST#, NAS-IP-Address=10.56.72.126, Service-Type=Outbound, AcsSessionID=npf-sjca-pdp02/195481465/270987, SelectedAccessService=NDAC_SGT_Service, FailureReason=11302 Received Secure RADIUS request without a cts-pac-opaque cisco-av-pair attribute, Step=11001, Step=11017, Step=15012, Step=11302, NetworkDeviceGroups=Location#All Locations#NTN, NetworkDeviceGroups=Device Type#All Device Types#Wired, CPMSessionID=0a2a0740000056F853E32571, Model Name=4503, Location=Location#All Locations#NTN, Device Type=Device Type#All Device Types#Wired, Resp
 onse={RadiusPacketType=Drop; },
+Aug  6 16:47:18 10.42.7.63 Aug  7 00:06:30 npf-sjca-pdp01 CISE_Failed_Attempts 0001970335 1 0 2014-08-07 00:06:30.119 -07:00 0098670037 5405 NOTICE Failed-Attempt: RADIUS Request dropped, ConfigVersionId=133, Device IP Address=10.56.72.126, Device Port=1645, DestinationIPAddress=10.42.7.63, DestinationPort=1812, RadiusPacketType=AccessRequest, UserName=#CTSREQUEST#, Protocol=Radius, RequestLatency=2, NetworkDeviceName=ntn01-11a-sw3, User-Name=#CTSREQUEST#, NAS-IP-Address=10.56.72.126, Service-Type=Outbound, AcsSessionID=npf-sjca-pdp01/195491152/2085618, SelectedAccessService=NDAC_SGT_Service, FailureReason=11302 Received Secure RADIUS request without a cts-pac-opaque cisco-av-pair attribute, Step=11001, Step=11017, Step=15012, Step=11302, NetworkDeviceGroups=Location#All Locations#NTN, NetworkDeviceGroups=Device Type#All Device Types#Wired, CPMSessionID=0a2a073f00005D5253E32576, Model Name=4503, Location=Location#All Locations#NTN, Device Type=Device Type#All Device Types#Wired, Res
 ponse={RadiusPacketType=Drop; },
+Aug  6 16:47:48 10.34.84.145 Aug  7 00:07:00 stage-pdp01 CISE_Failed_Attempts 0000024649 1 0 2014-08-07 00:07:00.418 -07:00 0000287210 5405 NOTICE Failed-Attempt: RADIUS Request dropped, ConfigVersionId=113, Device IP Address=172.23.91.132, Device Port=32769, DestinationIPAddress=10.34.84.145, DestinationPort=1812, Protocol=Radius, User-Name=test, NAS-IP-Address=192.168.30.11, Service-Type=NAS Prompt, NAS-Identifier=Cisco_1b:e0:84, AcsSessionID=stage-pdp01/196593288/19342, FailureReason=11007 Could not locate Network Device or AAA Client, Step=11001, Step=11017, Step=11007, Step=5405, 
+Aug  6 16:47:50 10.34.84.145 Aug  7 00:07:02 stage-pdp01 CISE_Failed_Attempts 0000024650 1 0 2014-08-07 00:07:02.421 -07:00 0000287214 5405 NOTICE Failed-Attempt: RADIUS Request dropped, ConfigVersionId=113, Device IP Address=172.23.91.132, Device Port=32769, DestinationIPAddress=10.34.84.145, DestinationPort=1812, Protocol=Radius, User-Name=test, NAS-IP-Address=192.168.30.11, Service-Type=NAS Prompt, NAS-Identifier=Cisco_1b:e0:84, AcsSessionID=stage-pdp01/196593288/19343, FailureReason=11007 Could not locate Network Device or AAA Client, Step=11001, Step=11017, Step=11007, Step=5405, 
+Aug  6 16:47:52 10.34.84.145 Aug  7 00:07:04 stage-pdp01 CISE_Failed_Attempts 0000024651 1 0 2014-08-07 00:07:04.425 -07:00 0000287218 5405 NOTICE Failed-Attempt: RADIUS Request dropped, ConfigVersionId=113, Device IP Address=172.23.91.132, Device Port=32769, DestinationIPAddress=10.34.84.145, DestinationPort=1812, Protocol=Radius, User-Name=test, NAS-IP-Address=192.168.30.11, Service-Type=NAS Prompt, NAS-Identifier=Cisco_1b:e0:84, AcsSessionID=stage-pdp01/196593288/19344, FailureReason=11007 Could not locate Network Device or AAA Client, Step=11001, Step=11017, Step=11007, Step=5405, 
+Aug  6 16:47:54 10.34.84.145 Aug  7 00:07:06 stage-pdp01 CISE_Failed_Attempts 0000024652 1 0 2014-08-07 00:07:06.429 -07:00 0000287222 5405 NOTICE Failed-Attempt: RADIUS Request dropped, ConfigVersionId=113, Device IP Address=172.23.91.132, Device Port=32769, DestinationIPAddress=10.34.84.145, DestinationPort=1812, Protocol=Radius, User-Name=test, NAS-IP-Address=192.168.30.11, Service-Type=NAS Prompt, NAS-Identifier=Cisco_1b:e0:84, AcsSessionID=stage-pdp01/196593288/19345, FailureReason=11007 Could not locate Network Device or AAA Client, Step=11001, Step=11017, Step=11007, Step=5405, 
+Aug  6 16:47:56 10.34.84.145 Aug  7 00:07:08 stage-pdp01 CISE_Failed_Attempts 0000024653 1 0 2014-08-07 00:07:08.429 -07:00 0000287226 5405 NOTICE Failed-Attempt: RADIUS Request dropped, ConfigVersionId=113, Device IP Address=172.23.91.132, Device Port=32769, DestinationIPAddress=10.34.84.145, DestinationPort=1812, Protocol=Radius, User-Name=test, NAS-IP-Address=192.168.30.11, Service-Type=NAS Prompt, NAS-Identifier=Cisco_1b:e0:84, AcsSessionID=stage-pdp01/196593288/19346, FailureReason=11007 Could not locate Network Device or AAA Client, Step=11001, Step=11017, Step=11007, Step=5405, 
+Aug  6 16:47:58 10.34.84.145 Aug  7 00:07:10 stage-pdp01 CISE_Failed_Attempts 0000024654 1 0 2014-08-07 00:07:10.433 -07:00 0000287230 5405 NOTICE Failed-Attempt: RADIUS Request dropped, ConfigVersionId=113, Device IP Address=172.23.91.132, Device Port=32769, DestinationIPAddress=10.34.84.145, DestinationPort=1812, Protocol=Radius, User-Name=test, NAS-IP-Address=192.168.30.11, Service-Type=NAS Prompt, NAS-Identifier=Cisco_1b:e0:84, AcsSessionID=stage-pdp01/196593288/19347, FailureReason=11007 Could not locate Network Device or AAA Client, Step=11001, Step=11017, Step=11007, Step=5405, 
+Aug  6 16:48:23 10.42.7.64 Aug  7 00:07:35 npf-sjca-pdp02 CISE_Failed_Attempts 0000370955 1 0 2014-08-07 00:07:35.138 -07:00 0011268472 5405 NOTICE Failed-Attempt: RADIUS Request dropped, ConfigVersionId=240, Device IP Address=10.56.72.126, Device Port=1645, DestinationIPAddress=10.42.7.64, DestinationPort=1812, RadiusPacketType=AccessRequest, UserName=#CTSREQUEST#, Protocol=Radius, RequestLatency=5, NetworkDeviceName=ntn01-11a-sw3, User-Name=#CTSREQUEST#, NAS-IP-Address=10.56.72.126, Service-Type=Outbound, AcsSessionID=npf-sjca-pdp02/195481465/271001, SelectedAccessService=NDAC_SGT_Service, FailureReason=11302 Received Secure RADIUS request without a cts-pac-opaque cisco-av-pair attribute, Step=11001, Step=11017, Step=15012, Step=11302, NetworkDeviceGroups=Location#All Locations#NTN, NetworkDeviceGroups=Device Type#All Device Types#Wired, CPMSessionID=0a2a0740000056F953E325B7, Model Name=4503, Location=Location#All Locations#NTN, Device Type=Device Type#All Device Types#Wired, Resp
 onse={RadiusPacketType=Drop; },
+Aug  6 16:48:28 10.42.7.63 Aug  7 00:07:40 npf-sjca-pdp01 CISE_Failed_Attempts 0001970420 1 0 2014-08-07 00:07:40.178 -07:00 0098673462 5405 NOTICE Failed-Attempt: RADIUS Request dropped, ConfigVersionId=133, Device IP Address=10.56.72.126, Device Port=1645, DestinationIPAddress=10.42.7.63, DestinationPort=1812, RadiusPacketType=AccessRequest, UserName=#CTSREQUEST#, Protocol=Radius, RequestLatency=1, NetworkDeviceName=ntn01-11a-sw3, User-Name=#CTSREQUEST#, NAS-IP-Address=10.56.72.126, Service-Type=Outbound, AcsSessionID=npf-sjca-pdp01/195491152/2085757, SelectedAccessService=NDAC_SGT_Service, FailureReason=11302 Received Secure RADIUS request without a cts-pac-opaque cisco-av-pair attribute, Step=11001, Step=11017, Step=15012, Step=11302, NetworkDeviceGroups=Location#All Locations#NTN, NetworkDeviceGroups=Device Type#All Device Types#Wired, CPMSessionID=0a2a073f00005D5353E325BC, Model Name=4503, Location=Location#All Locations#NTN, Device Type=Device Type#All Device Types#Wired, Res
 ponse={RadiusPacketType=Drop; },
+Aug  6 16:49:33 10.42.7.64 Aug  7 00:08:45 npf-sjca-pdp02 CISE_Failed_Attempts 0000370984 1 0 2014-08-07 00:08:45.219 -07:00 0011269071 5405 NOTICE Failed-Attempt: RADIUS Request dropped, ConfigVersionId=240, Device IP Address=10.56.72.126, Device Port=1645, DestinationIPAddress=10.42.7.64, DestinationPort=1812, RadiusPacketType=AccessRequest, UserName=#CTSREQUEST#, Protocol=Radius, RequestLatency=5, NetworkDeviceName=ntn01-11a-sw3, User-Name=#CTSREQUEST#, NAS-IP-Address=10.56.72.126, Service-Type=Outbound, AcsSessionID=npf-sjca-pdp02/195481465/271016, SelectedAccessService=NDAC_SGT_Service, FailureReason=11302 Received Secure RADIUS request without a cts-pac-opaque cisco-av-pair attribute, Step=11001, Step=11017, Step=15012, Step=11302, NetworkDeviceGroups=Location#All Locations#NTN, NetworkDeviceGroups=Device Type#All Device Types#Wired, CPMSessionID=0a2a0740000056FB53E325FD, Model Name=4503, Location=Location#All Locations#NTN, Device Type=Device Type#All Device Types#Wired, Resp
 onse={RadiusPacketType=Drop; },
+Aug  6 16:49:38 10.42.7.63 Aug  7 00:08:50 npf-sjca-pdp01 CISE_Failed_Attempts 0001970519 1 0 2014-08-07 00:08:50.259 -07:00 0098677825 5405 NOTICE Failed-Attempt: RADIUS Request dropped, ConfigVersionId=133, Device IP Address=10.56.72.126, Device Port=1645, DestinationIPAddress=10.42.7.63, DestinationPort=1812, RadiusPacketType=AccessRequest, UserName=#CTSREQUEST#, Protocol=Radius, RequestLatency=2, NetworkDeviceName=ntn01-11a-sw3, User-Name=#CTSREQUEST#, NAS-IP-Address=10.56.72.126, Service-Type=Outbound, AcsSessionID=npf-sjca-pdp01/195491152/2085892, SelectedAccessService=NDAC_SGT_Service, FailureReason=11302 Received Secure RADIUS request without a cts-pac-opaque cisco-av-pair attribute, Step=11001, Step=11017, Step=15012, Step=11302, NetworkDeviceGroups=Location#All Locations#NTN, NetworkDeviceGroups=Device Type#All Device Types#Wired, CPMSessionID=0a2a073f00005D5453E32602, Model Name=4503, Location=Location#All Locations#NTN, Device Type=Device Type#All Device Types#Wired, Res
 ponse={RadiusPacketType=Drop; },
+Aug  6 16:49:43 10.42.7.64 Aug  7 00:08:55 npf-sjca-pdp02 CISE_Failed_Attempts 0000370986 1 0 2014-08-07 00:08:55.298 -07:00 0011269076 5405 NOTICE Failed-Attempt: RADIUS Request dropped, ConfigVersionId=240, Device IP Address=10.56.72.126, Device Port=1645, DestinationIPAddress=10.42.7.64, DestinationPort=1812, RadiusPacketType=AccessRequest, UserName=#CTSREQUEST#, Protocol=Radius, RequestLatency=5, NetworkDeviceName=ntn01-11a-sw3, User-Name=#CTSREQUEST#, NAS-IP-Address=10.56.72.126, Service-Type=Outbound, AcsSessionID=npf-sjca-pdp02/195481465/271017, SelectedAccessService=NDAC_SGT_Service, FailureReason=11302 Received Secure RADIUS request without a cts-pac-opaque cisco-av-pair attribute, Step=11001, Step=11017, Step=15012, Step=11302, NetworkDeviceGroups=Location#All Locations#NTN, NetworkDeviceGroups=Device Type#All Device Types#Wired, CPMSessionID=0a2a0740000056FC53E32607, Model Name=4503, Location=Location#All Locations#NTN, Device Type=Device Type#All Device Types#Wired, Resp
 onse={RadiusPacketType=Drop; },
+Aug  6 16:49:48 10.42.7.63 Aug  7 00:09:00 npf-sjca-pdp01 CISE_Failed_Attempts 0001970524 1 0 2014-08-07 00:09:00.330 -07:00 0098678019 5405 NOTICE Failed-Attempt: RADIUS Request dropped, ConfigVersionId=133, Device IP Address=10.56.72.126, Device Port=1645, DestinationIPAddress=10.42.7.63, DestinationPort=1812, RadiusPacketType=AccessRequest, UserName=#CTSREQUEST#, Protocol=Radius, RequestLatency=2, NetworkDeviceName=ntn01-11a-sw3, User-Name=#CTSREQUEST#, NAS-IP-Address=10.56.72.126, Service-Type=Outbound, AcsSessionID=npf-sjca-pdp01/195491152/2085909, SelectedAccessService=NDAC_SGT_Service, FailureReason=11302 Received Secure RADIUS request without a cts-pac-opaque cisco-av-pair attribute, Step=11001, Step=11017, Step=15012, Step=11302, NetworkDeviceGroups=Location#All Locations#NTN, NetworkDeviceGroups=Device Type#All Device Types#Wired, CPMSessionID=0a2a073f00005D5553E3260C, Model Name=4503, Location=Location#All Locations#NTN, Device Type=Device Type#All Device Types#Wired, Res
 ponse={RadiusPacketType=Drop; },
+Aug  6 16:50:53 10.42.7.64 Aug  7 00:10:05 npf-sjca-pdp02 CISE_Failed_Attempts 0000370999 1 0 2014-08-07 00:10:05.339 -07:00 0011269371 5405 NOTICE Failed-Attempt: RADIUS Request dropped, ConfigVersionId=240, Device IP Address=10.56.72.126, Device Port=1645, DestinationIPAddress=10.42.7.64, DestinationPort=1812, RadiusPacketType=AccessRequest, UserName=#CTSREQUEST#, Protocol=Radius, RequestLatency=5, NetworkDeviceName=ntn01-11a-sw3, User-Name=#CTSREQUEST#, NAS-IP-Address=10.56.72.126, Service-Type=Outbound, AcsSessionID=npf-sjca-pdp02/195481465/271027, SelectedAccessService=NDAC_SGT_Service, FailureReason=11302 Received Secure RADIUS request without a cts-pac-opaque cisco-av-pair attribute, Step=11001, Step=11017, Step=15012, Step=11302, NetworkDeviceGroups=Location#All Locations#NTN, NetworkDeviceGroups=Device Type#All Device Types#Wired, CPMSessionID=0a2a0740000056FD53E3264D, Model Name=4503, Location=Location#All Locations#NTN, Device Type=Device Type#All Device Types#Wired, Resp
 onse={RadiusPacketType=Drop; },
+Aug  6 16:50:58 10.42.7.63 Aug  7 00:10:10 npf-sjca-pdp01 CISE_Failed_Attempts 0001970625 1 0 2014-08-07 00:10:10.388 -07:00 0098682297 5405 NOTICE Failed-Attempt: RADIUS Request dropped, ConfigVersionId=133, Device IP Address=10.56.72.126, Device Port=1645, DestinationIPAddress=10.42.7.63, DestinationPort=1812, RadiusPacketType=AccessRequest, UserName=#CTSREQUEST#, Protocol=Radius, RequestLatency=2, NetworkDeviceName=ntn01-11a-sw3, User-Name=#CTSREQUEST#, NAS-IP-Address=10.56.72.126, Service-Type=Outbound, AcsSessionID=npf-sjca-pdp01/195491152/2086061, SelectedAccessService=NDAC_SGT_Service, FailureReason=11302 Received Secure RADIUS request without a cts-pac-opaque cisco-av-pair attribute, Step=11001, Step=11017, Step=15012, Step=11302, NetworkDeviceGroups=Location#All Locations#NTN, NetworkDeviceGroups=Device Type#All Device Types#Wired, CPMSessionID=0a2a073f00005D5653E32652, Model Name=4503, Location=Location#All Locations#NTN, Device Type=Device Type#All Device Types#Wired, Res
 ponse={RadiusPacketType=Drop; },
+Aug  6 16:51:00 10.34.84.145 Aug  7 00:10:12 stage-pdp01 CISE_Failed_Attempts 0000024661 1 0 2014-08-07 00:10:12.492 -07:00 0000287258 5405 NOTICE Failed-Attempt: RADIUS Request dropped, ConfigVersionId=113, Device IP Address=172.23.91.132, Device Port=32769, DestinationIPAddress=10.34.84.145, DestinationPort=1812, Protocol=Radius, User-Name=test, NAS-IP-Address=192.168.30.11, Service-Type=NAS Prompt, NAS-Identifier=Cisco_1b:e0:84, AcsSessionID=stage-pdp01/196593288/19354, FailureReason=11007 Could not locate Network Device or AAA Client, Step=11001, Step=11017, Step=11007, Step=5405, 
+Aug  6 16:51:02 10.34.84.145 Aug  7 00:10:14 stage-pdp01 CISE_Failed_Attempts 0000024662 1 0 2014-08-07 00:10:14.496 -07:00 0000287262 5405 NOTICE Failed-Attempt: RADIUS Request dropped, ConfigVersionId=113, Device IP Address=172.23.91.132, Device Port=32769, DestinationIPAddress=10.34.84.145, DestinationPort=1812, Protocol=Radius, User-Name=test, NAS-IP-Address=192.168.30.11, Service-Type=NAS Prompt, NAS-Identifier=Cisco_1b:e0:84, AcsSessionID=stage-pdp01/196593288/19355, FailureReason=11007 Could not locate Network Device or AAA Client, Step=11001, Step=11017, Step=11007, Step=5405, 
+Aug  6 16:51:04 10.34.84.145 Aug  7 00:10:16 stage-pdp01 CISE_Failed_Attempts 0000024663 1 0 2014-08-07 00:10:16.496 -07:00 0000287266 5405 NOTICE Failed-Attempt: RADIUS Request dropped, ConfigVersionId=113, Device IP Address=172.23.91.132, Device Port=32769, DestinationIPAddress=10.34.84.145, DestinationPort=1812, Protocol=Radius, User-Name=test, NAS-IP-Address=192.168.30.11, Service-Type=NAS Prompt, NAS-Identifier=Cisco_1b:e0:84, AcsSessionID=stage-pdp01/196593288/19356, FailureReason=11007 Could not locate Network Device or AAA Client, Step=11001, Step=11017, Step=11007, Step=5405, 
+Aug  6 16:51:06 10.34.84.145 Aug  7 00:10:18 stage-pdp01 CISE_Failed_Attempts 0000024664 1 0 2014-08-07 00:10:18.500 -07:00 0000287270 5405 NOTICE Failed-Attempt: RADIUS Request dropped, ConfigVersionId=113, Device IP Address=172.23.91.132, Device Port=32769, DestinationIPAddress=10.34.84.145, DestinationPort=1812, Protocol=Radius, User-Name=test, NAS-IP-Address=192.168.30.11, Service-Type=NAS Prompt, NAS-Identifier=Cisco_1b:e0:84, AcsSessionID=stage-pdp01/196593288/19357, FailureReason=11007 Could not locate Network Device or AAA Client, Step=11001, Step=11017, Step=11007, Step=5405, 
+Aug  6 16:51:08 10.34.84.145 Aug  7 00:10:20 stage-pdp01 CISE_Failed_Attempts 0000024665 1 0 2014-08-07 00:10:20.504 -07:00 0000287274 5405 NOTICE Failed-Attempt: RADIUS Request dropped, ConfigVersionId=113, Device IP Address=172.23.91.132, Device Port=32769, DestinationIPAddress=10.34.84.145, DestinationPort=1812, Protocol=Radius, User-Name=test, NAS-IP-Address=192.168.30.11, Service-Type=NAS Prompt, NAS-Identifier=Cisco_1b:e0:84, AcsSessionID=stage-pdp01/196593288/19358, FailureReason=11007 Could not locate Network Device or AAA Client, Step=11001, Step=11017, Step=11007, Step=5405, 
+Aug  6 16:51:10 10.34.84.145 Aug  7 00:10:22 stage-pdp01 CISE_Failed_Attempts 0000024667 1 0 2014-08-07 00:10:22.507 -07:00 0000287279 5405 NOTICE Failed-Attempt: RADIUS Request dropped, ConfigVersionId=113, Device IP Address=172.23.91.132, Device Port=32769, DestinationIPAddress=10.34.84.145, DestinationPort=1812, Protocol=Radius, User-Name=test, NAS-IP-Address=192.168.30.11, Service-Type=NAS Prompt, NAS-Identifier=Cisco_1b:e0:84, AcsSessionID=stage-pdp01/196593288/19359, FailureReason=11007 Could not locate Network Device or AAA Client, Step=11001, Step=11017, Step=11007, Step=5405, 
+Aug  6 16:52:03 10.42.7.64 Aug  7 00:11:15 npf-sjca-pdp02 CISE_Failed_Attempts 0000371005 1 0 2014-08-07 00:11:15.432 -07:00 0011269421 5405 NOTICE Failed-Attempt: RADIUS Request dropped, ConfigVersionId=240, Device IP Address=10.56.72.126, Device Port=1645, DestinationIPAddress=10.42.7.64, DestinationPort=1812, RadiusPacketType=AccessRequest, UserName=#CTSREQUEST#, Protocol=Radius, RequestLatency=4, NetworkDeviceName=ntn01-11a-sw3, User-Name=#CTSREQUEST#, NAS-IP-Address=10.56.72.126, Service-Type=Outbound, AcsSessionID=npf-sjca-pdp02/195481465/271031, SelectedAccessService=NDAC_SGT_Service, FailureReason=11302 Received Secure RADIUS request without a cts-pac-opaque cisco-av-pair attribute, Step=11001, Step=11017, Step=15012, Step=11302, NetworkDeviceGroups=Location#All Locations#NTN, NetworkDeviceGroups=Device Type#All Device Types#Wired, CPMSessionID=0a2a0740000056FE53E32693, Model Name=4503, Location=Location#All Locations#NTN, Device Type=Device Type#All Device Types#Wired, Resp
 onse={RadiusPacketType=Drop; },
+Aug  6 16:52:08 10.42.7.63 Aug  7 00:11:20 npf-sjca-pdp01 CISE_Failed_Attempts 0001970691 1 0 2014-08-07 00:11:20.468 -07:00 0098685176 5405 NOTICE Failed-Attempt: RADIUS Request dropped, ConfigVersionId=133, Device IP Address=10.56.72.126, Device Port=1645, DestinationIPAddress=10.42.7.63, DestinationPort=1812, RadiusPacketType=AccessRequest, UserName=#CTSREQUEST#, Protocol=Radius, RequestLatency=2, NetworkDeviceName=ntn01-11a-sw3, User-Name=#CTSREQUEST#, NAS-IP-Address=10.56.72.126, Service-Type=Outbound, AcsSessionID=npf-sjca-pdp01/195491152/2086181, SelectedAccessService=NDAC_SGT_Service, FailureReason=11302 Received Secure RADIUS request without a cts-pac-opaque cisco-av-pair attribute, Step=11001, Step=11017, Step=15012, Step=11302, NetworkDeviceGroups=Location#All Locations#NTN, NetworkDeviceGroups=Device Type#All Device Types#Wired, CPMSessionID=0a2a073f00005D5753E32698, Model Name=4503, Location=Location#All Locations#NTN, Device Type=Device Type#All Device Types#Wired, Res
 ponse={RadiusPacketType=Drop; },
+Aug  6 16:52:13 10.42.7.64 Aug  7 00:11:25 npf-sjca-pdp02 CISE_Failed_Attempts 0000371007 1 0 2014-08-07 00:11:25.515 -07:00 0011269426 5405 NOTICE Failed-Attempt: RADIUS Request dropped, ConfigVersionId=240, Device IP Address=10.56.72.126, Device Port=1645, DestinationIPAddress=10.42.7.64, DestinationPort=1812, RadiusPacketType=AccessRequest, UserName=#CTSREQUEST#, Protocol=Radius, RequestLatency=5, NetworkDeviceName=ntn01-11a-sw3, User-Name=#CTSREQUEST#, NAS-IP-Address=10.56.72.126, Service-Type=Outbound, AcsSessionID=npf-sjca-pdp02/195481465/271032, SelectedAccessService=NDAC_SGT_Service, FailureReason=11302 Received Secure RADIUS request without a cts-pac-opaque cisco-av-pair attribute, Step=11001, Step=11017, Step=15012, Step=11302, NetworkDeviceGroups=Location#All Locations#NTN, NetworkDeviceGroups=Device Type#All Device Types#Wired, CPMSessionID=0a2a0740000056FF53E3269D, Model Name=4503, Location=Location#All Locations#NTN, Device Type=Device Type#All Device Types#Wired, Resp
 onse={RadiusPacketType=Drop; },
+Aug  6 16:52:18 10.42.7.63 Aug  7 00:11:30 npf-sjca-pdp01 CISE_Failed_Attempts 0001970708 1 0 2014-08-07 00:11:30.551 -07:00 0098685669 5405 NOTICE Failed-Attempt: RADIUS Request dropped, ConfigVersionId=133, Device IP Address=10.56.72.126, Device Port=1645, DestinationIPAddress=10.42.7.63, DestinationPort=1812, RadiusPacketType=AccessRequest, UserName=#CTSREQUEST#, Protocol=Radius, RequestLatency=8, NetworkDeviceName=ntn01-11a-sw3, User-Name=#CTSREQUEST#, NAS-IP-Address=10.56.72.126, Service-Type=Outbound, AcsSessionID=npf-sjca-pdp01/195491152/2086202, SelectedAccessService=NDAC_SGT_Service, FailureReason=11302 Received Secure RADIUS request without a cts-pac-opaque cisco-av-pair attribute, Step=11001, Step=11017, Step=15012, Step=11302, NetworkDeviceGroups=Location#All Locations#NTN, NetworkDeviceGroups=Device Type#All Device Types#Wired, CPMSessionID=0a2a073f00005D5853E326A2, Model Name=4503, Location=Location#All Locations#NTN, Device Type=Device Type#All Device Types#Wired, Res
 ponse={RadiusPacketType=Drop; },
+Aug  6 16:53:23 10.42.7.64 Aug  7 00:12:35 npf-sjca-pdp02 CISE_Failed_Attempts 0000371016 1 0 2014-08-07 00:12:35.547 -07:00 0011269586 5405 NOTICE Failed-Attempt: RADIUS Request dropped, ConfigVersionId=240, Device IP Address=10.56.72.126, Device Port=1645, DestinationIPAddress=10.42.7.64, DestinationPort=1812, RadiusPacketType=AccessRequest, UserName=#CTSREQUEST#, Protocol=Radius, RequestLatency=5, NetworkDeviceName=ntn01-11a-sw3, User-Name=#CTSREQUEST#, NAS-IP-Address=10.56.72.126, Service-Type=Outbound, AcsSessionID=npf-sjca-pdp02/195481465/271040, SelectedAccessService=NDAC_SGT_Service, FailureReason=11302 Received Secure RADIUS request without a cts-pac-opaque cisco-av-pair attribute, Step=11001, Step=11017, Step=15012, Step=11302, NetworkDeviceGroups=Location#All Locations#NTN, NetworkDeviceGroups=Device Type#All Device Types#Wired, CPMSessionID=0a2a07400000570053E326E3, Model Name=4503, Location=Location#All Locations#NTN, Device Type=Device Type#All Device Types#Wired, Resp
 onse={RadiusPacketType=Drop; },
+Aug  6 16:53:28 10.42.7.63 Aug  7 00:12:40 npf-sjca-pdp01 CISE_Failed_Attempts 0001970802 1 0 2014-08-07 00:12:40.596 -07:00 0098689883 5405 NOTICE Failed-Attempt: RADIUS Request dropped, ConfigVersionId=133, Device IP Address=10.56.72.126, Device Port=1645, DestinationIPAddress=10.42.7.63, DestinationPort=1812, RadiusPacketType=AccessRequest, UserName=#CTSREQUEST#, Protocol=Radius, RequestLatency=2, NetworkDeviceName=ntn01-11a-sw3, User-Name=#CTSREQUEST#, NAS-IP-Address=10.56.72.126, Service-Type=Outbound, AcsSessionID=npf-sjca-pdp01/195491152/2086334, SelectedAccessService=NDAC_SGT_Service, FailureReason=11302 Received Secure RADIUS request without a cts-pac-opaque cisco-av-pair attribute, Step=11001, Step=11017, Step=15012, Step=11302, NetworkDeviceGroups=Location#All Locations#NTN, NetworkDeviceGroups=Device Type#All Device Types#Wired, CPMSessionID=0a2a073f00005D5953E326E8, Model Name=4503, Location=Location#All Locations#NTN, Device Type=Device Type#All Device Types#Wired, Res
 ponse={RadiusPacketType=Drop; },
+Aug  6 16:54:12 10.34.84.145 Aug  7 00:13:24 stage-pdp01 CISE_Failed_Attempts 0000024680 1 0 2014-08-07 00:13:24.527 -07:00 0000287388 5405 NOTICE Failed-Attempt: RADIUS Request dropped, ConfigVersionId=113, Device IP Address=172.23.91.132, Device Port=32769, DestinationIPAddress=10.34.84.145, DestinationPort=1812, Protocol=Radius, User-Name=test, NAS-IP-Address=192.168.30.11, Service-Type=NAS Prompt, NAS-Identifier=Cisco_1b:e0:84, AcsSessionID=stage-pdp01/196593288/19368, FailureReason=11007 Could not locate Network Device or AAA Client, Step=11001, Step=11017, Step=11007, Step=5405, 
+Aug  6 16:54:14 10.34.84.145 Aug  7 00:13:26 stage-pdp01 CISE_Failed_Attempts 0000024681 1 0 2014-08-07 00:13:26.531 -07:00 0000287392 5405 NOTICE Failed-Attempt: RADIUS Request dropped, ConfigVersionId=113, Device IP Address=172.23.91.132, Device Port=32769, DestinationIPAddress=10.34.84.145, DestinationPort=1812, Protocol=Radius, User-Name=test, NAS-IP-Address=192.168.30.11, Service-Type=NAS Prompt, NAS-Identifier=Cisco_1b:e0:84, AcsSessionID=stage-pdp01/196593288/19369, FailureReason=11007 Could not locate Network Device or AAA Client, Step=11001, Step=11017, Step=11007, Step=5405, 
+Aug  6 16:54:16 10.34.84.145 Aug  7 00:13:28 stage-pdp01 CISE_Failed_Attempts 0000024682 1 0 2014-08-07 00:13:28.534 -07:00 0000287396 5405 NOTICE Failed-Attempt: RADIUS Request dropped, ConfigVersionId=113, Device IP Address=172.23.91.132, Device Port=32769, DestinationIPAddress=10.34.84.145, DestinationPort=1812, Protocol=Radius, User-Name=test, NAS-IP-Address=192.168.30.11, Service-Type=NAS Prompt, NAS-Identifier=Cisco_1b:e0:84, AcsSessionID=stage-pdp01/196593288/19370, FailureReason=11007 Could not locate Network Device or AAA Client, Step=11001, Step=11017, Step=11007, Step=5405, 
+Aug  6 16:54:18 10.34.84.145 Aug  7 00:13:30 stage-pdp01 CISE_Failed_Attempts 0000024683 1 0 2014-08-07 00:13:30.538 -07:00 0000287400 5405 NOTICE Failed-Attempt: RADIUS Request dropped, ConfigVersionId=113, Device IP Address=172.23.91.132, Device Port=32769, DestinationIPAddress=10.34.84.145, DestinationPort=1812, Protocol=Radius, User-Name=test, NAS-IP-Address=192.168.30.11, Service-Type=NAS Prompt, NAS-Identifier=Cisco_1b:e0:84, AcsSessionID=stage-pdp01/196593288/19371, FailureReason=11007 Could not locate Network Device or AAA Client, Step=11001, Step=11017, Step=11007, Step=5405, 
+Aug  6 16:54:20 10.34.84.145 Aug  7 00:13:32 stage-pdp01 CISE_Failed_Attempts 0000024684 1 0 2014-08-07 00:13:32.538 -07:00 0000287404 5405 NOTICE Failed-Attempt: RADIUS Request dropped, ConfigVersionId=113, Device IP Address=172.23.91.132, Device Port=32769, DestinationIPAddress=10.34.84.145, DestinationPort=1812, Protocol=Radius, User-Name=test, NAS-IP-Address=192.168.30.11, Service-Type=NAS Prompt, NAS-Identifier=Cisco_1b:e0:84, AcsSessionID=stage-pdp01/196593288/19372, FailureReason=11007 Could not locate Network Device or AAA Client, Step=11001, Step=11017, Step=11007, Step=5405, 
+Aug  6 16:54:22 10.34.84.145 Aug  7 00:13:34 stage-pdp01 CISE_Failed_Attempts 0000024685 1 0 2014-08-07 00:13:34.542 -07:00 0000287408 5405 NOTICE Failed-Attempt: RADIUS Request dropped, ConfigVersionId=113, Device IP Address=172.23.91.132, Device Port=32769, DestinationIPAddress=10.34.84.145, DestinationPort=1812, Protocol=Radius, User-Name=test, NAS-IP-Address=192.168.30.11, Service-Type=NAS Prompt, NAS-Identifier=Cisco_1b:e0:84, AcsSessionID=stage-pdp01/196593288/19373, FailureReason=11007 Could not locate Network Device or AAA Client, Step=11001, Step=11017, Step=11007, Step=5405, 
+Aug  6 16:54:33 10.42.7.64 Aug  7 00:13:45 npf-sjca-pdp02 CISE_Failed_Attempts 0000371020 1 0 2014-08-07 00:13:45.628 -07:00 0011269631 5405 NOTICE Failed-Attempt: RADIUS Request dropped, ConfigVersionId=240, Device IP Address=10.56.72.126, Device Port=1645, DestinationIPAddress=10.42.7.64, DestinationPort=1812, RadiusPacketType=AccessRequest, UserName=#CTSREQUEST#, Protocol=Radius, RequestLatency=5, NetworkDeviceName=ntn01-11a-sw3, User-Name=#CTSREQUEST#, NAS-IP-Address=10.56.72.126, Service-Type=Outbound, AcsSessionID=npf-sjca-pdp02/195481465/271044, SelectedAccessService=NDAC_SGT_Service, FailureReason=11302 Received Secure RADIUS request without a cts-pac-opaque cisco-av-pair attribute, Step=11001, Step=11017, Step=15012, Step=11302, NetworkDeviceGroups=Location#All Locations#NTN, NetworkDeviceGroups=Device Type#All Device Types#Wired, CPMSessionID=0a2a07400000570153E32729, Model Name=4503, Location=Location#All Locations#NTN, Device Type=Device Type#All Device Types#Wired, Resp
 onse={RadiusPacketType=Drop; },
+Aug  6 16:54:38 10.42.7.63 Aug  7 00:13:50 npf-sjca-pdp01 CISE_Failed_Attempts 0001970913 1 0 2014-08-07 00:13:50.668 -07:00 0098695334 5405 NOTICE Failed-Attempt: RADIUS Request dropped, ConfigVersionId=133, Device IP Address=10.56.72.126, Device Port=1645, DestinationIPAddress=10.42.7.63, DestinationPort=1812, RadiusPacketType=AccessRequest, UserName=#CTSREQUEST#, Protocol=Radius, RequestLatency=2, NetworkDeviceName=ntn01-11a-sw3, User-Name=#CTSREQUEST#, NAS-IP-Address=10.56.72.126, Service-Type=Outbound, AcsSessionID=npf-sjca-pdp01/195491152/2086486, SelectedAccessService=NDAC_SGT_Service, FailureReason=11302 Received Secure RADIUS request without a cts-pac-opaque cisco-av-pair attribute, Step=11001, Step=11017, Step=15012, Step=11302, NetworkDeviceGroups=Location#All Locations#NTN, NetworkDeviceGroups=Device Type#All Device Types#Wired, CPMSessionID=0a2a073f00005D5A53E3272E, Model Name=4503, Location=Location#All Locations#NTN, Device Type=Device Type#All Device Types#Wired, Res
 ponse={RadiusPacketType=Drop; },
+Aug  6 16:54:43 10.42.7.64 Aug  7 00:13:55 npf-sjca-pdp02 CISE_Failed_Attempts 0000371025 1 0 2014-08-07 00:13:55.694 -07:00 0011269740 5405 NOTICE Failed-Attempt: RADIUS Request dropped, ConfigVersionId=240, Device IP Address=10.56.72.126, Device Port=1645, DestinationIPAddress=10.42.7.64, DestinationPort=1812, RadiusPacketType=AccessRequest, UserName=#CTSREQUEST#, Protocol=Radius, RequestLatency=7, NetworkDeviceName=ntn01-11a-sw3, User-Name=#CTSREQUEST#, NAS-IP-Address=10.56.72.126, Service-Type=Outbound, AcsSessionID=npf-sjca-pdp02/195481465/271048, SelectedAccessService=NDAC_SGT_Service, FailureReason=11302 Received Secure RADIUS request without a cts-pac-opaque cisco-av-pair attribute, Step=11001, Step=11017, Step=15012, Step=11302, NetworkDeviceGroups=Location#All Locations#NTN, NetworkDeviceGroups=Device Type#All Device Types#Wired, CPMSessionID=0a2a07400000570253E32733, Model Name=4503, Location=Location#All Locations#NTN, Device Type=Device Type#All Device Types#Wired, Resp
 onse={RadiusPacketType=Drop; },
+Aug  6 16:54:48 10.42.7.63 Aug  7 00:14:00 npf-sjca-pdp01 CISE_Failed_Attempts 0001970924 1 0 2014-08-07 00:14:00.705 -07:00 0098695591 5405 NOTICE Failed-Attempt: RADIUS Request dropped, ConfigVersionId=133, Device IP Address=10.56.72.126, Device Port=1645, DestinationIPAddress=10.42.7.63, DestinationPort=1812, RadiusPacketType=AccessRequest, UserName=#CTSREQUEST#, Protocol=Radius, RequestLatency=1, NetworkDeviceName=ntn01-11a-sw3, User-Name=#CTSREQUEST#, NAS-IP-Address=10.56.72.126, Service-Type=Outbound, AcsSessionID=npf-sjca-pdp01/195491152/2086505, SelectedAccessService=NDAC_SGT_Service, FailureReason=11302 Received Secure RADIUS request without a cts-pac-opaque cisco-av-pair attribute, Step=11001, Step=11017, Step=15012, Step=11302, NetworkDeviceGroups=Location#All Locations#NTN, NetworkDeviceGroups=Device Type#All Device Types#Wired, CPMSessionID=0a2a073f00005D5B53E32738, Model Name=4503, Location=Location#All Locations#NTN, Device Type=Device Type#All Device Types#Wired, Res
 ponse={RadiusPacketType=Drop; },
+Aug  6 16:55:53 10.42.7.64 Aug  7 00:15:05 npf-sjca-pdp02 CISE_Failed_Attempts 0000371036 1 0 2014-08-07 00:15:05.742 -07:00 0011270054 5405 NOTICE Failed-Attempt: RADIUS Request dropped, ConfigVersionId=240, Device IP Address=10.56.72.126, Device Port=1645, DestinationIPAddress=10.42.7.64, DestinationPort=1812, RadiusPacketType=AccessRequest, UserName=#CTSREQUEST#, Protocol=Radius, RequestLatency=6, NetworkDeviceName=ntn01-11a-sw3, User-Name=#CTSREQUEST#, NAS-IP-Address=10.56.72.126, Service-Type=Outbound, AcsSessionID=npf-sjca-pdp02/195481465/271057, SelectedAccessService=NDAC_SGT_Service, FailureReason=11302 Received Secure RADIUS request without a cts-pac-opaque cisco-av-pair attribute, Step=11001, Step=11017, Step=15012, Step=11302, NetworkDeviceGroups=Location#All Locations#NTN, NetworkDeviceGroups=Device Type#All Device Types#Wired, CPMSessionID=0a2a07400000570353E32779, Model Name=4503, Location=Location#All Locations#NTN, Device Type=Device Type#All Device Types#Wired, Resp
 onse={RadiusPacketType=Drop; },
+Aug  6 16:55:58 10.42.7.63 Aug  7 00:15:10 npf-sjca-pdp01 CISE_Failed_Attempts 0001970997 1 0 2014-08-07 00:15:10.772 -07:00 0098698954 5405 NOTICE Failed-Attempt: RADIUS Request dropped, ConfigVersionId=133, Device IP Address=10.56.72.126, Device Port=1645, DestinationIPAddress=10.42.7.63, DestinationPort=1812, RadiusPacketType=AccessRequest, UserName=#CTSREQUEST#, Protocol=Radius, RequestLatency=1, NetworkDeviceName=ntn01-11a-sw3, User-Name=#CTSREQUEST#, NAS-IP-Address=10.56.72.126, Service-Type=Outbound, AcsSessionID=npf-sjca-pdp01/195491152/2086621, SelectedAccessService=NDAC_SGT_Service, FailureReason=11302 Received Secure RADIUS request without a cts-pac-opaque cisco-av-pair attribute, Step=11001, Step=11017, Step=15012, Step=11302, NetworkDeviceGroups=Location#All Locations#NTN, NetworkDeviceGroups=Device Type#All Device Types#Wired, CPMSessionID=0a2a073f00005D5C53E3277E, Model Name=4503, Location=Location#All Locations#NTN, Device Type=Device Type#All Device Types#Wired, Res
 ponse={RadiusPacketType=Drop; },
+Aug  6 16:57:03 10.42.7.64 Aug  7 00:16:15 npf-sjca-pdp02 CISE_Failed_Attempts 0000371051 1 0 2014-08-07 00:16:15.827 -07:00 0011270497 5405 NOTICE Failed-Attempt: RADIUS Request dropped, ConfigVersionId=240, Device IP Address=10.56.72.126, Device Port=1645, DestinationIPAddress=10.42.7.64, DestinationPort=1812, RadiusPacketType=AccessRequest, UserName=#CTSREQUEST#, Protocol=Radius, RequestLatency=6, NetworkDeviceName=ntn01-11a-sw3, User-Name=#CTSREQUEST#, NAS-IP-Address=10.56.72.126, Service-Type=Outbound, AcsSessionID=npf-sjca-pdp02/195481465/271067, SelectedAccessService=NDAC_SGT_Service, FailureReason=11302 Received Secure RADIUS request without a cts-pac-opaque cisco-av-pair attribute, Step=11001, Step=11017, Step=15012, Step=11302, NetworkDeviceGroups=Location#All Locations#NTN, NetworkDeviceGroups=Device Type#All Device Types#Wired, CPMSessionID=0a2a07400000570453E327BF, Model Name=4503, Location=Location#All Locations#NTN, Device Type=Device Type#All Device Types#Wired, Resp
 onse={RadiusPacketType=Drop; },
+Aug  6 16:57:08 10.42.7.63 Aug  7 00:16:20 npf-sjca-pdp01 CISE_Failed_Attempts 0001971096 1 0 2014-08-07 00:16:20.857 -07:00 0098703837 5405 NOTICE Failed-Attempt: RADIUS Request dropped, ConfigVersionId=133, Device IP Address=10.56.72.126, Device Port=1645, DestinationIPAddress=10.42.7.63, DestinationPort=1812, RadiusPacketType=AccessRequest, UserName=#CTSREQUEST#, Protocol=Radius, RequestLatency=2, NetworkDeviceName=ntn01-11a-sw3, User-Name=#CTSREQUEST#, NAS-IP-Address=10.56.72.126, Service-Type=Outbound, AcsSessionID=npf-sjca-pdp01/195491152/2086806, SelectedAccessService=NDAC_SGT_Service, FailureReason=11302 Received Secure RADIUS request without a cts-pac-opaque cisco-av-pair attribute, Step=11001, Step=11017, Step=15012, Step=11302, NetworkDeviceGroups=Location#All Locations#NTN, NetworkDeviceGroups=Device Type#All Device Types#Wired, CPMSessionID=0a2a073f00005D5D53E327C4, Model Name=4503, Location=Location#All Locations#NTN, Device Type=Device Type#All Device Types#Wired, Res
 ponse={RadiusPacketType=Drop; },
+Aug  6 16:57:24 10.34.84.145 Aug  7 00:16:36 stage-pdp01 CISE_Failed_Attempts 0000024697 1 0 2014-08-07 00:16:36.602 -07:00 0000287553 5405 NOTICE Failed-Attempt: RADIUS Request dropped, ConfigVersionId=113, Device IP Address=172.23.91.132, Device Port=32769, DestinationIPAddress=10.34.84.145, DestinationPort=1812, Protocol=Radius, User-Name=test, NAS-IP-Address=192.168.30.11, Service-Type=NAS Prompt, NAS-Identifier=Cisco_1b:e0:84, AcsSessionID=stage-pdp01/196593288/19384, FailureReason=11007 Could not locate Network Device or AAA Client, Step=11001, Step=11017, Step=11007, Step=5405, 
+Aug  6 16:57:26 10.34.84.145 Aug  7 00:16:38 stage-pdp01 CISE_Failed_Attempts 0000024698 1 0 2014-08-07 00:16:38.605 -07:00 0000287557 5405 NOTICE Failed-Attempt: RADIUS Request dropped, ConfigVersionId=113, Device IP Address=172.23.91.132, Device Port=32769, DestinationIPAddress=10.34.84.145, DestinationPort=1812, Protocol=Radius, User-Name=test, NAS-IP-Address=192.168.30.11, Service-Type=NAS Prompt, NAS-Identifier=Cisco_1b:e0:84, AcsSessionID=stage-pdp01/196593288/19385, FailureReason=11007 Could not locate Network Device or AAA Client, Step=11001, Step=11017, Step=11007, Step=5405, 
+Aug  6 16:57:28 10.34.84.145 Aug  7 00:16:40 stage-pdp01 CISE_Failed_Attempts 0000024699 1 0 2014-08-07 00:16:40.609 -07:00 0000287561 5405 NOTICE Failed-Attempt: RADIUS Request dropped, ConfigVersionId=113, Device IP Address=172.23.91.132, Device Port=32769, DestinationIPAddress=10.34.84.145, DestinationPort=1812, Protocol=Radius, User-Name=test, NAS-IP-Address=192.168.30.11, Service-Type=NAS Prompt, NAS-Identifier=Cisco_1b:e0:84, AcsSessionID=stage-pdp01/196593288/19386, FailureReason=11007 Could not locate Network Device or AAA Client, Step=11001, Step=11017, Step=11007, Step=5405, 
+Aug  6 16:57:30 10.34.84.145 Aug  7 00:16:42 stage-pdp01 CISE_Failed_Attempts 0000024700 1 0 2014-08-07 00:16:42.613 -07:00 0000287565 5405 NOTICE Failed-Attempt: RADIUS Request dropped, ConfigVersionId=113, Device IP Address=172.23.91.132, Device Port=32769, DestinationIPAddress=10.34.84.145, DestinationPort=1812, Protocol=Radius, User-Name=test, NAS-IP-Address=192.168.30.11, Service-Type=NAS Prompt, NAS-Identifier=Cisco_1b:e0:84, AcsSessionID=stage-pdp01/196593288/19387, FailureReason=11007 Could not locate Network Device or AAA Client, Step=11001, Step=11017, Step=11007, Step=5405, 
+Aug  6 16:57:32 10.34.84.145 Aug  7 00:16:44 stage-pdp01 CISE_Failed_Attempts 0000024701 1 0 2014-08-07 00:16:44.613 -07:00 0000287569 5405 NOTICE Failed-Attempt: RADIUS Request dropped, ConfigVersionId=113, Device IP Address=172.23.91.132, Device Port=32769, DestinationIPAddress=10.34.84.145, DestinationPort=1812, Protocol=Radius, User-Name=test, NAS-IP-Address=192.168.30.11, Service-Type=NAS Prompt, NAS-Identifier=Cisco_1b:e0:84, AcsSessionID=stage-pdp01/196593288/19388, FailureReason=11007 Could not locate Network Device or AAA Client, Step=11001, Step=11017, Step=11007, Step=5405, 
+Aug  6 16:57:34 10.34.84.145 Aug  7 00:16:46 stage-pdp01 CISE_Failed_Attempts 0000024702 1 0 2014-08-07 00:16:46.617 -07:00 0000287573 5405 NOTICE Failed-Attempt: RADIUS Request dropped, ConfigVersionId=113, Device IP Address=172.23.91.132, Device Port=32769, DestinationIPAddress=10.34.84.145, DestinationPort=1812, Protocol=Radius, User-Name=test, NAS-IP-Address=192.168.30.11, Service-Type=NAS Prompt, NAS-Identifier=Cisco_1b:e0:84, AcsSessionID=stage-pdp01/196593288/19389, FailureReason=11007 Could not locate Network Device or AAA Client, Step=11001, Step=11017, Step=11007, Step=5405, 
+Aug  6 16:58:03 10.42.7.64 Aug  7 00:17:15 npf-sjca-pdp02 CISE_Failed_Attempts 0000371063 1 0 2014-08-07 00:17:15.966 -07:00 0011270832 5435 NOTICE RADIUS: NAS conducted several failed authentications of the same scenario, ConfigVersionId=240, Device IP Address=10.34.76.212, Device Port=32770, DestinationIPAddress=10.42.7.64, DestinationPort=1813, RadiusPacketType=AccountingRequest, UserName=hslai, Protocol=Radius, RequestLatency=25, NetworkDeviceName=sjcm-00a-npf-wlc1, User-Name=hslai, NAS-IP-Address=10.34.76.212, NAS-Port=1, Framed-IP-Address=10.34.94.11, Class=CACS:0a224cd40002fdf953e327f2:npf-sjca-pdp02/195481465/271072, Called-Station-ID=88-43-e1-62-1d-20, Calling-Station-ID=24-a2-e1-3b-4b-cb, NAS-Identifier=sjcm-00a-npf-wlc1, Acct-Status-Type=Interim-Update, Acct-Delay-Time=0, Acct-Input-Octets=5198, Acct-Output-Octets=4093, Acct-Session-Id=53e327f2/24:a2:e1:3b:4b:cb/174403, Acct-Authentic=RADIUS, Acct-Session-Time=9, Acct-Input-Packets=37, Acct-Output-Packets=13, undefined-52
 =
+Aug  6 16:58:13 10.42.7.64 Aug  7 00:17:25 npf-sjca-pdp02 CISE_Failed_Attempts 0000371065 1 0 2014-08-07 00:17:25.902 -07:00 0011270838 5405 NOTICE Failed-Attempt: RADIUS Request dropped, ConfigVersionId=240, Device IP Address=10.56.72.126, Device Port=1645, DestinationIPAddress=10.42.7.64, DestinationPort=1812, RadiusPacketType=AccessRequest, UserName=#CTSREQUEST#, Protocol=Radius, RequestLatency=4, NetworkDeviceName=ntn01-11a-sw3, User-Name=#CTSREQUEST#, NAS-IP-Address=10.56.72.126, Service-Type=Outbound, AcsSessionID=npf-sjca-pdp02/195481465/271076, SelectedAccessService=NDAC_SGT_Service, FailureReason=11302 Received Secure RADIUS request without a cts-pac-opaque cisco-av-pair attribute, Step=11001, Step=11017, Step=15012, Step=11302, NetworkDeviceGroups=Location#All Locations#NTN, NetworkDeviceGroups=Device Type#All Device Types#Wired, CPMSessionID=0a2a07400000570553E32805, Model Name=4503, Location=Location#All Locations#NTN, Device Type=Device Type#All Device Types#Wired, Resp
 onse={RadiusPacketType=Drop; },
+Aug  6 16:58:18 10.42.7.63 Aug  7 00:17:30 npf-sjca-pdp01 CISE_Failed_Attempts 0001971204 1 0 2014-08-07 00:17:30.916 -07:00 0098707928 5405 NOTICE Failed-Attempt: RADIUS Request dropped, ConfigVersionId=133, Device IP Address=10.56.72.126, Device Port=1645, DestinationIPAddress=10.42.7.63, DestinationPort=1812, RadiusPacketType=AccessRequest, UserName=#CTSREQUEST#, Protocol=Radius, RequestLatency=1, NetworkDeviceName=ntn01-11a-sw3, User-Name=#CTSREQUEST#, NAS-IP-Address=10.56.72.126, Service-Type=Outbound, AcsSessionID=npf-sjca-pdp01/195491152/2086981, SelectedAccessService=NDAC_SGT_Service, FailureReason=11302 Received Secure RADIUS request without a cts-pac-opaque cisco-av-pair attribute, Step=11001, Step=11017, Step=15012, Step=11302, NetworkDeviceGroups=Location#All Locations#NTN, NetworkDeviceGroups=Device Type#All Device Types#Wired, CPMSessionID=0a2a073f00005D5E53E3280A, Model Name=4503, Location=Location#All Locations#NTN, Device Type=Device Type#All Device Types#Wired, Res
 ponse={RadiusPacketType=Drop; },
+Aug  6 16:59:23 10.42.7.64 Aug  7 00:18:35 npf-sjca-pdp02 CISE_Failed_Attempts 0000371070 1 0 2014-08-07 00:18:35.942 -07:00 0011271044 5405 NOTICE Failed-Attempt: RADIUS Request dropped, ConfigVersionId=240, Device IP Address=10.56.72.126, Device Port=1645, DestinationIPAddress=10.42.7.64, DestinationPort=1812, RadiusPacketType=AccessRequest, UserName=#CTSREQUEST#, Protocol=Radius, RequestLatency=5, NetworkDeviceName=ntn01-11a-sw3, User-Name=#CTSREQUEST#, NAS-IP-Address=10.56.72.126, Service-Type=Outbound, AcsSessionID=npf-sjca-pdp02/195481465/271081, SelectedAccessService=NDAC_SGT_Service, FailureReason=11302 Received Secure RADIUS request without a cts-pac-opaque cisco-av-pair attribute, Step=11001, Step=11017, Step=15012, Step=11302, NetworkDeviceGroups=Location#All Locations#NTN, NetworkDeviceGroups=Device Type#All Device Types#Wired, CPMSessionID=0a2a07400000570653E3284B, Model Name=4503, Location=Location#All Locations#NTN, Device Type=Device Type#All Device Types#Wired, Resp
 onse={RadiusPacketType=Drop; },
+Aug  6 16:59:28 10.42.7.64 Aug  7 00:18:40 npf-sjca-pdp02 CISE_Failed_Attempts 0000371072 1 0 2014-08-07 00:18:40.669 -07:00 0011271053 5400 NOTICE Failed-Attempt: Authentication failed, ConfigVersionId=240, Device IP Address=10.56.129.4, Device Port=32770, DestinationIPAddress=10.42.7.64, DestinationPort=1812, RadiusPacketType=AccessRequest, UserName=istern, Protocol=Radius, RequestLatency=12, NetworkDeviceName=NTN-WLC1, User-Name=istern, NAS-IP-Address=10.56.129.4, NAS-Port=1, Service-Type=Framed, Framed-MTU=1300, State=37CPMSessionID=0a388104000045de53e2c750\;41SessionID=npf-sjca-pdp02/195481465/271077\;, Called-Station-ID=70-10-5c-f3-2f-80:alpha_byod, Calling-Station-ID=f0-27-65-48-8c-8f, NAS-Identifier=ntn01-11a-wlc1, NAS-Port-Type=Wireless - IEEE 802.11, Tunnel-Type=(tag=0) VLAN, Tunnel-Medium-Type=(tag=0) 802, Tunnel-Private-Group-ID=(tag=0) 604, undefined-89=
+Aug  6 16:59:28 10.42.7.63 Aug  7 00:18:40 npf-sjca-pdp01 CISE_Failed_Attempts 0001971282 1 0 2014-08-07 00:18:40.981 -07:00 0098711291 5405 NOTICE Failed-Attempt: RADIUS Request dropped, ConfigVersionId=133, Device IP Address=10.56.72.126, Device Port=1645, DestinationIPAddress=10.42.7.63, DestinationPort=1812, RadiusPacketType=AccessRequest, UserName=#CTSREQUEST#, Protocol=Radius, RequestLatency=2, NetworkDeviceName=ntn01-11a-sw3, User-Name=#CTSREQUEST#, NAS-IP-Address=10.56.72.126, Service-Type=Outbound, AcsSessionID=npf-sjca-pdp01/195491152/2087140, SelectedAccessService=NDAC_SGT_Service, FailureReason=11302 Received Secure RADIUS request without a cts-pac-opaque cisco-av-pair attribute, Step=11001, Step=11017, Step=15012, Step=11302, NetworkDeviceGroups=Location#All Locations#NTN, NetworkDeviceGroups=Device Type#All Device Types#Wired, CPMSessionID=0a2a073f00005D5F53E32850, Model Name=4503, Location=Location#All Locations#NTN, Device Type=Device Type#All Device Types#Wired, Res
 ponse={RadiusPacketType=Drop; },
+Aug  6 17:00:33 10.42.7.64 Aug  7 00:19:46 npf-sjca-pdp02 CISE_Failed_Attempts 0000371080 1 0 2014-08-07 00:19:46.020 -07:00 0011271232 5405 NOTICE Failed-Attempt: RADIUS Request dropped, ConfigVersionId=240, Device IP Address=10.56.72.126, Device Port=1645, DestinationIPAddress=10.42.7.64, DestinationPort=1812, RadiusPacketType=AccessRequest, UserName=#CTSREQUEST#, Protocol=Radius, RequestLatency=5, NetworkDeviceName=ntn01-11a-sw3, User-Name=#CTSREQUEST#, NAS-IP-Address=10.56.72.126, Service-Type=Outbound, AcsSessionID=npf-sjca-pdp02/195481465/271087, SelectedAccessService=NDAC_SGT_Service, FailureReason=11302 Received Secure RADIUS request without a cts-pac-opaque cisco-av-pair attribute, Step=11001, Step=11017, Step=15012, Step=11302, NetworkDeviceGroups=Location#All Locations#NTN, NetworkDeviceGroups=Device Type#All Device Types#Wired, CPMSessionID=0a2a07400000570753E32892, Model Name=4503, Location=Location#All Locations#NTN, Device Type=Device Type#All Device Types#Wired, Resp
 onse={RadiusPacketType=Drop; },
+Aug  6 17:00:36 10.34.84.145 Aug  7 00:19:48 stage-pdp01 CISE_Failed_Attempts 0000024712 1 0 2014-08-07 00:19:48.660 -07:00 0000287604 5405 NOTICE Failed-Attempt: RADIUS Request dropped, ConfigVersionId=113, Device IP Address=172.23.91.132, Device Port=32769, DestinationIPAddress=10.34.84.145, DestinationPort=1812, Protocol=Radius, User-Name=test, NAS-IP-Address=192.168.30.11, Service-Type=NAS Prompt, NAS-Identifier=Cisco_1b:e0:84, AcsSessionID=stage-pdp01/196593288/19396, FailureReason=11007 Could not locate Network Device or AAA Client, Step=11001, Step=11017, Step=11007, Step=5405, 
+Aug  6 17:00:38 10.34.84.145 Aug  7 00:19:50 stage-pdp01 CISE_Failed_Attempts 0000024713 1 0 2014-08-07 00:19:50.664 -07:00 0000287608 5405 NOTICE Failed-Attempt: RADIUS Request dropped, ConfigVersionId=113, Device IP Address=172.23.91.132, Device Port=32769, DestinationIPAddress=10.34.84.145, DestinationPort=1812, Protocol=Radius, User-Name=test, NAS-IP-Address=192.168.30.11, Service-Type=NAS Prompt, NAS-Identifier=Cisco_1b:e0:84, AcsSessionID=stage-pdp01/196593288/19397, FailureReason=11007 Could not locate Network Device or AAA Client, Step=11001, Step=11017, Step=11007, Step=5405, 
+Aug  6 17:00:39 10.42.7.63 Aug  7 00:19:51 npf-sjca-pdp01 CISE_Failed_Attempts 0001971393 1 0 2014-08-07 00:19:51.042 -07:00 0098716185 5405 NOTICE Failed-Attempt: RADIUS Request dropped, ConfigVersionId=133, Device IP Address=10.56.72.126, Device Port=1645, DestinationIPAddress=10.42.7.63, DestinationPort=1812, RadiusPacketType=AccessRequest, UserName=#CTSREQUEST#, Protocol=Radius, RequestLatency=1, NetworkDeviceName=ntn01-11a-sw3, User-Name=#CTSREQUEST#, NAS-IP-Address=10.56.72.126, Service-Type=Outbound, AcsSessionID=npf-sjca-pdp01/195491152/2087311, SelectedAccessService=NDAC_SGT_Service, FailureReason=11302 Received Secure RADIUS request without a cts-pac-opaque cisco-av-pair attribute, Step=11001, Step=11017, Step=15012, Step=11302, NetworkDeviceGroups=Location#All Locations#NTN, NetworkDeviceGroups=Device Type#All Device Types#Wired, CPMSessionID=0a2a073f00005D6053E32897, Model Name=4503, Location=Location#All Locations#NTN, Device Type=Device Type#All Device Types#Wired, Res
 ponse={RadiusPacketType=Drop; },
+Aug  6 17:00:40 10.34.84.145 Aug  7 00:19:52 stage-pdp01 CISE_Failed_Attempts 0000024714 1 0 2014-08-07 00:19:52.664 -07:00 0000287612 5405 NOTICE Failed-Attempt: RADIUS Request dropped, ConfigVersionId=113, Device IP Address=172.23.91.132, Device Port=32769, DestinationIPAddress=10.34.84.145, DestinationPort=1812, Protocol=Radius, User-Name=test, NAS-IP-Address=192.168.30.11, Service-Type=NAS Prompt, NAS-Identifier=Cisco_1b:e0:84, AcsSessionID=stage-pdp01/196593288/19398, FailureReason=11007 Could not locate Network Device or AAA Client, Step=11001, Step=11017, Step=11007, Step=5405, 
+Aug  6 17:00:42 10.34.84.145 Aug  7 00:19:54 stage-pdp01 CISE_Failed_Attempts 0000024715 1 0 2014-08-07 00:19:54.668 -07:00 0000287616 5405 NOTICE Failed-Attempt: RADIUS Request dropped, ConfigVersionId=113, Device IP Address=172.23.91.132, Device Port=32769, DestinationIPAddress=10.34.84.145, DestinationPort=1812, Protocol=Radius, User-Name=test, NAS-IP-Address=192.168.30.11, Service-Type=NAS Prompt, NAS-Identifier=Cisco_1b:e0:84, AcsSessionID=stage-pdp01/196593288/19399, FailureReason=11007 Could not locate Network Device or AAA Client, Step=11001, Step=11017, Step=11007, Step=5405, 
+Aug  6 17:00:44 10.34.84.145 Aug  7 00:19:56 stage-pdp01 CISE_Failed_Attempts 0000024716 1 0 2014-08-07 00:19:56.672 -07:00 0000287620 5405 NOTICE Failed-Attempt: RADIUS Request dropped, ConfigVersionId=113, Device IP Address=172.23.91.132, Device Port=32769, DestinationIPAddress=10.34.84.145, DestinationPort=1812, Protocol=Radius, User-Name=test, NAS-IP-Address=192.168.30.11, Service-Type=NAS Prompt, NAS-Identifier=Cisco_1b:e0:84, AcsSessionID=stage-pdp01/196593288/19400, FailureReason=11007 Could not locate Network Device or AAA Client, Step=11001, Step=11017, Step=11007, Step=5405, 
+Aug  6 17:00:46 10.34.84.145 Aug  7 00:19:58 stage-pdp01 CISE_Failed_Attempts 0000024717 1 0 2014-08-07 00:19:58.675 -07:00 0000287624 5405 NOTICE Failed-Attempt: RADIUS Request dropped, ConfigVersionId=113, Device IP Address=172.23.91.132, Device Port=32769, DestinationIPAddress=10.34.84.145, DestinationPort=1812, Protocol=Radius, User-Name=test, NAS-IP-Address=192.168.30.11, Service-Type=NAS Prompt, NAS-Identifier=Cisco_1b:e0:84, AcsSessionID=stage-pdp01/196593288/19401, FailureReason=11007 Could not locate Network Dev

<TRUNCATED>


[10/15] incubator-metron git commit: METRON-142 Simplify Parser configuration (merrimanr via cestella) closes apache/incubator-metron#120

Posted by ce...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/df8d682e/metron-platform/metron-integration-test/src/main/resources/sample/data/SampleInput/ISESampleOutput
----------------------------------------------------------------------
diff --git a/metron-platform/metron-integration-test/src/main/resources/sample/data/SampleInput/ISESampleOutput b/metron-platform/metron-integration-test/src/main/resources/sample/data/SampleInput/ISESampleOutput
deleted file mode 100644
index 1cb0678..0000000
--- a/metron-platform/metron-integration-test/src/main/resources/sample/data/SampleInput/ISESampleOutput
+++ /dev/null
@@ -1,308 +0,0 @@
-Aug  6 17:26:31 10.34.84.145 Aug  7 00:45:43 stage-pdp01 CISE_Profiler 0000024855 1 0 2014-08-07 00:45:43.741 -07:00 0000288542 80002 INFO  Profiler: Profiler EndPoint profiling event occurred, ConfigVersionId=113, EndpointCertainityMetric=10, EndpointIPAddress=10.56.111.14, EndpointMacAddress=3C:97:0E:C3:F8:F1, EndpointMatchedPolicy=Nortel-Device, EndpointNADAddress=10.56.72.127, EndpointOUI=Wistron InfoComm(Kunshan)Co.\,Ltd., EndpointPolicy=Nortel-Device, EndpointProperty=StaticAssignment=false\,PostureApplicable=Yes\,PolicyVersion=402\,IdentityGroupID=0c1d9270-68a6-11e1-bc72-0050568e013c\,Total Certainty Factor=10\,BYODRegistration=Unknown\,FeedService=false\,EndPointPolicyID=49054ed0-68a6-11e1-bc72-0050568e013c\,FirstCollection=1407397543718\,MatchedPolicyID=49054ed0-68a6-11e1-bc72-0050568e013c\,TimeToProfile=19\,StaticGroupAssignment=false\,NmapSubnetScanID=0\,DeviceRegistrationStatus=NotRegistered\,PortalUser=, EndpointSourceEvent=SNMPQuery Probe, EndpointIdentityGroup=Profile
 d, ProfilerServer=stage-pdp01.cisco.com, 
-Aug  6 17:26:31 10.34.84.145 Aug  7 00:45:43 stage-pdp01 CISE_Profiler 0000024856 1 0 2014-08-07 00:45:43.786 -07:00 0000288543 80002 INFO  Profiler: Profiler EndPoint profiling event occurred, ConfigVersionId=113, EndpointCertainityMetric=10, EndpointIPAddress=10.56.111.14, EndpointMacAddress=3C:97:0E:C3:F8:F1, EndpointMatchedPolicy=Nortel-Device, EndpointNADAddress=10.56.72.127, EndpointOUI=Wistron InfoComm(Kunshan)Co.\,Ltd., EndpointPolicy=Nortel-Device, EndpointProperty=StaticAssignment=false\,PostureApplicable=Yes\,BYODRegistration=Unknown\,EndPointPolicyID=49054ed0-68a6-11e1-bc72-0050568e013c\,FirstCollection=1407397543718\,TimeToProfile=19\,LastNmapScanTime=0\,StaticGroupAssignment=false\,DeviceRegistrationStatus=NotRegistered\,UpdateTime=0\,PolicyVersion=402\,IdentityGroupID=0c1d9270-68a6-11e1-bc72-0050568e013c\,Total Certainty Factor=10\,FeedService=false\,MatchedPolicyID=49054ed0-68a6-11e1-bc72-0050568e013c\,NmapScanCount=0\,NmapSubnetScanID=0\,PortalUser=, EndpointSourceE
 vent=SNMPQuery Probe, EndpointIdentityGroup=Profiled, ProfilerServer=stage-pdp01.cisco.com, 
-Aug  6 20:00:52 10.42.7.64 Aug  7 03:20:05 npf-sjca-pdp02 CISE_Profiler 0000373185 1 0 2014-08-07 03:20:05.549 -07:00 0011310202 80002 INFO  Profiler: Profiler EndPoint profiling event occurred, ConfigVersionId=241, EndpointCertainityMetric=90, EndpointIPAddress=10.56.129.142, EndpointMacAddress=3C:A9:F4:46:75:CC, EndpointMatchedPolicy=Windows7-Workstation, EndpointNADAddress=10.56.129.4, EndpointOUI=Intel Corporate, EndpointPolicy=Windows7-Workstation, EndpointProperty=StaticAssignment=false\,Calling-Station-ID=3c-a9-f4-46-75-cc\,Device Identifier=\,PostureApplicable=Yes\,dhcp-class-identifier=MSFT 5.0\,host-name=HASSI-WS03\,BYODRegistration=Unknown\,EndPointPolicyID=615ed410-68a6-11e1-bc72-0050568e013c\,FirstCollection=1406112353750\,TimeToProfile=11\,Framed-IP-Address=10.56.129.142\,LastNmapScanTime=0\,StaticGroupAssignment=false\,DeviceRegistrationStatus=NotRegistered\,NAS-Port-Type=Wireless - IEEE 802.11\,RegistrationTimeStamp=0\,UpdateTime=1407394245820\,PolicyVersion=403\,Ide
 ntityGroupID=5cb39b80-68a6-11e1-bc72-0050568e013c\,Total Certainty Factor=90\,FeedService=false\,MatchedPolicyID=615ed410-68a6-11e1-bc72-0050568e013c\,DestinationIPAddress=10.42.7.64\,CreateTime=1394526689397\,NmapScanCount=0\,NmapSubnetScanID=0\,AAA-Server=npf-sjca-pdp02\,PortalUser=, EndpointSourceEvent=RADIUS Probe, EndpointUserAgent=Mozilla/5.0 (Windows NT 6.1\\\; WOW64\\ rv:30.0) Gecko/20100101 Firefox/30.0, EndpointIdentityGroup=Workstation, ProfilerServer=npf-sjca-pdp02.cisco.com, 
-Aug  6 21:00:48 10.42.7.64 Aug  7 04:20:00 npf-sjca-pdp02 CISE_Profiler 0000373902 1 0 2014-08-07 04:20:00.983 -07:00 0011322557 80002 INFO  Profiler: Profiler EndPoint profiling event occurred, ConfigVersionId=241, EndpointCertainityMetric=30, EndpointIPAddress=10.56.129.142, EndpointMacAddress=3C:A9:F4:46:75:CC, EndpointMatchedPolicy=Microsoft-Workstation, EndpointNADAddress=10.56.129.4, EndpointOUI=Intel Corporate, EndpointPolicy=Microsoft-Workstation, EndpointProperty=StaticAssignment=false\,Calling-Station-ID=3c-a9-f4-46-75-cc\,Device Identifier=\,PostureApplicable=Yes\,dhcp-class-identifier=MSFT 5.0\,host-name=HASSI-WS03\,BYODRegistration=Unknown\,EndPointPolicyID=5f4a24e0-68a6-11e1-bc72-0050568e013c\,FirstCollection=1406112353750\,TimeToProfile=11\,Framed-IP-Address=10.56.129.142\,LastNmapScanTime=0\,StaticGroupAssignment=false\,DeviceRegistrationStatus=NotRegistered\,NAS-Port-Type=Wireless - IEEE 802.11\,RegistrationTimeStamp=0\,UpdateTime=1407406806572\,PolicyVersion=403\,I
 dentityGroupID=5cb39b80-68a6-11e1-bc72-0050568e013c\,Total Certainty Factor=30\,FeedService=false\,MatchedPolicyID=5f4a24e0-68a6-11e1-bc72-0050568e013c\,DestinationIPAddress=10.42.7.64\,CreateTime=1394526689397\,NmapScanCount=0\,NmapSubnetScanID=0\,AAA-Server=npf-sjca-pdp02\,PortalUser=, EndpointSourceEvent=RADIUS Probe, EndpointUserAgent=MS-WebServices/1.0, EndpointIdentityGroup=Workstation, ProfilerServer=npf-sjca-pdp02.cisco.com, 
-Aug  6 22:22:50 10.42.7.64 Aug  7 05:42:03 npf-sjca-pdp02 CISE_Profiler 0000374846 1 0 2014-08-07 05:42:03.617 -07:00 0011340138 80002 INFO  Profiler: Profiler EndPoint profiling event occurred, ConfigVersionId=241, EndpointCertainityMetric=10, EndpointMacAddress=68:A8:6D:4E:0D:86, EndpointMatchedPolicy=Apple-Device, EndpointOUI=Apple, EndpointPolicy=Apple-Device, EndpointProperty=StaticAssignment=false\,PostureApplicable=Yes\,host-name=PGIANG-M-306R\,BYODRegistration=Unknown\,EndPointPolicyID=377d8ba0-68a6-11e1-bc72-0050568e013c\,FirstCollection=1407415322895\,TimeToProfile=717\,StaticGroupAssignment=false\,DeviceRegistrationStatus=NotRegistered\,PolicyVersion=403\,IdentityGroupID=abbbcac0-89e6-11e1-bf14-005056aa4dd7\,Total Certainty Factor=10\,ciaddr=0.0.0.0\,FeedService=false\,dhcp-parameter-request-list=1\, 3\, 6\, 15\, 119\, 95\, 252\, 44\, 46\,MatchedPolicyID=377d8ba0-68a6-11e1-bc72-0050568e013c\,NmapSubnetScanID=0\,PortalUser=, EndpointSourceEvent=DHCP Probe, EndpointIdentity
 Group=Apple-Device, ProfilerServer=npf-sjca-pdp02.cisco.com, 
-Aug  6 23:30:10 10.42.7.64 Aug  7 06:49:23 npf-sjca-pdp02 CISE_Profiler 0000375603 1 0 2014-08-07 06:49:23.920 -07:00 0011353768 80002 INFO  Profiler: Profiler EndPoint profiling event occurred, ConfigVersionId=241, EndpointCertainityMetric=90, EndpointIPAddress=10.56.129.142, EndpointMacAddress=3C:A9:F4:46:75:CC, EndpointMatchedPolicy=Windows7-Workstation, EndpointNADAddress=10.56.129.4, EndpointOUI=Intel Corporate, EndpointPolicy=Windows7-Workstation, EndpointProperty=StaticAssignment=false\,Calling-Station-ID=3c-a9-f4-46-75-cc\,Device Identifier=\,PostureApplicable=Yes\,dhcp-class-identifier=MSFT 5.0\,host-name=HASSI-WS03\,BYODRegistration=Unknown\,EndPointPolicyID=615ed410-68a6-11e1-bc72-0050568e013c\,FirstCollection=1406112353750\,TimeToProfile=11\,Framed-IP-Address=10.56.129.142\,LastNmapScanTime=0\,StaticGroupAssignment=false\,DeviceRegistrationStatus=NotRegistered\,NAS-Port-Type=Wireless - IEEE 802.11\,RegistrationTimeStamp=0\,UpdateTime=1407410402099\,PolicyVersion=403\,Ide
 ntityGroupID=5cb39b80-68a6-11e1-bc72-0050568e013c\,Total Certainty Factor=90\,FeedService=false\,MatchedPolicyID=615ed410-68a6-11e1-bc72-0050568e013c\,DestinationIPAddress=10.42.7.64\,CreateTime=1394526689397\,NmapScanCount=0\,NmapSubnetScanID=0\,AAA-Server=npf-sjca-pdp02\,PortalUser=, EndpointSourceEvent=RADIUS Probe, EndpointUserAgent=Mozilla/5.0 (Windows NT 6.1\\\; WOW64\\ rv:30.0) Gecko/20100101 Firefox/30.0, EndpointIdentityGroup=Workstation, ProfilerServer=npf-sjca-pdp02.cisco.com, 
-Aug  6 23:30:48 10.42.7.64 Aug  7 06:50:01 npf-sjca-pdp02 CISE_Profiler 0000375611 1 0 2014-08-07 06:50:01.377 -07:00 0011353875 80002 INFO  Profiler: Profiler EndPoint profiling event occurred, ConfigVersionId=241, EndpointCertainityMetric=50, EndpointIPAddress=10.34.92.103, EndpointMacAddress=3C:A9:F4:29:FC:3C, EndpointMatchedPolicy=Microsoft-Workstation, EndpointNADAddress=10.34.76.212, EndpointOUI=Intel Corporate, EndpointPolicy=Microsoft-Workstation, EndpointProperty=StaticAssignment=false\,Calling-Station-ID=3c-a9-f4-29-fc-3c\,Device Identifier=\,PostureApplicable=Yes\,dhcp-class-identifier=MSFT 5.0\,host-name=AMIBASU-WS01\,BYODRegistration=Unknown\,EndPointPolicyID=5f4a24e0-68a6-11e1-bc72-0050568e013c\,FirstCollection=1406109860322\,L4_DST_PORT=50428\,TimeToProfile=7\,Framed-IP-Address=10.34.92.103\,LastNmapScanTime=1380758278898\,StaticGroupAssignment=false\,DeviceRegistrationStatus=NotRegistered\,NAS-Port-Type=Wireless - IEEE 802.11\,RegistrationTimeStamp=0\,UpdateTime=1406
 686034558\,PolicyVersion=403\,IdentityGroupID=5cb39b80-68a6-11e1-bc72-0050568e013c\,Total Certainty Factor=50\,operating-system=Microsoft Windows Vista SP0 - SP2\, Server 2008\, or Windows 7 Ultimate\,FeedService=false\,MatchedPolicyID=5f4a24e0-68a6-11e1-bc72-0050568e013c\,DestinationIPAddress=10.42.7.64\,CreateTime=1373657280926\,NmapScanCount=3\,NmapSubnetScanID=0\,AAA-Server=npf-sjca-pdp02\,PortalUser=, EndpointSourceEvent=RADIUS Probe, EndpointUserAgent=MS-WebServices/1.0, EndpointIdentityGroup=Workstation, ProfilerServer=npf-sjca-pdp02.cisco.com, 
-Aug  6 23:32:52 10.42.7.64 Aug  7 06:52:05 npf-sjca-pdp02 CISE_Profiler 0000375636 1 0 2014-08-07 06:52:05.272 -07:00 0011354313 80002 INFO  Profiler: Profiler EndPoint profiling event occurred, ConfigVersionId=241, EndpointCertainityMetric=30, EndpointIPAddress=10.56.129.143, EndpointMacAddress=E8:2A:EA:23:5E:3D, EndpointMatchedPolicy=Microsoft-Workstation, EndpointNADAddress=10.56.129.4, EndpointOUI=Intel Corporate, EndpointPolicy=Microsoft-Workstation, EndpointProperty=StaticAssignment=false\,Calling-Station-ID=e8-2a-ea-23-5e-3d\,Device Identifier=\,PostureApplicable=Yes\,dhcp-class-identifier=MSFT 5.0\,host-name=ANOY-WS01\,BYODRegistration=Unknown\,EndPointPolicyID=5f4a24e0-68a6-11e1-bc72-0050568e013c\,FirstCollection=1406114784910\,TimeToProfile=7\,Framed-IP-Address=10.56.129.143\,LastNmapScanTime=0\,StaticGroupAssignment=false\,DeviceRegistrationStatus=NotRegistered\,NAS-Port-Type=Wireless - IEEE 802.11\,RegistrationTimeStamp=0\,UpdateTime=1407395211208\,PolicyVersion=403\,Ide
 ntityGroupID=5cb39b80-68a6-11e1-bc72-0050568e013c\,Total Certainty Factor=30\,FeedService=false\,MatchedPolicyID=5f4a24e0-68a6-11e1-bc72-0050568e013c\,DestinationIPAddress=10.42.7.64\,CreateTime=1405408515121\,NmapScanCount=0\,NmapSubnetScanID=0\,AAA-Server=npf-sjca-pdp02\,PortalUser=, EndpointSourceEvent=RADIUS Probe, EndpointUserAgent=MS-WebServices/1.0, EndpointIdentityGroup=Workstation, ProfilerServer=npf-sjca-pdp02.cisco.com, 
-Aug  6 16:40:52 10.42.7.64 Aug  7 00:00:04 npf-sjca-pdp02 CISE_Failed_Attempts 0000370855 1 0 2014-08-07 00:00:04.527 -07:00 0011266584 5405 NOTICE Failed-Attempt: RADIUS Request dropped, ConfigVersionId=240, Device IP Address=10.56.72.126, Device Port=1645, DestinationIPAddress=10.42.7.64, DestinationPort=1812, RadiusPacketType=AccessRequest, UserName=#CTSREQUEST#, Protocol=Radius, RequestLatency=5, NetworkDeviceName=ntn01-11a-sw3, User-Name=#CTSREQUEST#, NAS-IP-Address=10.56.72.126, Service-Type=Outbound, AcsSessionID=npf-sjca-pdp02/195481465/270932, SelectedAccessService=NDAC_SGT_Service, FailureReason=11302 Received Secure RADIUS request without a cts-pac-opaque cisco-av-pair attribute, Step=11001, Step=11017, Step=15012, Step=11302, NetworkDeviceGroups=Location#All Locations#NTN, NetworkDeviceGroups=Device Type#All Device Types#Wired, CPMSessionID=0a2a0740000056EF53E323F4, Model Name=4503, Location=Location#All Locations#NTN, Device Type=Device Type#All Device Types#Wired, Resp
 onse={RadiusPacketType=Drop; },
-Aug  6 16:40:57 10.42.7.63 Aug  7 00:00:09 npf-sjca-pdp01 CISE_Failed_Attempts 0001969834 1 0 2014-08-07 00:00:09.568 -07:00 0098648519 5405 NOTICE Failed-Attempt: RADIUS Request dropped, ConfigVersionId=133, Device IP Address=10.56.72.126, Device Port=1645, DestinationIPAddress=10.42.7.63, DestinationPort=1812, RadiusPacketType=AccessRequest, UserName=#CTSREQUEST#, Protocol=Radius, RequestLatency=2, NetworkDeviceName=ntn01-11a-sw3, User-Name=#CTSREQUEST#, NAS-IP-Address=10.56.72.126, Service-Type=Outbound, AcsSessionID=npf-sjca-pdp01/195491152/2084839, SelectedAccessService=NDAC_SGT_Service, FailureReason=11302 Received Secure RADIUS request without a cts-pac-opaque cisco-av-pair attribute, Step=11001, Step=11017, Step=15012, Step=11302, NetworkDeviceGroups=Location#All Locations#NTN, NetworkDeviceGroups=Device Type#All Device Types#Wired, CPMSessionID=0a2a073f00005D4A53E323F9, Model Name=4503, Location=Location#All Locations#NTN, Device Type=Device Type#All Device Types#Wired, Res
 ponse={RadiusPacketType=Drop; },
-Aug  6 16:41:24 10.34.84.145 Aug  7 00:00:36 stage-pdp01 CISE_Failed_Attempts 0000024616 1 0 2014-08-07 00:00:36.332 -07:00 0000287007 5405 NOTICE Failed-Attempt: RADIUS Request dropped, ConfigVersionId=113, Device IP Address=172.23.91.132, Device Port=32769, DestinationIPAddress=10.34.84.145, DestinationPort=1812, Protocol=Radius, User-Name=test, NAS-IP-Address=192.168.30.11, Service-Type=NAS Prompt, NAS-Identifier=Cisco_1b:e0:84, AcsSessionID=stage-pdp01/196593288/19317, FailureReason=11007 Could not locate Network Device or AAA Client, Step=11001, Step=11017, Step=11007, Step=5405, 
-Aug  6 16:41:26 10.34.84.145 Aug  7 00:00:38 stage-pdp01 CISE_Failed_Attempts 0000024617 1 0 2014-08-07 00:00:38.336 -07:00 0000287011 5405 NOTICE Failed-Attempt: RADIUS Request dropped, ConfigVersionId=113, Device IP Address=172.23.91.132, Device Port=32769, DestinationIPAddress=10.34.84.145, DestinationPort=1812, Protocol=Radius, User-Name=test, NAS-IP-Address=192.168.30.11, Service-Type=NAS Prompt, NAS-Identifier=Cisco_1b:e0:84, AcsSessionID=stage-pdp01/196593288/19318, FailureReason=11007 Could not locate Network Device or AAA Client, Step=11001, Step=11017, Step=11007, Step=5405, 
-Aug  6 16:41:28 10.34.84.145 Aug  7 00:00:40 stage-pdp01 CISE_Failed_Attempts 0000024618 1 0 2014-08-07 00:00:40.336 -07:00 0000287015 5405 NOTICE Failed-Attempt: RADIUS Request dropped, ConfigVersionId=113, Device IP Address=172.23.91.132, Device Port=32769, DestinationIPAddress=10.34.84.145, DestinationPort=1812, Protocol=Radius, User-Name=test, NAS-IP-Address=192.168.30.11, Service-Type=NAS Prompt, NAS-Identifier=Cisco_1b:e0:84, AcsSessionID=stage-pdp01/196593288/19319, FailureReason=11007 Could not locate Network Device or AAA Client, Step=11001, Step=11017, Step=11007, Step=5405, 
-Aug  6 16:41:30 10.34.84.145 Aug  7 00:00:42 stage-pdp01 CISE_Failed_Attempts 0000024619 1 0 2014-08-07 00:00:42.340 -07:00 0000287019 5405 NOTICE Failed-Attempt: RADIUS Request dropped, ConfigVersionId=113, Device IP Address=172.23.91.132, Device Port=32769, DestinationIPAddress=10.34.84.145, DestinationPort=1812, Protocol=Radius, User-Name=test, NAS-IP-Address=192.168.30.11, Service-Type=NAS Prompt, NAS-Identifier=Cisco_1b:e0:84, AcsSessionID=stage-pdp01/196593288/19320, FailureReason=11007 Could not locate Network Device or AAA Client, Step=11001, Step=11017, Step=11007, Step=5405, 
-Aug  6 16:41:32 10.34.84.145 Aug  7 00:00:44 stage-pdp01 CISE_Failed_Attempts 0000024620 1 0 2014-08-07 00:00:44.340 -07:00 0000287023 5405 NOTICE Failed-Attempt: RADIUS Request dropped, ConfigVersionId=113, Device IP Address=172.23.91.132, Device Port=32769, DestinationIPAddress=10.34.84.145, DestinationPort=1812, Protocol=Radius, User-Name=test, NAS-IP-Address=192.168.30.11, Service-Type=NAS Prompt, NAS-Identifier=Cisco_1b:e0:84, AcsSessionID=stage-pdp01/196593288/19321, FailureReason=11007 Could not locate Network Device or AAA Client, Step=11001, Step=11017, Step=11007, Step=5405, 
-Aug  6 16:41:34 10.34.84.145 Aug  7 00:00:46 stage-pdp01 CISE_Failed_Attempts 0000024621 1 0 2014-08-07 00:00:46.344 -07:00 0000287027 5405 NOTICE Failed-Attempt: RADIUS Request dropped, ConfigVersionId=113, Device IP Address=172.23.91.132, Device Port=32769, DestinationIPAddress=10.34.84.145, DestinationPort=1812, Protocol=Radius, User-Name=test, NAS-IP-Address=192.168.30.11, Service-Type=NAS Prompt, NAS-Identifier=Cisco_1b:e0:84, AcsSessionID=stage-pdp01/196593288/19322, FailureReason=11007 Could not locate Network Device or AAA Client, Step=11001, Step=11017, Step=11007, Step=5405, 
-Aug  6 16:42:02 10.42.7.64 Aug  7 00:01:14 npf-sjca-pdp02 CISE_Failed_Attempts 0000370865 1 0 2014-08-07 00:01:14.610 -07:00 0011266810 5405 NOTICE Failed-Attempt: RADIUS Request dropped, ConfigVersionId=240, Device IP Address=10.56.72.126, Device Port=1645, DestinationIPAddress=10.42.7.64, DestinationPort=1812, RadiusPacketType=AccessRequest, UserName=#CTSREQUEST#, Protocol=Radius, RequestLatency=7, NetworkDeviceName=ntn01-11a-sw3, User-Name=#CTSREQUEST#, NAS-IP-Address=10.56.72.126, Service-Type=Outbound, AcsSessionID=npf-sjca-pdp02/195481465/270940, SelectedAccessService=NDAC_SGT_Service, FailureReason=11302 Received Secure RADIUS request without a cts-pac-opaque cisco-av-pair attribute, Step=11001, Step=11017, Step=15012, Step=11302, NetworkDeviceGroups=Location#All Locations#NTN, NetworkDeviceGroups=Device Type#All Device Types#Wired, CPMSessionID=0a2a0740000056F053E3243A, Model Name=4503, Location=Location#All Locations#NTN, Device Type=Device Type#All Device Types#Wired, Resp
 onse={RadiusPacketType=Drop; },
-Aug  6 16:42:07 10.42.7.63 Aug  7 00:01:19 npf-sjca-pdp01 CISE_Failed_Attempts 0001969923 1 0 2014-08-07 00:01:19.665 -07:00 0098652715 5405 NOTICE Failed-Attempt: RADIUS Request dropped, ConfigVersionId=133, Device IP Address=10.56.72.126, Device Port=1645, DestinationIPAddress=10.42.7.63, DestinationPort=1812, RadiusPacketType=AccessRequest, UserName=#CTSREQUEST#, Protocol=Radius, RequestLatency=2, NetworkDeviceName=ntn01-11a-sw3, User-Name=#CTSREQUEST#, NAS-IP-Address=10.56.72.126, Service-Type=Outbound, AcsSessionID=npf-sjca-pdp01/195491152/2084986, SelectedAccessService=NDAC_SGT_Service, FailureReason=11302 Received Secure RADIUS request without a cts-pac-opaque cisco-av-pair attribute, Step=11001, Step=11017, Step=15012, Step=11302, NetworkDeviceGroups=Location#All Locations#NTN, NetworkDeviceGroups=Device Type#All Device Types#Wired, CPMSessionID=0a2a073f00005D4B53E3243F, Model Name=4503, Location=Location#All Locations#NTN, Device Type=Device Type#All Device Types#Wired, Res
 ponse={RadiusPacketType=Drop; },
-Aug  6 16:42:12 10.42.7.64 Aug  7 00:01:24 npf-sjca-pdp02 CISE_Failed_Attempts 0000370867 1 0 2014-08-07 00:01:24.701 -07:00 0011266815 5405 NOTICE Failed-Attempt: RADIUS Request dropped, ConfigVersionId=240, Device IP Address=10.56.72.126, Device Port=1645, DestinationIPAddress=10.42.7.64, DestinationPort=1812, RadiusPacketType=AccessRequest, UserName=#CTSREQUEST#, Protocol=Radius, RequestLatency=5, NetworkDeviceName=ntn01-11a-sw3, User-Name=#CTSREQUEST#, NAS-IP-Address=10.56.72.126, Service-Type=Outbound, AcsSessionID=npf-sjca-pdp02/195481465/270941, SelectedAccessService=NDAC_SGT_Service, FailureReason=11302 Received Secure RADIUS request without a cts-pac-opaque cisco-av-pair attribute, Step=11001, Step=11017, Step=15012, Step=11302, NetworkDeviceGroups=Location#All Locations#NTN, NetworkDeviceGroups=Device Type#All Device Types#Wired, CPMSessionID=0a2a0740000056F153E32444, Model Name=4503, Location=Location#All Locations#NTN, Device Type=Device Type#All Device Types#Wired, Resp
 onse={RadiusPacketType=Drop; },
-Aug  6 16:42:17 10.42.7.63 Aug  7 00:01:29 npf-sjca-pdp01 CISE_Failed_Attempts 0001969935 1 0 2014-08-07 00:01:29.746 -07:00 0098653362 5405 NOTICE Failed-Attempt: RADIUS Request dropped, ConfigVersionId=133, Device IP Address=10.56.72.126, Device Port=1645, DestinationIPAddress=10.42.7.63, DestinationPort=1812, RadiusPacketType=AccessRequest, UserName=#CTSREQUEST#, Protocol=Radius, RequestLatency=1, NetworkDeviceName=ntn01-11a-sw3, User-Name=#CTSREQUEST#, NAS-IP-Address=10.56.72.126, Service-Type=Outbound, AcsSessionID=npf-sjca-pdp01/195491152/2085007, SelectedAccessService=NDAC_SGT_Service, FailureReason=11302 Received Secure RADIUS request without a cts-pac-opaque cisco-av-pair attribute, Step=11001, Step=11017, Step=15012, Step=11302, NetworkDeviceGroups=Location#All Locations#NTN, NetworkDeviceGroups=Device Type#All Device Types#Wired, CPMSessionID=0a2a073f00005D4C53E32449, Model Name=4503, Location=Location#All Locations#NTN, Device Type=Device Type#All Device Types#Wired, Res
 ponse={RadiusPacketType=Drop; },
-Aug  6 16:43:22 10.42.7.64 Aug  7 00:02:34 npf-sjca-pdp02 CISE_Failed_Attempts 0000370885 1 0 2014-08-07 00:02:34.792 -07:00 0011267367 5405 NOTICE Failed-Attempt: RADIUS Request dropped, ConfigVersionId=240, Device IP Address=10.56.72.126, Device Port=1645, DestinationIPAddress=10.42.7.64, DestinationPort=1812, RadiusPacketType=AccessRequest, UserName=#CTSREQUEST#, Protocol=Radius, RequestLatency=4, NetworkDeviceName=ntn01-11a-sw3, User-Name=#CTSREQUEST#, NAS-IP-Address=10.56.72.126, Service-Type=Outbound, AcsSessionID=npf-sjca-pdp02/195481465/270956, SelectedAccessService=NDAC_SGT_Service, FailureReason=11302 Received Secure RADIUS request without a cts-pac-opaque cisco-av-pair attribute, Step=11001, Step=11017, Step=15012, Step=11302, NetworkDeviceGroups=Location#All Locations#NTN, NetworkDeviceGroups=Device Type#All Device Types#Wired, CPMSessionID=0a2a0740000056F353E3248A, Model Name=4503, Location=Location#All Locations#NTN, Device Type=Device Type#All Device Types#Wired, Resp
 onse={RadiusPacketType=Drop; },
-Aug  6 16:43:27 10.42.7.63 Aug  7 00:02:39 npf-sjca-pdp01 CISE_Failed_Attempts 0001970043 1 0 2014-08-07 00:02:39.808 -07:00 0098657578 5405 NOTICE Failed-Attempt: RADIUS Request dropped, ConfigVersionId=133, Device IP Address=10.56.72.126, Device Port=1645, DestinationIPAddress=10.42.7.63, DestinationPort=1812, RadiusPacketType=AccessRequest, UserName=#CTSREQUEST#, Protocol=Radius, RequestLatency=1, NetworkDeviceName=ntn01-11a-sw3, User-Name=#CTSREQUEST#, NAS-IP-Address=10.56.72.126, Service-Type=Outbound, AcsSessionID=npf-sjca-pdp01/195491152/2085161, SelectedAccessService=NDAC_SGT_Service, FailureReason=11302 Received Secure RADIUS request without a cts-pac-opaque cisco-av-pair attribute, Step=11001, Step=11017, Step=15012, Step=11302, NetworkDeviceGroups=Location#All Locations#NTN, NetworkDeviceGroups=Device Type#All Device Types#Wired, CPMSessionID=0a2a073f00005D4D53E3248F, Model Name=4503, Location=Location#All Locations#NTN, Device Type=Device Type#All Device Types#Wired, Res
 ponse={RadiusPacketType=Drop; },
-Aug  6 16:43:56 10.42.7.64 Aug  7 00:03:08 npf-sjca-pdp02 CISE_Failed_Attempts 0000370897 1 0 2014-08-07 00:03:08.902 -07:00 0011267657 5435 NOTICE RADIUS: NAS conducted several failed authentications of the same scenario, ConfigVersionId=240, Device IP Address=10.56.129.4, Device Port=32770, DestinationIPAddress=10.42.7.64, DestinationPort=1813, RadiusPacketType=AccountingRequest, UserName=yshchory, Protocol=Radius, RequestLatency=49, NetworkDeviceName=NTN-WLC1, User-Name=yshchory, NAS-IP-Address=10.56.129.4, NAS-Port=1, Framed-IP-Address=10.56.129.141, Class=CACS:0a388104000045cd53e2be75:npf-sjca-pdp02/195481465/270958, Called-Station-ID=6c-41-6a-5f-6e-c0, Calling-Station-ID=90-18-7c-7b-59-01, NAS-Identifier=ntn01-11a-wlc1, Acct-Status-Type=Interim-Update, Acct-Delay-Time=0, Acct-Input-Octets=2359603, Acct-Output-Octets=26928466, Acct-Session-Id=53e2be78/90:18:7c:7b:59:01/13844, Acct-Authentic=RADIUS, Acct-Session-Time=1466, Acct-Input-Packets=14866, Acct-Output-Packets=23043, und
 efined-52=
-Aug  6 16:44:01 10.42.7.63 Aug  7 00:03:13 npf-sjca-pdp01 CISE_Failed_Attempts 0001970072 1 0 2014-08-07 00:03:13.112 -07:00 0098658804 5435 NOTICE RADIUS: NAS conducted several failed authentications of the same scenario, ConfigVersionId=133, Device IP Address=10.56.72.127, Device Port=1646, DestinationIPAddress=10.42.7.63, DestinationPort=1813, Protocol=Radius, NetworkDeviceName=ntn01-11a-sw4, User-Name=host/salfi-pc.cisco.com, NAS-IP-Address=10.56.72.127, NAS-Port=50212, Service-Type=Framed, Framed-IP-Address=10.56.111.14, Class=CACS:0A38487F00000397BDA7BCAC:npf-sjca-pdp02/195481465/270957, Called-Station-ID=00-26-99-28-5E-BB, Calling-Station-ID=3C-97-0E-C3-F8-F1, Acct-Status-Type=Interim-Update, Acct-Delay-Time=4, Acct-Input-Octets=225395, Acct-Output-Octets=761436, Acct-Session-Id=00000560, Acct-Authentic=RADIUS, Acct-Session-Time=43, Acct-Input-Packets=1163, Acct-Output-Packets=1080, NAS-Port-Type=Ethernet, NAS-Port-Id=GigabitEthernet2/12, undefined-151=F54C88B0, cisco-av-pair
 =audit-session-id=0A38487F00000397BDA7BCAC, cisco-av-pair=connect-progress=Auth Open, AcsSessionID=npf-sjca-pdp01/195491152/2085221, FailureReason=11038 RADIUS Accounting-Request header contains invalid Authenticator field, Step=11004, Step=11017, Step=11038, Step=5435, NetworkDeviceGroups=Location#All Locations#NTN, NetworkDeviceGroups=Device Type#All Device Types#Wired, CPMSessionID=0A38487F00000397BDA7BCAC, TotalFailedAttempts=2, TotalFailedTime=42, Model Name=4503, Location=Location#All Locations#NTN, Device Type=Device Type#All Device Types#Wired, 
-Aug  6 16:44:32 10.42.7.64 Aug  7 00:03:44 npf-sjca-pdp02 CISE_Failed_Attempts 0000370899 1 0 2014-08-07 00:03:44.851 -07:00 0011267663 5405 NOTICE Failed-Attempt: RADIUS Request dropped, ConfigVersionId=240, Device IP Address=10.56.72.126, Device Port=1645, DestinationIPAddress=10.42.7.64, DestinationPort=1812, RadiusPacketType=AccessRequest, UserName=#CTSREQUEST#, Protocol=Radius, RequestLatency=7, NetworkDeviceName=ntn01-11a-sw3, User-Name=#CTSREQUEST#, NAS-IP-Address=10.56.72.126, Service-Type=Outbound, AcsSessionID=npf-sjca-pdp02/195481465/270963, SelectedAccessService=NDAC_SGT_Service, FailureReason=11302 Received Secure RADIUS request without a cts-pac-opaque cisco-av-pair attribute, Step=11001, Step=11017, Step=15012, Step=11302, NetworkDeviceGroups=Location#All Locations#NTN, NetworkDeviceGroups=Device Type#All Device Types#Wired, CPMSessionID=0a2a0740000056F453E324D0, Model Name=4503, Location=Location#All Locations#NTN, Device Type=Device Type#All Device Types#Wired, Resp
 onse={RadiusPacketType=Drop; },
-Aug  6 16:44:36 10.34.84.145 Aug  7 00:03:48 stage-pdp01 CISE_Failed_Attempts 0000024632 1 0 2014-08-07 00:03:48.375 -07:00 0000287084 5405 NOTICE Failed-Attempt: RADIUS Request dropped, ConfigVersionId=113, Device IP Address=172.23.91.132, Device Port=32769, DestinationIPAddress=10.34.84.145, DestinationPort=1812, Protocol=Radius, User-Name=test, NAS-IP-Address=192.168.30.11, Service-Type=NAS Prompt, NAS-Identifier=Cisco_1b:e0:84, AcsSessionID=stage-pdp01/196593288/19329, FailureReason=11007 Could not locate Network Device or AAA Client, Step=11001, Step=11017, Step=11007, Step=5405, 
-Aug  6 16:44:37 10.42.7.63 Aug  7 00:03:49 npf-sjca-pdp01 CISE_Failed_Attempts 0001970128 1 0 2014-08-07 00:03:49.893 -07:00 0098661643 5405 NOTICE Failed-Attempt: RADIUS Request dropped, ConfigVersionId=133, Device IP Address=10.56.72.126, Device Port=1645, DestinationIPAddress=10.42.7.63, DestinationPort=1812, RadiusPacketType=AccessRequest, UserName=#CTSREQUEST#, Protocol=Radius, RequestLatency=1, NetworkDeviceName=ntn01-11a-sw3, User-Name=#CTSREQUEST#, NAS-IP-Address=10.56.72.126, Service-Type=Outbound, AcsSessionID=npf-sjca-pdp01/195491152/2085307, SelectedAccessService=NDAC_SGT_Service, FailureReason=11302 Received Secure RADIUS request without a cts-pac-opaque cisco-av-pair attribute, Step=11001, Step=11017, Step=15012, Step=11302, NetworkDeviceGroups=Location#All Locations#NTN, NetworkDeviceGroups=Device Type#All Device Types#Wired, CPMSessionID=0a2a073f00005D4E53E324D5, Model Name=4503, Location=Location#All Locations#NTN, Device Type=Device Type#All Device Types#Wired, Res
 ponse={RadiusPacketType=Drop; },
-Aug  6 16:44:38 10.34.84.145 Aug  7 00:03:50 stage-pdp01 CISE_Failed_Attempts 0000024633 1 0 2014-08-07 00:03:50.379 -07:00 0000287088 5405 NOTICE Failed-Attempt: RADIUS Request dropped, ConfigVersionId=113, Device IP Address=172.23.91.132, Device Port=32769, DestinationIPAddress=10.34.84.145, DestinationPort=1812, Protocol=Radius, User-Name=test, NAS-IP-Address=192.168.30.11, Service-Type=NAS Prompt, NAS-Identifier=Cisco_1b:e0:84, AcsSessionID=stage-pdp01/196593288/19330, FailureReason=11007 Could not locate Network Device or AAA Client, Step=11001, Step=11017, Step=11007, Step=5405, 
-Aug  6 16:44:40 10.34.84.145 Aug  7 00:03:52 stage-pdp01 CISE_Failed_Attempts 0000024634 1 0 2014-08-07 00:03:52.379 -07:00 0000287092 5405 NOTICE Failed-Attempt: RADIUS Request dropped, ConfigVersionId=113, Device IP Address=172.23.91.132, Device Port=32769, DestinationIPAddress=10.34.84.145, DestinationPort=1812, Protocol=Radius, User-Name=test, NAS-IP-Address=192.168.30.11, Service-Type=NAS Prompt, NAS-Identifier=Cisco_1b:e0:84, AcsSessionID=stage-pdp01/196593288/19331, FailureReason=11007 Could not locate Network Device or AAA Client, Step=11001, Step=11017, Step=11007, Step=5405, 
-Aug  6 16:44:42 10.34.84.145 Aug  7 00:03:54 stage-pdp01 CISE_Failed_Attempts 0000024635 1 0 2014-08-07 00:03:54.387 -07:00 0000287096 5405 NOTICE Failed-Attempt: RADIUS Request dropped, ConfigVersionId=113, Device IP Address=172.23.91.132, Device Port=32769, DestinationIPAddress=10.34.84.145, DestinationPort=1812, Protocol=Radius, User-Name=test, NAS-IP-Address=192.168.30.11, Service-Type=NAS Prompt, NAS-Identifier=Cisco_1b:e0:84, AcsSessionID=stage-pdp01/196593288/19332, FailureReason=11007 Could not locate Network Device or AAA Client, Step=11001, Step=11017, Step=11007, Step=5405, 
-Aug  6 16:44:42 10.42.7.64 Aug  7 00:03:54 npf-sjca-pdp02 CISE_Failed_Attempts 0000370903 1 0 2014-08-07 00:03:54.924 -07:00 0011267670 5405 NOTICE Failed-Attempt: RADIUS Request dropped, ConfigVersionId=240, Device IP Address=10.56.72.126, Device Port=1645, DestinationIPAddress=10.42.7.64, DestinationPort=1812, RadiusPacketType=AccessRequest, UserName=#CTSREQUEST#, Protocol=Radius, RequestLatency=4, NetworkDeviceName=ntn01-11a-sw3, User-Name=#CTSREQUEST#, NAS-IP-Address=10.56.72.126, Service-Type=Outbound, AcsSessionID=npf-sjca-pdp02/195481465/270964, SelectedAccessService=NDAC_SGT_Service, FailureReason=11302 Received Secure RADIUS request without a cts-pac-opaque cisco-av-pair attribute, Step=11001, Step=11017, Step=15012, Step=11302, NetworkDeviceGroups=Location#All Locations#NTN, NetworkDeviceGroups=Device Type#All Device Types#Wired, CPMSessionID=0a2a0740000056F553E324DA, Model Name=4503, Location=Location#All Locations#NTN, Device Type=Device Type#All Device Types#Wired, Resp
 onse={RadiusPacketType=Drop; },
-Aug  6 16:44:44 10.34.84.145 Aug  7 00:03:56 stage-pdp01 CISE_Failed_Attempts 0000024636 1 0 2014-08-07 00:03:56.386 -07:00 0000287100 5405 NOTICE Failed-Attempt: RADIUS Request dropped, ConfigVersionId=113, Device IP Address=172.23.91.132, Device Port=32769, DestinationIPAddress=10.34.84.145, DestinationPort=1812, Protocol=Radius, User-Name=test, NAS-IP-Address=192.168.30.11, Service-Type=NAS Prompt, NAS-Identifier=Cisco_1b:e0:84, AcsSessionID=stage-pdp01/196593288/19333, FailureReason=11007 Could not locate Network Device or AAA Client, Step=11001, Step=11017, Step=11007, Step=5405, 
-Aug  6 16:44:46 10.34.84.145 Aug  7 00:03:58 stage-pdp01 CISE_Failed_Attempts 0000024637 1 0 2014-08-07 00:03:58.390 -07:00 0000287104 5405 NOTICE Failed-Attempt: RADIUS Request dropped, ConfigVersionId=113, Device IP Address=172.23.91.132, Device Port=32769, DestinationIPAddress=10.34.84.145, DestinationPort=1812, Protocol=Radius, User-Name=test, NAS-IP-Address=192.168.30.11, Service-Type=NAS Prompt, NAS-Identifier=Cisco_1b:e0:84, AcsSessionID=stage-pdp01/196593288/19334, FailureReason=11007 Could not locate Network Device or AAA Client, Step=11001, Step=11017, Step=11007, Step=5405, 
-Aug  6 16:44:47 10.42.7.63 Aug  7 00:03:59 npf-sjca-pdp01 CISE_Failed_Attempts 0001970140 1 0 2014-08-07 00:03:59.951 -07:00 0098662310 5405 NOTICE Failed-Attempt: RADIUS Request dropped, ConfigVersionId=133, Device IP Address=10.56.72.126, Device Port=1645, DestinationIPAddress=10.42.7.63, DestinationPort=1812, RadiusPacketType=AccessRequest, UserName=#CTSREQUEST#, Protocol=Radius, RequestLatency=1, NetworkDeviceName=ntn01-11a-sw3, User-Name=#CTSREQUEST#, NAS-IP-Address=10.56.72.126, Service-Type=Outbound, AcsSessionID=npf-sjca-pdp01/195491152/2085331, SelectedAccessService=NDAC_SGT_Service, FailureReason=11302 Received Secure RADIUS request without a cts-pac-opaque cisco-av-pair attribute, Step=11001, Step=11017, Step=15012, Step=11302, NetworkDeviceGroups=Location#All Locations#NTN, NetworkDeviceGroups=Device Type#All Device Types#Wired, CPMSessionID=0a2a073f00005D4F53E324DF, Model Name=4503, Location=Location#All Locations#NTN, Device Type=Device Type#All Device Types#Wired, Res
 ponse={RadiusPacketType=Drop; },
-Aug  6 16:44:48 10.42.7.64 Aug  7 00:04:00 npf-sjca-pdp02 CISE_Failed_Attempts 0000370905 1 0 2014-08-07 00:04:00.526 -07:00 0011267674 5435 NOTICE RADIUS: NAS conducted several failed authentications of the same scenario, ConfigVersionId=240, Device IP Address=10.56.72.127, Device Port=1646, DestinationIPAddress=10.42.7.64, DestinationPort=1813, Protocol=Radius, NetworkDeviceName=ntn01-11a-sw4, User-Name=host/salfi-pc.cisco.com, NAS-IP-Address=10.56.72.127, NAS-Port=50212, Service-Type=Framed, Framed-IP-Address=169.254.53.87, Class=CACS:0A38487F00000397BDA7BCAC:npf-sjca-pdp02/195481465/270957, Called-Station-ID=00-26-99-28-5E-BB, Calling-Station-ID=3C-97-0E-C3-F8-F1, Acct-Status-Type=Interim-Update, Acct-Delay-Time=0, Acct-Input-Octets=1458615, Acct-Output-Octets=3836368, Acct-Session-Id=00000560, Acct-Authentic=RADIUS, Acct-Session-Time=95, Acct-Input-Packets=4505, Acct-Output-Packets=5619, NAS-Port-Type=Ethernet, NAS-Port-Id=GigabitEthernet2/12, undefined-151=F54C88B0, cisco-av-p
 air=audit-session-id=0A38487F00000397BDA7BCAC, cisco-av-pair=connect-progress=Auth Open, AcsSessionID=npf-sjca-pdp02/195481465/270965, FailureReason=11038 RADIUS Accounting-Request header contains invalid Authenticator field, Step=11004, Step=11017, Step=11038, Step=5435, NetworkDeviceGroups=Location#All Locations#NTN, NetworkDeviceGroups=Device Type#All Device Types#Wired, CPMSessionID=0A38487F00000397BDA7BCAC, TotalFailedAttempts=2, TotalFailedTime=52, Model Name=4503, Location=Location#All Locations#NTN, Device Type=Device Type#All Device Types#Wired, 
-Aug  6 16:45:52 10.42.7.64 Aug  7 00:05:04 npf-sjca-pdp02 CISE_Failed_Attempts 0000370920 1 0 2014-08-07 00:05:04.969 -07:00 0011267987 5405 NOTICE Failed-Attempt: RADIUS Request dropped, ConfigVersionId=240, Device IP Address=10.56.72.126, Device Port=1645, DestinationIPAddress=10.42.7.64, DestinationPort=1812, RadiusPacketType=AccessRequest, UserName=#CTSREQUEST#, Protocol=Radius, RequestLatency=6, NetworkDeviceName=ntn01-11a-sw3, User-Name=#CTSREQUEST#, NAS-IP-Address=10.56.72.126, Service-Type=Outbound, AcsSessionID=npf-sjca-pdp02/195481465/270977, SelectedAccessService=NDAC_SGT_Service, FailureReason=11302 Received Secure RADIUS request without a cts-pac-opaque cisco-av-pair attribute, Step=11001, Step=11017, Step=15012, Step=11302, NetworkDeviceGroups=Location#All Locations#NTN, NetworkDeviceGroups=Device Type#All Device Types#Wired, CPMSessionID=0a2a0740000056F653E32520, Model Name=4503, Location=Location#All Locations#NTN, Device Type=Device Type#All Device Types#Wired, Resp
 onse={RadiusPacketType=Drop; },
-Aug  6 16:45:58 10.42.7.63 Aug  7 00:05:09 npf-sjca-pdp01 CISE_Failed_Attempts 0001970212 1 0 2014-08-07 00:05:09.998 -07:00 0098665518 5405 NOTICE Failed-Attempt: RADIUS Request dropped, ConfigVersionId=133, Device IP Address=10.56.72.126, Device Port=1645, DestinationIPAddress=10.42.7.63, DestinationPort=1812, RadiusPacketType=AccessRequest, UserName=#CTSREQUEST#, Protocol=Radius, RequestLatency=1, NetworkDeviceName=ntn01-11a-sw3, User-Name=#CTSREQUEST#, NAS-IP-Address=10.56.72.126, Service-Type=Outbound, AcsSessionID=npf-sjca-pdp01/195491152/2085460, SelectedAccessService=NDAC_SGT_Service, FailureReason=11302 Received Secure RADIUS request without a cts-pac-opaque cisco-av-pair attribute, Step=11001, Step=11017, Step=15012, Step=11302, NetworkDeviceGroups=Location#All Locations#NTN, NetworkDeviceGroups=Device Type#All Device Types#Wired, CPMSessionID=0a2a073f00005D5053E32525, Model Name=4503, Location=Location#All Locations#NTN, Device Type=Device Type#All Device Types#Wired, Res
 ponse={RadiusPacketType=Drop; },
-Aug  6 16:47:03 10.42.7.64 Aug  7 00:06:15 npf-sjca-pdp02 CISE_Failed_Attempts 0000370931 1 0 2014-08-07 00:06:15.016 -07:00 0011268196 5405 NOTICE Failed-Attempt: RADIUS Request dropped, ConfigVersionId=240, Device IP Address=10.56.72.126, Device Port=1645, DestinationIPAddress=10.42.7.64, DestinationPort=1812, RadiusPacketType=AccessRequest, UserName=#CTSREQUEST#, Protocol=Radius, RequestLatency=5, NetworkDeviceName=ntn01-11a-sw3, User-Name=#CTSREQUEST#, NAS-IP-Address=10.56.72.126, Service-Type=Outbound, AcsSessionID=npf-sjca-pdp02/195481465/270985, SelectedAccessService=NDAC_SGT_Service, FailureReason=11302 Received Secure RADIUS request without a cts-pac-opaque cisco-av-pair attribute, Step=11001, Step=11017, Step=15012, Step=11302, NetworkDeviceGroups=Location#All Locations#NTN, NetworkDeviceGroups=Device Type#All Device Types#Wired, CPMSessionID=0a2a0740000056F753E32567, Model Name=4503, Location=Location#All Locations#NTN, Device Type=Device Type#All Device Types#Wired, Resp
 onse={RadiusPacketType=Drop; },
-Aug  6 16:47:08 10.42.7.63 Aug  7 00:06:20 npf-sjca-pdp01 CISE_Failed_Attempts 0001970324 1 0 2014-08-07 00:06:20.055 -07:00 0098669942 5405 NOTICE Failed-Attempt: RADIUS Request dropped, ConfigVersionId=133, Device IP Address=10.56.72.126, Device Port=1645, DestinationIPAddress=10.42.7.63, DestinationPort=1812, RadiusPacketType=AccessRequest, UserName=#CTSREQUEST#, Protocol=Radius, RequestLatency=2, NetworkDeviceName=ntn01-11a-sw3, User-Name=#CTSREQUEST#, NAS-IP-Address=10.56.72.126, Service-Type=Outbound, AcsSessionID=npf-sjca-pdp01/195491152/2085599, SelectedAccessService=NDAC_SGT_Service, FailureReason=11302 Received Secure RADIUS request without a cts-pac-opaque cisco-av-pair attribute, Step=11001, Step=11017, Step=15012, Step=11302, NetworkDeviceGroups=Location#All Locations#NTN, NetworkDeviceGroups=Device Type#All Device Types#Wired, CPMSessionID=0a2a073f00005D5153E3256C, Model Name=4503, Location=Location#All Locations#NTN, Device Type=Device Type#All Device Types#Wired, Res
 ponse={RadiusPacketType=Drop; },
-Aug  6 16:47:13 10.42.7.64 Aug  7 00:06:25 npf-sjca-pdp02 CISE_Failed_Attempts 0000370934 1 0 2014-08-07 00:06:25.097 -07:00 0011268209 5405 NOTICE Failed-Attempt: RADIUS Request dropped, ConfigVersionId=240, Device IP Address=10.56.72.126, Device Port=1645, DestinationIPAddress=10.42.7.64, DestinationPort=1812, RadiusPacketType=AccessRequest, UserName=#CTSREQUEST#, Protocol=Radius, RequestLatency=5, NetworkDeviceName=ntn01-11a-sw3, User-Name=#CTSREQUEST#, NAS-IP-Address=10.56.72.126, Service-Type=Outbound, AcsSessionID=npf-sjca-pdp02/195481465/270987, SelectedAccessService=NDAC_SGT_Service, FailureReason=11302 Received Secure RADIUS request without a cts-pac-opaque cisco-av-pair attribute, Step=11001, Step=11017, Step=15012, Step=11302, NetworkDeviceGroups=Location#All Locations#NTN, NetworkDeviceGroups=Device Type#All Device Types#Wired, CPMSessionID=0a2a0740000056F853E32571, Model Name=4503, Location=Location#All Locations#NTN, Device Type=Device Type#All Device Types#Wired, Resp
 onse={RadiusPacketType=Drop; },
-Aug  6 16:47:18 10.42.7.63 Aug  7 00:06:30 npf-sjca-pdp01 CISE_Failed_Attempts 0001970335 1 0 2014-08-07 00:06:30.119 -07:00 0098670037 5405 NOTICE Failed-Attempt: RADIUS Request dropped, ConfigVersionId=133, Device IP Address=10.56.72.126, Device Port=1645, DestinationIPAddress=10.42.7.63, DestinationPort=1812, RadiusPacketType=AccessRequest, UserName=#CTSREQUEST#, Protocol=Radius, RequestLatency=2, NetworkDeviceName=ntn01-11a-sw3, User-Name=#CTSREQUEST#, NAS-IP-Address=10.56.72.126, Service-Type=Outbound, AcsSessionID=npf-sjca-pdp01/195491152/2085618, SelectedAccessService=NDAC_SGT_Service, FailureReason=11302 Received Secure RADIUS request without a cts-pac-opaque cisco-av-pair attribute, Step=11001, Step=11017, Step=15012, Step=11302, NetworkDeviceGroups=Location#All Locations#NTN, NetworkDeviceGroups=Device Type#All Device Types#Wired, CPMSessionID=0a2a073f00005D5253E32576, Model Name=4503, Location=Location#All Locations#NTN, Device Type=Device Type#All Device Types#Wired, Res
 ponse={RadiusPacketType=Drop; },
-Aug  6 16:47:48 10.34.84.145 Aug  7 00:07:00 stage-pdp01 CISE_Failed_Attempts 0000024649 1 0 2014-08-07 00:07:00.418 -07:00 0000287210 5405 NOTICE Failed-Attempt: RADIUS Request dropped, ConfigVersionId=113, Device IP Address=172.23.91.132, Device Port=32769, DestinationIPAddress=10.34.84.145, DestinationPort=1812, Protocol=Radius, User-Name=test, NAS-IP-Address=192.168.30.11, Service-Type=NAS Prompt, NAS-Identifier=Cisco_1b:e0:84, AcsSessionID=stage-pdp01/196593288/19342, FailureReason=11007 Could not locate Network Device or AAA Client, Step=11001, Step=11017, Step=11007, Step=5405, 
-Aug  6 16:47:50 10.34.84.145 Aug  7 00:07:02 stage-pdp01 CISE_Failed_Attempts 0000024650 1 0 2014-08-07 00:07:02.421 -07:00 0000287214 5405 NOTICE Failed-Attempt: RADIUS Request dropped, ConfigVersionId=113, Device IP Address=172.23.91.132, Device Port=32769, DestinationIPAddress=10.34.84.145, DestinationPort=1812, Protocol=Radius, User-Name=test, NAS-IP-Address=192.168.30.11, Service-Type=NAS Prompt, NAS-Identifier=Cisco_1b:e0:84, AcsSessionID=stage-pdp01/196593288/19343, FailureReason=11007 Could not locate Network Device or AAA Client, Step=11001, Step=11017, Step=11007, Step=5405, 
-Aug  6 16:47:52 10.34.84.145 Aug  7 00:07:04 stage-pdp01 CISE_Failed_Attempts 0000024651 1 0 2014-08-07 00:07:04.425 -07:00 0000287218 5405 NOTICE Failed-Attempt: RADIUS Request dropped, ConfigVersionId=113, Device IP Address=172.23.91.132, Device Port=32769, DestinationIPAddress=10.34.84.145, DestinationPort=1812, Protocol=Radius, User-Name=test, NAS-IP-Address=192.168.30.11, Service-Type=NAS Prompt, NAS-Identifier=Cisco_1b:e0:84, AcsSessionID=stage-pdp01/196593288/19344, FailureReason=11007 Could not locate Network Device or AAA Client, Step=11001, Step=11017, Step=11007, Step=5405, 
-Aug  6 16:47:54 10.34.84.145 Aug  7 00:07:06 stage-pdp01 CISE_Failed_Attempts 0000024652 1 0 2014-08-07 00:07:06.429 -07:00 0000287222 5405 NOTICE Failed-Attempt: RADIUS Request dropped, ConfigVersionId=113, Device IP Address=172.23.91.132, Device Port=32769, DestinationIPAddress=10.34.84.145, DestinationPort=1812, Protocol=Radius, User-Name=test, NAS-IP-Address=192.168.30.11, Service-Type=NAS Prompt, NAS-Identifier=Cisco_1b:e0:84, AcsSessionID=stage-pdp01/196593288/19345, FailureReason=11007 Could not locate Network Device or AAA Client, Step=11001, Step=11017, Step=11007, Step=5405, 
-Aug  6 16:47:56 10.34.84.145 Aug  7 00:07:08 stage-pdp01 CISE_Failed_Attempts 0000024653 1 0 2014-08-07 00:07:08.429 -07:00 0000287226 5405 NOTICE Failed-Attempt: RADIUS Request dropped, ConfigVersionId=113, Device IP Address=172.23.91.132, Device Port=32769, DestinationIPAddress=10.34.84.145, DestinationPort=1812, Protocol=Radius, User-Name=test, NAS-IP-Address=192.168.30.11, Service-Type=NAS Prompt, NAS-Identifier=Cisco_1b:e0:84, AcsSessionID=stage-pdp01/196593288/19346, FailureReason=11007 Could not locate Network Device or AAA Client, Step=11001, Step=11017, Step=11007, Step=5405, 
-Aug  6 16:47:58 10.34.84.145 Aug  7 00:07:10 stage-pdp01 CISE_Failed_Attempts 0000024654 1 0 2014-08-07 00:07:10.433 -07:00 0000287230 5405 NOTICE Failed-Attempt: RADIUS Request dropped, ConfigVersionId=113, Device IP Address=172.23.91.132, Device Port=32769, DestinationIPAddress=10.34.84.145, DestinationPort=1812, Protocol=Radius, User-Name=test, NAS-IP-Address=192.168.30.11, Service-Type=NAS Prompt, NAS-Identifier=Cisco_1b:e0:84, AcsSessionID=stage-pdp01/196593288/19347, FailureReason=11007 Could not locate Network Device or AAA Client, Step=11001, Step=11017, Step=11007, Step=5405, 
-Aug  6 16:48:23 10.42.7.64 Aug  7 00:07:35 npf-sjca-pdp02 CISE_Failed_Attempts 0000370955 1 0 2014-08-07 00:07:35.138 -07:00 0011268472 5405 NOTICE Failed-Attempt: RADIUS Request dropped, ConfigVersionId=240, Device IP Address=10.56.72.126, Device Port=1645, DestinationIPAddress=10.42.7.64, DestinationPort=1812, RadiusPacketType=AccessRequest, UserName=#CTSREQUEST#, Protocol=Radius, RequestLatency=5, NetworkDeviceName=ntn01-11a-sw3, User-Name=#CTSREQUEST#, NAS-IP-Address=10.56.72.126, Service-Type=Outbound, AcsSessionID=npf-sjca-pdp02/195481465/271001, SelectedAccessService=NDAC_SGT_Service, FailureReason=11302 Received Secure RADIUS request without a cts-pac-opaque cisco-av-pair attribute, Step=11001, Step=11017, Step=15012, Step=11302, NetworkDeviceGroups=Location#All Locations#NTN, NetworkDeviceGroups=Device Type#All Device Types#Wired, CPMSessionID=0a2a0740000056F953E325B7, Model Name=4503, Location=Location#All Locations#NTN, Device Type=Device Type#All Device Types#Wired, Resp
 onse={RadiusPacketType=Drop; },
-Aug  6 16:48:28 10.42.7.63 Aug  7 00:07:40 npf-sjca-pdp01 CISE_Failed_Attempts 0001970420 1 0 2014-08-07 00:07:40.178 -07:00 0098673462 5405 NOTICE Failed-Attempt: RADIUS Request dropped, ConfigVersionId=133, Device IP Address=10.56.72.126, Device Port=1645, DestinationIPAddress=10.42.7.63, DestinationPort=1812, RadiusPacketType=AccessRequest, UserName=#CTSREQUEST#, Protocol=Radius, RequestLatency=1, NetworkDeviceName=ntn01-11a-sw3, User-Name=#CTSREQUEST#, NAS-IP-Address=10.56.72.126, Service-Type=Outbound, AcsSessionID=npf-sjca-pdp01/195491152/2085757, SelectedAccessService=NDAC_SGT_Service, FailureReason=11302 Received Secure RADIUS request without a cts-pac-opaque cisco-av-pair attribute, Step=11001, Step=11017, Step=15012, Step=11302, NetworkDeviceGroups=Location#All Locations#NTN, NetworkDeviceGroups=Device Type#All Device Types#Wired, CPMSessionID=0a2a073f00005D5353E325BC, Model Name=4503, Location=Location#All Locations#NTN, Device Type=Device Type#All Device Types#Wired, Res
 ponse={RadiusPacketType=Drop; },
-Aug  6 16:49:33 10.42.7.64 Aug  7 00:08:45 npf-sjca-pdp02 CISE_Failed_Attempts 0000370984 1 0 2014-08-07 00:08:45.219 -07:00 0011269071 5405 NOTICE Failed-Attempt: RADIUS Request dropped, ConfigVersionId=240, Device IP Address=10.56.72.126, Device Port=1645, DestinationIPAddress=10.42.7.64, DestinationPort=1812, RadiusPacketType=AccessRequest, UserName=#CTSREQUEST#, Protocol=Radius, RequestLatency=5, NetworkDeviceName=ntn01-11a-sw3, User-Name=#CTSREQUEST#, NAS-IP-Address=10.56.72.126, Service-Type=Outbound, AcsSessionID=npf-sjca-pdp02/195481465/271016, SelectedAccessService=NDAC_SGT_Service, FailureReason=11302 Received Secure RADIUS request without a cts-pac-opaque cisco-av-pair attribute, Step=11001, Step=11017, Step=15012, Step=11302, NetworkDeviceGroups=Location#All Locations#NTN, NetworkDeviceGroups=Device Type#All Device Types#Wired, CPMSessionID=0a2a0740000056FB53E325FD, Model Name=4503, Location=Location#All Locations#NTN, Device Type=Device Type#All Device Types#Wired, Resp
 onse={RadiusPacketType=Drop; },
-Aug  6 16:49:38 10.42.7.63 Aug  7 00:08:50 npf-sjca-pdp01 CISE_Failed_Attempts 0001970519 1 0 2014-08-07 00:08:50.259 -07:00 0098677825 5405 NOTICE Failed-Attempt: RADIUS Request dropped, ConfigVersionId=133, Device IP Address=10.56.72.126, Device Port=1645, DestinationIPAddress=10.42.7.63, DestinationPort=1812, RadiusPacketType=AccessRequest, UserName=#CTSREQUEST#, Protocol=Radius, RequestLatency=2, NetworkDeviceName=ntn01-11a-sw3, User-Name=#CTSREQUEST#, NAS-IP-Address=10.56.72.126, Service-Type=Outbound, AcsSessionID=npf-sjca-pdp01/195491152/2085892, SelectedAccessService=NDAC_SGT_Service, FailureReason=11302 Received Secure RADIUS request without a cts-pac-opaque cisco-av-pair attribute, Step=11001, Step=11017, Step=15012, Step=11302, NetworkDeviceGroups=Location#All Locations#NTN, NetworkDeviceGroups=Device Type#All Device Types#Wired, CPMSessionID=0a2a073f00005D5453E32602, Model Name=4503, Location=Location#All Locations#NTN, Device Type=Device Type#All Device Types#Wired, Res
 ponse={RadiusPacketType=Drop; },
-Aug  6 16:49:43 10.42.7.64 Aug  7 00:08:55 npf-sjca-pdp02 CISE_Failed_Attempts 0000370986 1 0 2014-08-07 00:08:55.298 -07:00 0011269076 5405 NOTICE Failed-Attempt: RADIUS Request dropped, ConfigVersionId=240, Device IP Address=10.56.72.126, Device Port=1645, DestinationIPAddress=10.42.7.64, DestinationPort=1812, RadiusPacketType=AccessRequest, UserName=#CTSREQUEST#, Protocol=Radius, RequestLatency=5, NetworkDeviceName=ntn01-11a-sw3, User-Name=#CTSREQUEST#, NAS-IP-Address=10.56.72.126, Service-Type=Outbound, AcsSessionID=npf-sjca-pdp02/195481465/271017, SelectedAccessService=NDAC_SGT_Service, FailureReason=11302 Received Secure RADIUS request without a cts-pac-opaque cisco-av-pair attribute, Step=11001, Step=11017, Step=15012, Step=11302, NetworkDeviceGroups=Location#All Locations#NTN, NetworkDeviceGroups=Device Type#All Device Types#Wired, CPMSessionID=0a2a0740000056FC53E32607, Model Name=4503, Location=Location#All Locations#NTN, Device Type=Device Type#All Device Types#Wired, Resp
 onse={RadiusPacketType=Drop; },
-Aug  6 16:49:48 10.42.7.63 Aug  7 00:09:00 npf-sjca-pdp01 CISE_Failed_Attempts 0001970524 1 0 2014-08-07 00:09:00.330 -07:00 0098678019 5405 NOTICE Failed-Attempt: RADIUS Request dropped, ConfigVersionId=133, Device IP Address=10.56.72.126, Device Port=1645, DestinationIPAddress=10.42.7.63, DestinationPort=1812, RadiusPacketType=AccessRequest, UserName=#CTSREQUEST#, Protocol=Radius, RequestLatency=2, NetworkDeviceName=ntn01-11a-sw3, User-Name=#CTSREQUEST#, NAS-IP-Address=10.56.72.126, Service-Type=Outbound, AcsSessionID=npf-sjca-pdp01/195491152/2085909, SelectedAccessService=NDAC_SGT_Service, FailureReason=11302 Received Secure RADIUS request without a cts-pac-opaque cisco-av-pair attribute, Step=11001, Step=11017, Step=15012, Step=11302, NetworkDeviceGroups=Location#All Locations#NTN, NetworkDeviceGroups=Device Type#All Device Types#Wired, CPMSessionID=0a2a073f00005D5553E3260C, Model Name=4503, Location=Location#All Locations#NTN, Device Type=Device Type#All Device Types#Wired, Res
 ponse={RadiusPacketType=Drop; },
-Aug  6 16:50:53 10.42.7.64 Aug  7 00:10:05 npf-sjca-pdp02 CISE_Failed_Attempts 0000370999 1 0 2014-08-07 00:10:05.339 -07:00 0011269371 5405 NOTICE Failed-Attempt: RADIUS Request dropped, ConfigVersionId=240, Device IP Address=10.56.72.126, Device Port=1645, DestinationIPAddress=10.42.7.64, DestinationPort=1812, RadiusPacketType=AccessRequest, UserName=#CTSREQUEST#, Protocol=Radius, RequestLatency=5, NetworkDeviceName=ntn01-11a-sw3, User-Name=#CTSREQUEST#, NAS-IP-Address=10.56.72.126, Service-Type=Outbound, AcsSessionID=npf-sjca-pdp02/195481465/271027, SelectedAccessService=NDAC_SGT_Service, FailureReason=11302 Received Secure RADIUS request without a cts-pac-opaque cisco-av-pair attribute, Step=11001, Step=11017, Step=15012, Step=11302, NetworkDeviceGroups=Location#All Locations#NTN, NetworkDeviceGroups=Device Type#All Device Types#Wired, CPMSessionID=0a2a0740000056FD53E3264D, Model Name=4503, Location=Location#All Locations#NTN, Device Type=Device Type#All Device Types#Wired, Resp
 onse={RadiusPacketType=Drop; },
-Aug  6 16:50:58 10.42.7.63 Aug  7 00:10:10 npf-sjca-pdp01 CISE_Failed_Attempts 0001970625 1 0 2014-08-07 00:10:10.388 -07:00 0098682297 5405 NOTICE Failed-Attempt: RADIUS Request dropped, ConfigVersionId=133, Device IP Address=10.56.72.126, Device Port=1645, DestinationIPAddress=10.42.7.63, DestinationPort=1812, RadiusPacketType=AccessRequest, UserName=#CTSREQUEST#, Protocol=Radius, RequestLatency=2, NetworkDeviceName=ntn01-11a-sw3, User-Name=#CTSREQUEST#, NAS-IP-Address=10.56.72.126, Service-Type=Outbound, AcsSessionID=npf-sjca-pdp01/195491152/2086061, SelectedAccessService=NDAC_SGT_Service, FailureReason=11302 Received Secure RADIUS request without a cts-pac-opaque cisco-av-pair attribute, Step=11001, Step=11017, Step=15012, Step=11302, NetworkDeviceGroups=Location#All Locations#NTN, NetworkDeviceGroups=Device Type#All Device Types#Wired, CPMSessionID=0a2a073f00005D5653E32652, Model Name=4503, Location=Location#All Locations#NTN, Device Type=Device Type#All Device Types#Wired, Res
 ponse={RadiusPacketType=Drop; },
-Aug  6 16:51:00 10.34.84.145 Aug  7 00:10:12 stage-pdp01 CISE_Failed_Attempts 0000024661 1 0 2014-08-07 00:10:12.492 -07:00 0000287258 5405 NOTICE Failed-Attempt: RADIUS Request dropped, ConfigVersionId=113, Device IP Address=172.23.91.132, Device Port=32769, DestinationIPAddress=10.34.84.145, DestinationPort=1812, Protocol=Radius, User-Name=test, NAS-IP-Address=192.168.30.11, Service-Type=NAS Prompt, NAS-Identifier=Cisco_1b:e0:84, AcsSessionID=stage-pdp01/196593288/19354, FailureReason=11007 Could not locate Network Device or AAA Client, Step=11001, Step=11017, Step=11007, Step=5405, 
-Aug  6 16:51:02 10.34.84.145 Aug  7 00:10:14 stage-pdp01 CISE_Failed_Attempts 0000024662 1 0 2014-08-07 00:10:14.496 -07:00 0000287262 5405 NOTICE Failed-Attempt: RADIUS Request dropped, ConfigVersionId=113, Device IP Address=172.23.91.132, Device Port=32769, DestinationIPAddress=10.34.84.145, DestinationPort=1812, Protocol=Radius, User-Name=test, NAS-IP-Address=192.168.30.11, Service-Type=NAS Prompt, NAS-Identifier=Cisco_1b:e0:84, AcsSessionID=stage-pdp01/196593288/19355, FailureReason=11007 Could not locate Network Device or AAA Client, Step=11001, Step=11017, Step=11007, Step=5405, 
-Aug  6 16:51:04 10.34.84.145 Aug  7 00:10:16 stage-pdp01 CISE_Failed_Attempts 0000024663 1 0 2014-08-07 00:10:16.496 -07:00 0000287266 5405 NOTICE Failed-Attempt: RADIUS Request dropped, ConfigVersionId=113, Device IP Address=172.23.91.132, Device Port=32769, DestinationIPAddress=10.34.84.145, DestinationPort=1812, Protocol=Radius, User-Name=test, NAS-IP-Address=192.168.30.11, Service-Type=NAS Prompt, NAS-Identifier=Cisco_1b:e0:84, AcsSessionID=stage-pdp01/196593288/19356, FailureReason=11007 Could not locate Network Device or AAA Client, Step=11001, Step=11017, Step=11007, Step=5405, 
-Aug  6 16:51:06 10.34.84.145 Aug  7 00:10:18 stage-pdp01 CISE_Failed_Attempts 0000024664 1 0 2014-08-07 00:10:18.500 -07:00 0000287270 5405 NOTICE Failed-Attempt: RADIUS Request dropped, ConfigVersionId=113, Device IP Address=172.23.91.132, Device Port=32769, DestinationIPAddress=10.34.84.145, DestinationPort=1812, Protocol=Radius, User-Name=test, NAS-IP-Address=192.168.30.11, Service-Type=NAS Prompt, NAS-Identifier=Cisco_1b:e0:84, AcsSessionID=stage-pdp01/196593288/19357, FailureReason=11007 Could not locate Network Device or AAA Client, Step=11001, Step=11017, Step=11007, Step=5405, 
-Aug  6 16:51:08 10.34.84.145 Aug  7 00:10:20 stage-pdp01 CISE_Failed_Attempts 0000024665 1 0 2014-08-07 00:10:20.504 -07:00 0000287274 5405 NOTICE Failed-Attempt: RADIUS Request dropped, ConfigVersionId=113, Device IP Address=172.23.91.132, Device Port=32769, DestinationIPAddress=10.34.84.145, DestinationPort=1812, Protocol=Radius, User-Name=test, NAS-IP-Address=192.168.30.11, Service-Type=NAS Prompt, NAS-Identifier=Cisco_1b:e0:84, AcsSessionID=stage-pdp01/196593288/19358, FailureReason=11007 Could not locate Network Device or AAA Client, Step=11001, Step=11017, Step=11007, Step=5405, 
-Aug  6 16:51:10 10.34.84.145 Aug  7 00:10:22 stage-pdp01 CISE_Failed_Attempts 0000024667 1 0 2014-08-07 00:10:22.507 -07:00 0000287279 5405 NOTICE Failed-Attempt: RADIUS Request dropped, ConfigVersionId=113, Device IP Address=172.23.91.132, Device Port=32769, DestinationIPAddress=10.34.84.145, DestinationPort=1812, Protocol=Radius, User-Name=test, NAS-IP-Address=192.168.30.11, Service-Type=NAS Prompt, NAS-Identifier=Cisco_1b:e0:84, AcsSessionID=stage-pdp01/196593288/19359, FailureReason=11007 Could not locate Network Device or AAA Client, Step=11001, Step=11017, Step=11007, Step=5405, 
-Aug  6 16:52:03 10.42.7.64 Aug  7 00:11:15 npf-sjca-pdp02 CISE_Failed_Attempts 0000371005 1 0 2014-08-07 00:11:15.432 -07:00 0011269421 5405 NOTICE Failed-Attempt: RADIUS Request dropped, ConfigVersionId=240, Device IP Address=10.56.72.126, Device Port=1645, DestinationIPAddress=10.42.7.64, DestinationPort=1812, RadiusPacketType=AccessRequest, UserName=#CTSREQUEST#, Protocol=Radius, RequestLatency=4, NetworkDeviceName=ntn01-11a-sw3, User-Name=#CTSREQUEST#, NAS-IP-Address=10.56.72.126, Service-Type=Outbound, AcsSessionID=npf-sjca-pdp02/195481465/271031, SelectedAccessService=NDAC_SGT_Service, FailureReason=11302 Received Secure RADIUS request without a cts-pac-opaque cisco-av-pair attribute, Step=11001, Step=11017, Step=15012, Step=11302, NetworkDeviceGroups=Location#All Locations#NTN, NetworkDeviceGroups=Device Type#All Device Types#Wired, CPMSessionID=0a2a0740000056FE53E32693, Model Name=4503, Location=Location#All Locations#NTN, Device Type=Device Type#All Device Types#Wired, Resp
 onse={RadiusPacketType=Drop; },
-Aug  6 16:52:08 10.42.7.63 Aug  7 00:11:20 npf-sjca-pdp01 CISE_Failed_Attempts 0001970691 1 0 2014-08-07 00:11:20.468 -07:00 0098685176 5405 NOTICE Failed-Attempt: RADIUS Request dropped, ConfigVersionId=133, Device IP Address=10.56.72.126, Device Port=1645, DestinationIPAddress=10.42.7.63, DestinationPort=1812, RadiusPacketType=AccessRequest, UserName=#CTSREQUEST#, Protocol=Radius, RequestLatency=2, NetworkDeviceName=ntn01-11a-sw3, User-Name=#CTSREQUEST#, NAS-IP-Address=10.56.72.126, Service-Type=Outbound, AcsSessionID=npf-sjca-pdp01/195491152/2086181, SelectedAccessService=NDAC_SGT_Service, FailureReason=11302 Received Secure RADIUS request without a cts-pac-opaque cisco-av-pair attribute, Step=11001, Step=11017, Step=15012, Step=11302, NetworkDeviceGroups=Location#All Locations#NTN, NetworkDeviceGroups=Device Type#All Device Types#Wired, CPMSessionID=0a2a073f00005D5753E32698, Model Name=4503, Location=Location#All Locations#NTN, Device Type=Device Type#All Device Types#Wired, Res
 ponse={RadiusPacketType=Drop; },
-Aug  6 16:52:13 10.42.7.64 Aug  7 00:11:25 npf-sjca-pdp02 CISE_Failed_Attempts 0000371007 1 0 2014-08-07 00:11:25.515 -07:00 0011269426 5405 NOTICE Failed-Attempt: RADIUS Request dropped, ConfigVersionId=240, Device IP Address=10.56.72.126, Device Port=1645, DestinationIPAddress=10.42.7.64, DestinationPort=1812, RadiusPacketType=AccessRequest, UserName=#CTSREQUEST#, Protocol=Radius, RequestLatency=5, NetworkDeviceName=ntn01-11a-sw3, User-Name=#CTSREQUEST#, NAS-IP-Address=10.56.72.126, Service-Type=Outbound, AcsSessionID=npf-sjca-pdp02/195481465/271032, SelectedAccessService=NDAC_SGT_Service, FailureReason=11302 Received Secure RADIUS request without a cts-pac-opaque cisco-av-pair attribute, Step=11001, Step=11017, Step=15012, Step=11302, NetworkDeviceGroups=Location#All Locations#NTN, NetworkDeviceGroups=Device Type#All Device Types#Wired, CPMSessionID=0a2a0740000056FF53E3269D, Model Name=4503, Location=Location#All Locations#NTN, Device Type=Device Type#All Device Types#Wired, Resp
 onse={RadiusPacketType=Drop; },
-Aug  6 16:52:18 10.42.7.63 Aug  7 00:11:30 npf-sjca-pdp01 CISE_Failed_Attempts 0001970708 1 0 2014-08-07 00:11:30.551 -07:00 0098685669 5405 NOTICE Failed-Attempt: RADIUS Request dropped, ConfigVersionId=133, Device IP Address=10.56.72.126, Device Port=1645, DestinationIPAddress=10.42.7.63, DestinationPort=1812, RadiusPacketType=AccessRequest, UserName=#CTSREQUEST#, Protocol=Radius, RequestLatency=8, NetworkDeviceName=ntn01-11a-sw3, User-Name=#CTSREQUEST#, NAS-IP-Address=10.56.72.126, Service-Type=Outbound, AcsSessionID=npf-sjca-pdp01/195491152/2086202, SelectedAccessService=NDAC_SGT_Service, FailureReason=11302 Received Secure RADIUS request without a cts-pac-opaque cisco-av-pair attribute, Step=11001, Step=11017, Step=15012, Step=11302, NetworkDeviceGroups=Location#All Locations#NTN, NetworkDeviceGroups=Device Type#All Device Types#Wired, CPMSessionID=0a2a073f00005D5853E326A2, Model Name=4503, Location=Location#All Locations#NTN, Device Type=Device Type#All Device Types#Wired, Res
 ponse={RadiusPacketType=Drop; },
-Aug  6 16:53:23 10.42.7.64 Aug  7 00:12:35 npf-sjca-pdp02 CISE_Failed_Attempts 0000371016 1 0 2014-08-07 00:12:35.547 -07:00 0011269586 5405 NOTICE Failed-Attempt: RADIUS Request dropped, ConfigVersionId=240, Device IP Address=10.56.72.126, Device Port=1645, DestinationIPAddress=10.42.7.64, DestinationPort=1812, RadiusPacketType=AccessRequest, UserName=#CTSREQUEST#, Protocol=Radius, RequestLatency=5, NetworkDeviceName=ntn01-11a-sw3, User-Name=#CTSREQUEST#, NAS-IP-Address=10.56.72.126, Service-Type=Outbound, AcsSessionID=npf-sjca-pdp02/195481465/271040, SelectedAccessService=NDAC_SGT_Service, FailureReason=11302 Received Secure RADIUS request without a cts-pac-opaque cisco-av-pair attribute, Step=11001, Step=11017, Step=15012, Step=11302, NetworkDeviceGroups=Location#All Locations#NTN, NetworkDeviceGroups=Device Type#All Device Types#Wired, CPMSessionID=0a2a07400000570053E326E3, Model Name=4503, Location=Location#All Locations#NTN, Device Type=Device Type#All Device Types#Wired, Resp
 onse={RadiusPacketType=Drop; },
-Aug  6 16:53:28 10.42.7.63 Aug  7 00:12:40 npf-sjca-pdp01 CISE_Failed_Attempts 0001970802 1 0 2014-08-07 00:12:40.596 -07:00 0098689883 5405 NOTICE Failed-Attempt: RADIUS Request dropped, ConfigVersionId=133, Device IP Address=10.56.72.126, Device Port=1645, DestinationIPAddress=10.42.7.63, DestinationPort=1812, RadiusPacketType=AccessRequest, UserName=#CTSREQUEST#, Protocol=Radius, RequestLatency=2, NetworkDeviceName=ntn01-11a-sw3, User-Name=#CTSREQUEST#, NAS-IP-Address=10.56.72.126, Service-Type=Outbound, AcsSessionID=npf-sjca-pdp01/195491152/2086334, SelectedAccessService=NDAC_SGT_Service, FailureReason=11302 Received Secure RADIUS request without a cts-pac-opaque cisco-av-pair attribute, Step=11001, Step=11017, Step=15012, Step=11302, NetworkDeviceGroups=Location#All Locations#NTN, NetworkDeviceGroups=Device Type#All Device Types#Wired, CPMSessionID=0a2a073f00005D5953E326E8, Model Name=4503, Location=Location#All Locations#NTN, Device Type=Device Type#All Device Types#Wired, Res
 ponse={RadiusPacketType=Drop; },
-Aug  6 16:54:12 10.34.84.145 Aug  7 00:13:24 stage-pdp01 CISE_Failed_Attempts 0000024680 1 0 2014-08-07 00:13:24.527 -07:00 0000287388 5405 NOTICE Failed-Attempt: RADIUS Request dropped, ConfigVersionId=113, Device IP Address=172.23.91.132, Device Port=32769, DestinationIPAddress=10.34.84.145, DestinationPort=1812, Protocol=Radius, User-Name=test, NAS-IP-Address=192.168.30.11, Service-Type=NAS Prompt, NAS-Identifier=Cisco_1b:e0:84, AcsSessionID=stage-pdp01/196593288/19368, FailureReason=11007 Could not locate Network Device or AAA Client, Step=11001, Step=11017, Step=11007, Step=5405, 
-Aug  6 16:54:14 10.34.84.145 Aug  7 00:13:26 stage-pdp01 CISE_Failed_Attempts 0000024681 1 0 2014-08-07 00:13:26.531 -07:00 0000287392 5405 NOTICE Failed-Attempt: RADIUS Request dropped, ConfigVersionId=113, Device IP Address=172.23.91.132, Device Port=32769, DestinationIPAddress=10.34.84.145, DestinationPort=1812, Protocol=Radius, User-Name=test, NAS-IP-Address=192.168.30.11, Service-Type=NAS Prompt, NAS-Identifier=Cisco_1b:e0:84, AcsSessionID=stage-pdp01/196593288/19369, FailureReason=11007 Could not locate Network Device or AAA Client, Step=11001, Step=11017, Step=11007, Step=5405, 
-Aug  6 16:54:16 10.34.84.145 Aug  7 00:13:28 stage-pdp01 CISE_Failed_Attempts 0000024682 1 0 2014-08-07 00:13:28.534 -07:00 0000287396 5405 NOTICE Failed-Attempt: RADIUS Request dropped, ConfigVersionId=113, Device IP Address=172.23.91.132, Device Port=32769, DestinationIPAddress=10.34.84.145, DestinationPort=1812, Protocol=Radius, User-Name=test, NAS-IP-Address=192.168.30.11, Service-Type=NAS Prompt, NAS-Identifier=Cisco_1b:e0:84, AcsSessionID=stage-pdp01/196593288/19370, FailureReason=11007 Could not locate Network Device or AAA Client, Step=11001, Step=11017, Step=11007, Step=5405, 
-Aug  6 16:54:18 10.34.84.145 Aug  7 00:13:30 stage-pdp01 CISE_Failed_Attempts 0000024683 1 0 2014-08-07 00:13:30.538 -07:00 0000287400 5405 NOTICE Failed-Attempt: RADIUS Request dropped, ConfigVersionId=113, Device IP Address=172.23.91.132, Device Port=32769, DestinationIPAddress=10.34.84.145, DestinationPort=1812, Protocol=Radius, User-Name=test, NAS-IP-Address=192.168.30.11, Service-Type=NAS Prompt, NAS-Identifier=Cisco_1b:e0:84, AcsSessionID=stage-pdp01/196593288/19371, FailureReason=11007 Could not locate Network Device or AAA Client, Step=11001, Step=11017, Step=11007, Step=5405, 
-Aug  6 16:54:20 10.34.84.145 Aug  7 00:13:32 stage-pdp01 CISE_Failed_Attempts 0000024684 1 0 2014-08-07 00:13:32.538 -07:00 0000287404 5405 NOTICE Failed-Attempt: RADIUS Request dropped, ConfigVersionId=113, Device IP Address=172.23.91.132, Device Port=32769, DestinationIPAddress=10.34.84.145, DestinationPort=1812, Protocol=Radius, User-Name=test, NAS-IP-Address=192.168.30.11, Service-Type=NAS Prompt, NAS-Identifier=Cisco_1b:e0:84, AcsSessionID=stage-pdp01/196593288/19372, FailureReason=11007 Could not locate Network Device or AAA Client, Step=11001, Step=11017, Step=11007, Step=5405, 
-Aug  6 16:54:22 10.34.84.145 Aug  7 00:13:34 stage-pdp01 CISE_Failed_Attempts 0000024685 1 0 2014-08-07 00:13:34.542 -07:00 0000287408 5405 NOTICE Failed-Attempt: RADIUS Request dropped, ConfigVersionId=113, Device IP Address=172.23.91.132, Device Port=32769, DestinationIPAddress=10.34.84.145, DestinationPort=1812, Protocol=Radius, User-Name=test, NAS-IP-Address=192.168.30.11, Service-Type=NAS Prompt, NAS-Identifier=Cisco_1b:e0:84, AcsSessionID=stage-pdp01/196593288/19373, FailureReason=11007 Could not locate Network Device or AAA Client, Step=11001, Step=11017, Step=11007, Step=5405, 
-Aug  6 16:54:33 10.42.7.64 Aug  7 00:13:45 npf-sjca-pdp02 CISE_Failed_Attempts 0000371020 1 0 2014-08-07 00:13:45.628 -07:00 0011269631 5405 NOTICE Failed-Attempt: RADIUS Request dropped, ConfigVersionId=240, Device IP Address=10.56.72.126, Device Port=1645, DestinationIPAddress=10.42.7.64, DestinationPort=1812, RadiusPacketType=AccessRequest, UserName=#CTSREQUEST#, Protocol=Radius, RequestLatency=5, NetworkDeviceName=ntn01-11a-sw3, User-Name=#CTSREQUEST#, NAS-IP-Address=10.56.72.126, Service-Type=Outbound, AcsSessionID=npf-sjca-pdp02/195481465/271044, SelectedAccessService=NDAC_SGT_Service, FailureReason=11302 Received Secure RADIUS request without a cts-pac-opaque cisco-av-pair attribute, Step=11001, Step=11017, Step=15012, Step=11302, NetworkDeviceGroups=Location#All Locations#NTN, NetworkDeviceGroups=Device Type#All Device Types#Wired, CPMSessionID=0a2a07400000570153E32729, Model Name=4503, Location=Location#All Locations#NTN, Device Type=Device Type#All Device Types#Wired, Resp
 onse={RadiusPacketType=Drop; },
-Aug  6 16:54:38 10.42.7.63 Aug  7 00:13:50 npf-sjca-pdp01 CISE_Failed_Attempts 0001970913 1 0 2014-08-07 00:13:50.668 -07:00 0098695334 5405 NOTICE Failed-Attempt: RADIUS Request dropped, ConfigVersionId=133, Device IP Address=10.56.72.126, Device Port=1645, DestinationIPAddress=10.42.7.63, DestinationPort=1812, RadiusPacketType=AccessRequest, UserName=#CTSREQUEST#, Protocol=Radius, RequestLatency=2, NetworkDeviceName=ntn01-11a-sw3, User-Name=#CTSREQUEST#, NAS-IP-Address=10.56.72.126, Service-Type=Outbound, AcsSessionID=npf-sjca-pdp01/195491152/2086486, SelectedAccessService=NDAC_SGT_Service, FailureReason=11302 Received Secure RADIUS request without a cts-pac-opaque cisco-av-pair attribute, Step=11001, Step=11017, Step=15012, Step=11302, NetworkDeviceGroups=Location#All Locations#NTN, NetworkDeviceGroups=Device Type#All Device Types#Wired, CPMSessionID=0a2a073f00005D5A53E3272E, Model Name=4503, Location=Location#All Locations#NTN, Device Type=Device Type#All Device Types#Wired, Res
 ponse={RadiusPacketType=Drop; },
-Aug  6 16:54:43 10.42.7.64 Aug  7 00:13:55 npf-sjca-pdp02 CISE_Failed_Attempts 0000371025 1 0 2014-08-07 00:13:55.694 -07:00 0011269740 5405 NOTICE Failed-Attempt: RADIUS Request dropped, ConfigVersionId=240, Device IP Address=10.56.72.126, Device Port=1645, DestinationIPAddress=10.42.7.64, DestinationPort=1812, RadiusPacketType=AccessRequest, UserName=#CTSREQUEST#, Protocol=Radius, RequestLatency=7, NetworkDeviceName=ntn01-11a-sw3, User-Name=#CTSREQUEST#, NAS-IP-Address=10.56.72.126, Service-Type=Outbound, AcsSessionID=npf-sjca-pdp02/195481465/271048, SelectedAccessService=NDAC_SGT_Service, FailureReason=11302 Received Secure RADIUS request without a cts-pac-opaque cisco-av-pair attribute, Step=11001, Step=11017, Step=15012, Step=11302, NetworkDeviceGroups=Location#All Locations#NTN, NetworkDeviceGroups=Device Type#All Device Types#Wired, CPMSessionID=0a2a07400000570253E32733, Model Name=4503, Location=Location#All Locations#NTN, Device Type=Device Type#All Device Types#Wired, Resp
 onse={RadiusPacketType=Drop; },
-Aug  6 16:54:48 10.42.7.63 Aug  7 00:14:00 npf-sjca-pdp01 CISE_Failed_Attempts 0001970924 1 0 2014-08-07 00:14:00.705 -07:00 0098695591 5405 NOTICE Failed-Attempt: RADIUS Request dropped, ConfigVersionId=133, Device IP Address=10.56.72.126, Device Port=1645, DestinationIPAddress=10.42.7.63, DestinationPort=1812, RadiusPacketType=AccessRequest, UserName=#CTSREQUEST#, Protocol=Radius, RequestLatency=1, NetworkDeviceName=ntn01-11a-sw3, User-Name=#CTSREQUEST#, NAS-IP-Address=10.56.72.126, Service-Type=Outbound, AcsSessionID=npf-sjca-pdp01/195491152/2086505, SelectedAccessService=NDAC_SGT_Service, FailureReason=11302 Received Secure RADIUS request without a cts-pac-opaque cisco-av-pair attribute, Step=11001, Step=11017, Step=15012, Step=11302, NetworkDeviceGroups=Location#All Locations#NTN, NetworkDeviceGroups=Device Type#All Device Types#Wired, CPMSessionID=0a2a073f00005D5B53E32738, Model Name=4503, Location=Location#All Locations#NTN, Device Type=Device Type#All Device Types#Wired, Res
 ponse={RadiusPacketType=Drop; },
-Aug  6 16:55:53 10.42.7.64 Aug  7 00:15:05 npf-sjca-pdp02 CISE_Failed_Attempts 0000371036 1 0 2014-08-07 00:15:05.742 -07:00 0011270054 5405 NOTICE Failed-Attempt: RADIUS Request dropped, ConfigVersionId=240, Device IP Address=10.56.72.126, Device Port=1645, DestinationIPAddress=10.42.7.64, DestinationPort=1812, RadiusPacketType=AccessRequest, UserName=#CTSREQUEST#, Protocol=Radius, RequestLatency=6, NetworkDeviceName=ntn01-11a-sw3, User-Name=#CTSREQUEST#, NAS-IP-Address=10.56.72.126, Service-Type=Outbound, AcsSessionID=npf-sjca-pdp02/195481465/271057, SelectedAccessService=NDAC_SGT_Service, FailureReason=11302 Received Secure RADIUS request without a cts-pac-opaque cisco-av-pair attribute, Step=11001, Step=11017, Step=15012, Step=11302, NetworkDeviceGroups=Location#All Locations#NTN, NetworkDeviceGroups=Device Type#All Device Types#Wired, CPMSessionID=0a2a07400000570353E32779, Model Name=4503, Location=Location#All Locations#NTN, Device Type=Device Type#All Device Types#Wired, Resp
 onse={RadiusPacketType=Drop; },
-Aug  6 16:55:58 10.42.7.63 Aug  7 00:15:10 npf-sjca-pdp01 CISE_Failed_Attempts 0001970997 1 0 2014-08-07 00:15:10.772 -07:00 0098698954 5405 NOTICE Failed-Attempt: RADIUS Request dropped, ConfigVersionId=133, Device IP Address=10.56.72.126, Device Port=1645, DestinationIPAddress=10.42.7.63, DestinationPort=1812, RadiusPacketType=AccessRequest, UserName=#CTSREQUEST#, Protocol=Radius, RequestLatency=1, NetworkDeviceName=ntn01-11a-sw3, User-Name=#CTSREQUEST#, NAS-IP-Address=10.56.72.126, Service-Type=Outbound, AcsSessionID=npf-sjca-pdp01/195491152/2086621, SelectedAccessService=NDAC_SGT_Service, FailureReason=11302 Received Secure RADIUS request without a cts-pac-opaque cisco-av-pair attribute, Step=11001, Step=11017, Step=15012, Step=11302, NetworkDeviceGroups=Location#All Locations#NTN, NetworkDeviceGroups=Device Type#All Device Types#Wired, CPMSessionID=0a2a073f00005D5C53E3277E, Model Name=4503, Location=Location#All Locations#NTN, Device Type=Device Type#All Device Types#Wired, Res
 ponse={RadiusPacketType=Drop; },
-Aug  6 16:57:03 10.42.7.64 Aug  7 00:16:15 npf-sjca-pdp02 CISE_Failed_Attempts 0000371051 1 0 2014-08-07 00:16:15.827 -07:00 0011270497 5405 NOTICE Failed-Attempt: RADIUS Request dropped, ConfigVersionId=240, Device IP Address=10.56.72.126, Device Port=1645, DestinationIPAddress=10.42.7.64, DestinationPort=1812, RadiusPacketType=AccessRequest, UserName=#CTSREQUEST#, Protocol=Radius, RequestLatency=6, NetworkDeviceName=ntn01-11a-sw3, User-Name=#CTSREQUEST#, NAS-IP-Address=10.56.72.126, Service-Type=Outbound, AcsSessionID=npf-sjca-pdp02/195481465/271067, SelectedAccessService=NDAC_SGT_Service, FailureReason=11302 Received Secure RADIUS request without a cts-pac-opaque cisco-av-pair attribute, Step=11001, Step=11017, Step=15012, Step=11302, NetworkDeviceGroups=Location#All Locations#NTN, NetworkDeviceGroups=Device Type#All Device Types#Wired, CPMSessionID=0a2a07400000570453E327BF, Model Name=4503, Location=Location#All Locations#NTN, Device Type=Device Type#All Device Types#Wired, Resp
 onse={RadiusPacketType=Drop; },
-Aug  6 16:57:08 10.42.7.63 Aug  7 00:16:20 npf-sjca-pdp01 CISE_Failed_Attempts 0001971096 1 0 2014-08-07 00:16:20.857 -07:00 0098703837 5405 NOTICE Failed-Attempt: RADIUS Request dropped, ConfigVersionId=133, Device IP Address=10.56.72.126, Device Port=1645, DestinationIPAddress=10.42.7.63, DestinationPort=1812, RadiusPacketType=AccessRequest, UserName=#CTSREQUEST#, Protocol=Radius, RequestLatency=2, NetworkDeviceName=ntn01-11a-sw3, User-Name=#CTSREQUEST#, NAS-IP-Address=10.56.72.126, Service-Type=Outbound, AcsSessionID=npf-sjca-pdp01/195491152/2086806, SelectedAccessService=NDAC_SGT_Service, FailureReason=11302 Received Secure RADIUS request without a cts-pac-opaque cisco-av-pair attribute, Step=11001, Step=11017, Step=15012, Step=11302, NetworkDeviceGroups=Location#All Locations#NTN, NetworkDeviceGroups=Device Type#All Device Types#Wired, CPMSessionID=0a2a073f00005D5D53E327C4, Model Name=4503, Location=Location#All Locations#NTN, Device Type=Device Type#All Device Types#Wired, Res
 ponse={RadiusPacketType=Drop; },
-Aug  6 16:57:24 10.34.84.145 Aug  7 00:16:36 stage-pdp01 CISE_Failed_Attempts 0000024697 1 0 2014-08-07 00:16:36.602 -07:00 0000287553 5405 NOTICE Failed-Attempt: RADIUS Request dropped, ConfigVersionId=113, Device IP Address=172.23.91.132, Device Port=32769, DestinationIPAddress=10.34.84.145, DestinationPort=1812, Protocol=Radius, User-Name=test, NAS-IP-Address=192.168.30.11, Service-Type=NAS Prompt, NAS-Identifier=Cisco_1b:e0:84, AcsSessionID=stage-pdp01/196593288/19384, FailureReason=11007 Could not locate Network Device or AAA Client, Step=11001, Step=11017, Step=11007, Step=5405, 
-Aug  6 16:57:26 10.34.84.145 Aug  7 00:16:38 stage-pdp01 CISE_Failed_Attempts 0000024698 1 0 2014-08-07 00:16:38.605 -07:00 0000287557 5405 NOTICE Failed-Attempt: RADIUS Request dropped, ConfigVersionId=113, Device IP Address=172.23.91.132, Device Port=32769, DestinationIPAddress=10.34.84.145, DestinationPort=1812, Protocol=Radius, User-Name=test, NAS-IP-Address=192.168.30.11, Service-Type=NAS Prompt, NAS-Identifier=Cisco_1b:e0:84, AcsSessionID=stage-pdp01/196593288/19385, FailureReason=11007 Could not locate Network Device or AAA Client, Step=11001, Step=11017, Step=11007, Step=5405, 
-Aug  6 16:57:28 10.34.84.145 Aug  7 00:16:40 stage-pdp01 CISE_Failed_Attempts 0000024699 1 0 2014-08-07 00:16:40.609 -07:00 0000287561 5405 NOTICE Failed-Attempt: RADIUS Request dropped, ConfigVersionId=113, Device IP Address=172.23.91.132, Device Port=32769, DestinationIPAddress=10.34.84.145, DestinationPort=1812, Protocol=Radius, User-Name=test, NAS-IP-Address=192.168.30.11, Service-Type=NAS Prompt, NAS-Identifier=Cisco_1b:e0:84, AcsSessionID=stage-pdp01/196593288/19386, FailureReason=11007 Could not locate Network Device or AAA Client, Step=11001, Step=11017, Step=11007, Step=5405, 
-Aug  6 16:57:30 10.34.84.145 Aug  7 00:16:42 stage-pdp01 CISE_Failed_Attempts 0000024700 1 0 2014-08-07 00:16:42.613 -07:00 0000287565 5405 NOTICE Failed-Attempt: RADIUS Request dropped, ConfigVersionId=113, Device IP Address=172.23.91.132, Device Port=32769, DestinationIPAddress=10.34.84.145, DestinationPort=1812, Protocol=Radius, User-Name=test, NAS-IP-Address=192.168.30.11, Service-Type=NAS Prompt, NAS-Identifier=Cisco_1b:e0:84, AcsSessionID=stage-pdp01/196593288/19387, FailureReason=11007 Could not locate Network Device or AAA Client, Step=11001, Step=11017, Step=11007, Step=5405, 
-Aug  6 16:57:32 10.34.84.145 Aug  7 00:16:44 stage-pdp01 CISE_Failed_Attempts 0000024701 1 0 2014-08-07 00:16:44.613 -07:00 0000287569 5405 NOTICE Failed-Attempt: RADIUS Request dropped, ConfigVersionId=113, Device IP Address=172.23.91.132, Device Port=32769, DestinationIPAddress=10.34.84.145, DestinationPort=1812, Protocol=Radius, User-Name=test, NAS-IP-Address=192.168.30.11, Service-Type=NAS Prompt, NAS-Identifier=Cisco_1b:e0:84, AcsSessionID=stage-pdp01/196593288/19388, FailureReason=11007 Could not locate Network Device or AAA Client, Step=11001, Step=11017, Step=11007, Step=5405, 
-Aug  6 16:57:34 10.34.84.145 Aug  7 00:16:46 stage-pdp01 CISE_Failed_Attempts 0000024702 1 0 2014-08-07 00:16:46.617 -07:00 0000287573 5405 NOTICE Failed-Attempt: RADIUS Request dropped, ConfigVersionId=113, Device IP Address=172.23.91.132, Device Port=32769, DestinationIPAddress=10.34.84.145, DestinationPort=1812, Protocol=Radius, User-Name=test, NAS-IP-Address=192.168.30.11, Service-Type=NAS Prompt, NAS-Identifier=Cisco_1b:e0:84, AcsSessionID=stage-pdp01/196593288/19389, FailureReason=11007 Could not locate Network Device or AAA Client, Step=11001, Step=11017, Step=11007, Step=5405, 
-Aug  6 16:58:03 10.42.7.64 Aug  7 00:17:15 npf-sjca-pdp02 CISE_Failed_Attempts 0000371063 1 0 2014-08-07 00:17:15.966 -07:00 0011270832 5435 NOTICE RADIUS: NAS conducted several failed authentications of the same scenario, ConfigVersionId=240, Device IP Address=10.34.76.212, Device Port=32770, DestinationIPAddress=10.42.7.64, DestinationPort=1813, RadiusPacketType=AccountingRequest, UserName=hslai, Protocol=Radius, RequestLatency=25, NetworkDeviceName=sjcm-00a-npf-wlc1, User-Name=hslai, NAS-IP-Address=10.34.76.212, NAS-Port=1, Framed-IP-Address=10.34.94.11, Class=CACS:0a224cd40002fdf953e327f2:npf-sjca-pdp02/195481465/271072, Called-Station-ID=88-43-e1-62-1d-20, Calling-Station-ID=24-a2-e1-3b-4b-cb, NAS-Identifier=sjcm-00a-npf-wlc1, Acct-Status-Type=Interim-Update, Acct-Delay-Time=0, Acct-Input-Octets=5198, Acct-Output-Octets=4093, Acct-Session-Id=53e327f2/24:a2:e1:3b:4b:cb/174403, Acct-Authentic=RADIUS, Acct-Session-Time=9, Acct-Input-Packets=37, Acct-Output-Packets=13, undefined-52
 =
-Aug  6 16:58:13 10.42.7.64 Aug  7 00:17:25 npf-sjca-pdp02 CISE_Failed_Attempts 0000371065 1 0 2014-08-07 00:17:25.902 -07:00 0011270838 5405 NOTICE Failed-Attempt: RADIUS Request dropped, ConfigVersionId=240, Device IP Address=10.56.72.126, Device Port=1645, DestinationIPAddress=10.42.7.64, DestinationPort=1812, RadiusPacketType=AccessRequest, UserName=#CTSREQUEST#, Protocol=Radius, RequestLatency=4, NetworkDeviceName=ntn01-11a-sw3, User-Name=#CTSREQUEST#, NAS-IP-Address=10.56.72.126, Service-Type=Outbound, AcsSessionID=npf-sjca-pdp02/195481465/271076, SelectedAccessService=NDAC_SGT_Service, FailureReason=11302 Received Secure RADIUS request without a cts-pac-opaque cisco-av-pair attribute, Step=11001, Step=11017, Step=15012, Step=11302, NetworkDeviceGroups=Location#All Locations#NTN, NetworkDeviceGroups=Device Type#All Device Types#Wired, CPMSessionID=0a2a07400000570553E32805, Model Name=4503, Location=Location#All Locations#NTN, Device Type=Device Type#All Device Types#Wired, Resp
 onse={RadiusPacketType=Drop; },
-Aug  6 16:58:18 10.42.7.63 Aug  7 00:17:30 npf-sjca-pdp01 CISE_Failed_Attempts 0001971204 1 0 2014-08-07 00:17:30.916 -07:00 0098707928 5405 NOTICE Failed-Attempt: RADIUS Request dropped, ConfigVersionId=133, Device IP Address=10.56.72.126, Device Port=1645, DestinationIPAddress=10.42.7.63, DestinationPort=1812, RadiusPacketType=AccessRequest, UserName=#CTSREQUEST#, Protocol=Radius, RequestLatency=1, NetworkDeviceName=ntn01-11a-sw3, User-Name=#CTSREQUEST#, NAS-IP-Address=10.56.72.126, Service-Type=Outbound, AcsSessionID=npf-sjca-pdp01/195491152/2086981, SelectedAccessService=NDAC_SGT_Service, FailureReason=11302 Received Secure RADIUS request without a cts-pac-opaque cisco-av-pair attribute, Step=11001, Step=11017, Step=15012, Step=11302, NetworkDeviceGroups=Location#All Locations#NTN, NetworkDeviceGroups=Device Type#All Device Types#Wired, CPMSessionID=0a2a073f00005D5E53E3280A, Model Name=4503, Location=Location#All Locations#NTN, Device Type=Device Type#All Device Types#Wired, Res
 ponse={RadiusPacketType=Drop; },
-Aug  6 16:59:23 10.42.7.64 Aug  7 00:18:35 npf-sjca-pdp02 CISE_Failed_Attempts 0000371070 1 0 2014-08-07 00:18:35.942 -07:00 0011271044 5405 NOTICE Failed-Attempt: RADIUS Request dropped, ConfigVersionId=240, Device IP Address=10.56.72.126, Device Port=1645, DestinationIPAddress=10.42.7.64, DestinationPort=1812, RadiusPacketType=AccessRequest, UserName=#CTSREQUEST#, Protocol=Radius, RequestLatency=5, NetworkDeviceName=ntn01-11a-sw3, User-Name=#CTSREQUEST#, NAS-IP-Address=10.56.72.126, Service-Type=Outbound, AcsSessionID=npf-sjca-pdp02/195481465/271081, SelectedAccessService=NDAC_SGT_Service, FailureReason=11302 Received Secure RADIUS request without a cts-pac-opaque cisco-av-pair attribute, Step=11001, Step=11017, Step=15012, Step=11302, NetworkDeviceGroups=Location#All Locations#NTN, NetworkDeviceGroups=Device Type#All Device Types#Wired, CPMSessionID=0a2a07400000570653E3284B, Model Name=4503, Location=Location#All Locations#NTN, Device Type=Device Type#All Device Types#Wired, Resp
 onse={RadiusPacketType=Drop; },
-Aug  6 16:59:28 10.42.7.64 Aug  7 00:18:40 npf-sjca-pdp02 CISE_Failed_Attempts 0000371072 1 0 2014-08-07 00:18:40.669 -07:00 0011271053 5400 NOTICE Failed-Attempt: Authentication failed, ConfigVersionId=240, Device IP Address=10.56.129.4, Device Port=32770, DestinationIPAddress=10.42.7.64, DestinationPort=1812, RadiusPacketType=AccessRequest, UserName=istern, Protocol=Radius, RequestLatency=12, NetworkDeviceName=NTN-WLC1, User-Name=istern, NAS-IP-Address=10.56.129.4, NAS-Port=1, Service-Type=Framed, Framed-MTU=1300, State=37CPMSessionID=0a388104000045de53e2c750\;41SessionID=npf-sjca-pdp02/195481465/271077\;, Called-Station-ID=70-10-5c-f3-2f-80:alpha_byod, Calling-Station-ID=f0-27-65-48-8c-8f, NAS-Identifier=ntn01-11a-wlc1, NAS-Port-Type=Wireless - IEEE 802.11, Tunnel-Type=(tag=0) VLAN, Tunnel-Medium-Type=(tag=0) 802, Tunnel-Private-Group-ID=(tag=0) 604, undefined-89=
-Aug  6 16:59:28 10.42.7.63 Aug  7 00:18:40 npf-sjca-pdp01 CISE_Failed_Attempts 0001971282 1 0 2014-08-07 00:18:40.981 -07:00 0098711291 5405 NOTICE Failed-Attempt: RADIUS Request dropped, ConfigVersionId=133, Device IP Address=10.56.72.126, Device Port=1645, DestinationIPAddress=10.42.7.63, DestinationPort=1812, RadiusPacketType=AccessRequest, UserName=#CTSREQUEST#, Protocol=Radius, RequestLatency=2, NetworkDeviceName=ntn01-11a-sw3, User-Name=#CTSREQUEST#, NAS-IP-Address=10.56.72.126, Service-Type=Outbound, AcsSessionID=npf-sjca-pdp01/195491152/2087140, SelectedAccessService=NDAC_SGT_Service, FailureReason=11302 Received Secure RADIUS request without a cts-pac-opaque cisco-av-pair attribute, Step=11001, Step=11017, Step=15012, Step=11302, NetworkDeviceGroups=Location#All Locations#NTN, NetworkDeviceGroups=Device Type#All Device Types#Wired, CPMSessionID=0a2a073f00005D5F53E32850, Model Name=4503, Location=Location#All Locations#NTN, Device Type=Device Type#All Device Types#Wired, Res
 ponse={RadiusPacketType=Drop; },
-Aug  6 17:00:33 10.42.7.64 Aug  7 00:19:46 npf-sjca-pdp02 CISE_Failed_Attempts 0000371080 1 0 2014-08-07 00:19:46.020 -07:00 0011271232 5405 NOTICE Failed-Attempt: RADIUS Request dropped, ConfigVersionId=240, Device IP Address=10.56.72.126, Device Port=1645, DestinationIPAddress=10.42.7.64, DestinationPort=1812, RadiusPacketType=AccessRequest, UserName=#CTSREQUEST#, Protocol=Radius, RequestLatency=5, NetworkDeviceName=ntn01-11a-sw3, User-Name=#CTSREQUEST#, NAS-IP-Address=10.56.72.126, Service-Type=Outbound, AcsSessionID=npf-sjca-pdp02/195481465/271087, SelectedAccessService=NDAC_SGT_Service, FailureReason=11302 Received Secure RADIUS request without a cts-pac-opaque cisco-av-pair attribute, Step=11001, Step=11017, Step=15012, Step=11302, NetworkDeviceGroups=Location#All Locations#NTN, NetworkDeviceGroups=Device Type#All Device Types#Wired, CPMSessionID=0a2a07400000570753E32892, Model Name=4503, Location=Location#All Locations#NTN, Device Type=Device Type#All Device Types#Wired, Resp
 onse={RadiusPacketType=Drop; },
-Aug  6 17:00:36 10.34.84.145 Aug  7 00:19:48 stage-pdp01 CISE_Failed_Attempts 0000024712 1 0 2014-08-07 00:19:48.660 -07:00 0000287604 5405 NOTICE Failed-Attempt: RADIUS Request dropped, ConfigVersionId=113, Device IP Address=172.23.91.132, Device Port=32769, DestinationIPAddress=10.34.84.145, DestinationPort=1812, Protocol=Radius, User-Name=test, NAS-IP-Address=192.168.30.11, Service-Type=NAS Prompt, NAS-Identifier=Cisco_1b:e0:84, AcsSessionID=stage-pdp01/196593288/19396, FailureReason=11007 Could not locate Network Device or AAA Client, Step=11001, Step=11017, Step=11007, Step=5405, 
-Aug  6 17:00:38 10.34.84.145 Aug  7 00:19:50 stage-pdp01 CISE_Failed_Attempts 0000024713 1 0 2014-08-07 00:19:50.664 -07:00 0000287608 5405 NOTICE Failed-Attempt: RADIUS Request dropped, ConfigVersionId=113, Device IP Address=172.23.91.132, Device Port=32769, DestinationIPAddress=10.34.84.145, DestinationPort=1812, Protocol=Radius, User-Name=test, NAS-IP-Address=192.168.30.11, Service-Type=NAS Prompt, NAS-Identifier=Cisco_1b:e0:84, AcsSessionID=stage-pdp01/196593288/19397, FailureReason=11007 Could not locate Network Device or AAA Client, Step=11001, Step=11017, Step=11007, Step=5405, 
-Aug  6 17:00:39 10.42.7.63 Aug  7 00:19:51 npf-sjca-pdp01 CISE_Failed_Attempts 0001971393 1 0 2014-08-07 00:19:51.042 -07:00 0098716185 5405 NOTICE Failed-Attempt: RADIUS Request dropped, ConfigVersionId=133, Device IP Address=10.56.72.126, Device Port=1645, DestinationIPAddress=10.42.7.63, DestinationPort=1812, RadiusPacketType=AccessRequest, UserName=#CTSREQUEST#, Protocol=Radius, RequestLatency=1, NetworkDeviceName=ntn01-11a-sw3, User-Name=#CTSREQUEST#, NAS-IP-Address=10.56.72.126, Service-Type=Outbound, AcsSessionID=npf-sjca-pdp01/195491152/2087311, SelectedAccessService=NDAC_SGT_Service, FailureReason=11302 Received Secure RADIUS request without a cts-pac-opaque cisco-av-pair attribute, Step=11001, Step=11017, Step=15012, Step=11302, NetworkDeviceGroups=Location#All Locations#NTN, NetworkDeviceGroups=Device Type#All Device Types#Wired, CPMSessionID=0a2a073f00005D6053E32897, Model Name=4503, Location=Location#All Locations#NTN, Device Type=Device Type#All Device Types#Wired, Res
 ponse={RadiusPacketType=Drop; },
-Aug  6 17:00:40 10.34.84.145 Aug  7 00:19:52 stage-pdp01 CISE_Failed_Attempts 0000024714 1 0 2014-08-07 00:19:52.664 -07:00 0000287612 5405 NOTICE Failed-Attempt: RADIUS Request dropped, ConfigVersionId=113, Device IP Address=172.23.91.132, Device Port=32769, DestinationIPAddress=10.34.84.145, DestinationPort=1812, Protocol=Radius, User-Name=test, NAS-IP-Address=192.168.30.11, Service-Type=NAS Prompt, NAS-Identifier=Cisco_1b:e0:84, AcsSessionID=stage-pdp01/196593288/19398, FailureReason=11007 Could not locate Network Device or AAA Client, Step=11001, Step=11017, Step=11007, Step=5405, 
-Aug  6 17:00:42 10.34.84.145 Aug  7 00:19:54 stage-pdp01 CISE_Failed_Attempts 0000024715 1 0 2014-08-07 00:19:54.668 -07:00 0000287616 5405 NOTICE Failed-Attempt: RADIUS Request dropped, ConfigVersionId=113, Device IP Address=172.23.91.132, Device Port=32769, DestinationIPAddress=10.34.84.145, DestinationPort=1812, Protocol=Radius, User-Name=test, NAS-IP-Address=192.168.30.11, Service-Type=NAS Prompt, NAS-Identifier=Cisco_1b:e0:84, AcsSessionID=stage-pdp01/196593288/19399, FailureReason=11007 Could not locate Network Device or AAA Client, Step=11001, Step=11017, Step=11007, Step=5405, 
-Aug  6 17:00:44 10.34.84.145 Aug  7 00:19:56 stage-pdp01 CISE_Failed_Attempts 0000024716 1 0 2014-08-07 00:19:56.672 -07:00 0000287620 5405 NOTICE Failed-Attempt: RADIUS Request dropped, ConfigVersionId=113, Device IP Address=172.23.91.132, Device Port=32769, DestinationIPAddress=10.34.84.145, DestinationPort=1812, Protocol=Radius, User-Name=test, NAS-IP-Address=192.168.30.11, Service-Type=NAS Prompt, NAS-Identifier=Cisco_1b:e0:84, AcsSessionID=stage-pdp01/196593288/19400, FailureReason=11007 Could not locate Network Device or AAA Client, Step=11001, Step=11017, Step=11007, Step=5405, 
-Aug  6 17:00:46 10.34.84.145 Aug  7 00:19:58 stage-pdp01 CISE_Failed_Attempts 0000024717 1 0 2014-08-07 00:19:58.675 -07:00 0000287624 5405 NOTICE Failed-Attempt: RADIUS Request dropped, ConfigVersionId=113, Device IP Address=172.23.91.132, Device Port=32769, DestinationIPAddress=10.34.84.145, DestinationPort=1812, Protocol=Radius, User-Name=test, NAS-IP-Address=192.168.30.11, Service-Type=NAS Prompt, NAS-Identifier=Cisco_1b:e0:84, AcsSessionID=stage-pdp01/196593288/19401, Fail

<TRUNCATED>


[14/15] incubator-metron git commit: METRON-142 Simplify Parser configuration (merrimanr via cestella) closes apache/incubator-metron#120

Posted by ce...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/df8d682e/metron-platform/metron-common/src/test/java/org/apache/metron/common/bolt/ConfiguredParserBoltTest.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-common/src/test/java/org/apache/metron/common/bolt/ConfiguredParserBoltTest.java b/metron-platform/metron-common/src/test/java/org/apache/metron/common/bolt/ConfiguredParserBoltTest.java
new file mode 100644
index 0000000..f945dd0
--- /dev/null
+++ b/metron-platform/metron-common/src/test/java/org/apache/metron/common/bolt/ConfiguredParserBoltTest.java
@@ -0,0 +1,129 @@
+/**
+ * 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.common.bolt;
+
+import backtype.storm.topology.OutputFieldsDeclarer;
+import backtype.storm.tuple.Tuple;
+import org.apache.curator.test.TestingServer;
+import org.apache.metron.TestConstants;
+import org.apache.metron.common.Constants;
+import org.apache.metron.common.configuration.ConfigurationType;
+import org.apache.metron.common.configuration.ConfigurationsUtils;
+import org.apache.metron.common.configuration.ParserConfigurations;
+import org.apache.metron.common.configuration.SensorParserConfig;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+
+public class ConfiguredParserBoltTest extends BaseConfiguredBoltTest {
+
+  private Set<String> parserConfigurationTypes = new HashSet<>();
+  private String zookeeperUrl;
+
+  public static class StandAloneConfiguredParserBolt extends ConfiguredParserBolt {
+
+    public StandAloneConfiguredParserBolt(String zookeeperUrl) {
+      super(zookeeperUrl);
+    }
+
+    @Override
+    public void execute(Tuple input) {
+    }
+
+    @Override
+    public void declareOutputFields(OutputFieldsDeclarer declarer) {
+    }
+
+    @Override
+    public void reloadCallback(String name, ConfigurationType type) {
+      configsUpdated.add(name);
+    }
+  }
+
+  @Before
+  public void setupConfiguration() throws Exception {
+    TestingServer testZkServer = new TestingServer(true);
+    this.zookeeperUrl = testZkServer.getConnectString();
+    byte[] globalConfig = ConfigurationsUtils.readGlobalConfigFromFile(TestConstants.SAMPLE_CONFIG_PATH);
+    ConfigurationsUtils.writeGlobalConfigToZookeeper(globalConfig, zookeeperUrl);
+    parserConfigurationTypes.add(ConfigurationType.GLOBAL.getName());
+    Map<String, byte[]> sensorEnrichmentConfigs = ConfigurationsUtils.readSensorEnrichmentConfigsFromFile(TestConstants.ENRICHMENTS_CONFIGS_PATH);
+    for (String sensorType : sensorEnrichmentConfigs.keySet()) {
+      ConfigurationsUtils.writeSensorEnrichmentConfigToZookeeper(sensorType, sensorEnrichmentConfigs.get(sensorType), zookeeperUrl);
+    }
+    Map<String, byte[]> sensorParserConfigs = ConfigurationsUtils.readSensorParserConfigsFromFile(TestConstants.PARSER_CONFIGS_PATH);
+    for (String sensorType : sensorParserConfigs.keySet()) {
+      ConfigurationsUtils.writeSensorParserConfigToZookeeper(sensorType, sensorParserConfigs.get(sensorType), zookeeperUrl);
+      parserConfigurationTypes.add(sensorType);
+    }
+  }
+
+  @Test
+  public void test() throws Exception {
+    ParserConfigurations sampleConfigurations = new ParserConfigurations();
+    try {
+      StandAloneConfiguredParserBolt configuredBolt = new StandAloneConfiguredParserBolt(null);
+      configuredBolt.prepare(new HashMap(), topologyContext, outputCollector);
+      Assert.fail("A valid zookeeper url must be supplied");
+    } catch (RuntimeException e){}
+
+    configsUpdated = new HashSet<>();
+    sampleConfigurations.updateGlobalConfig(ConfigurationsUtils.readGlobalConfigFromFile(TestConstants.SAMPLE_CONFIG_PATH));
+    Map<String, byte[]> sensorParserConfigs = ConfigurationsUtils.readSensorParserConfigsFromFile(TestConstants.PARSER_CONFIGS_PATH);
+    for (String sensorType : sensorParserConfigs.keySet()) {
+      sampleConfigurations.updateSensorParserConfig(sensorType, sensorParserConfigs.get(sensorType));
+    }
+
+    StandAloneConfiguredParserBolt configuredBolt = new StandAloneConfiguredParserBolt(zookeeperUrl);
+    configuredBolt.prepare(new HashMap(), topologyContext, outputCollector);
+    waitForConfigUpdate(parserConfigurationTypes);
+    Assert.assertEquals(sampleConfigurations, configuredBolt.configurations);
+
+    configsUpdated = new HashSet<>();
+    Map<String, Object> sampleGlobalConfig = sampleConfigurations.getGlobalConfig();
+    sampleGlobalConfig.put("newGlobalField", "newGlobalValue");
+    ConfigurationsUtils.writeGlobalConfigToZookeeper(sampleGlobalConfig, zookeeperUrl);
+    waitForConfigUpdate(ConfigurationType.GLOBAL.getName());
+    Assert.assertEquals("Add global config field", sampleConfigurations.getGlobalConfig(), configuredBolt.configurations.getGlobalConfig());
+
+    configsUpdated = new HashSet<>();
+    sampleGlobalConfig.remove("newGlobalField");
+    ConfigurationsUtils.writeGlobalConfigToZookeeper(sampleGlobalConfig, zookeeperUrl);
+    waitForConfigUpdate(ConfigurationType.GLOBAL.getName());
+    Assert.assertEquals("Remove global config field", sampleConfigurations, configuredBolt.configurations);
+
+    configsUpdated = new HashSet<>();
+    String sensorType = "testSensorConfig";
+    SensorParserConfig testSensorConfig = new SensorParserConfig();
+    testSensorConfig.setParserClassName("className");
+    testSensorConfig.setSensorTopic("sensorTopic");
+    testSensorConfig.setParserConfig(new HashMap<String, Object>() {{
+      put("configName", "configObject");
+    }});
+    sampleConfigurations.updateSensorParserConfig(sensorType, testSensorConfig);
+    ConfigurationsUtils.writeSensorParserConfigToZookeeper(sensorType, testSensorConfig, zookeeperUrl);
+    waitForConfigUpdate(sensorType);
+    Assert.assertEquals("Add new sensor config", sampleConfigurations, configuredBolt.configurations);
+    configuredBolt.cleanup();
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/df8d682e/metron-platform/metron-common/src/test/java/org/apache/metron/common/cli/ConfigurationManagerIntegrationTest.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-common/src/test/java/org/apache/metron/common/cli/ConfigurationManagerIntegrationTest.java b/metron-platform/metron-common/src/test/java/org/apache/metron/common/cli/ConfigurationManagerIntegrationTest.java
index e17b877..c9a7d22 100644
--- a/metron-platform/metron-common/src/test/java/org/apache/metron/common/cli/ConfigurationManagerIntegrationTest.java
+++ b/metron-platform/metron-common/src/test/java/org/apache/metron/common/cli/ConfigurationManagerIntegrationTest.java
@@ -69,7 +69,7 @@ public class ConfigurationManagerIntegrationTest {
     zookeeperUrl = testZkServer.getConnectString();
     client = ConfigurationsUtils.getClient(zookeeperUrl);
     client.start();
-    File sensorDir = new File(new File(TestConstants.SAMPLE_CONFIG_PATH), ConfigurationType.SENSOR.getDirectory());
+    File sensorDir = new File(new File(TestConstants.SAMPLE_CONFIG_PATH), ConfigurationType.ENRICHMENT.getDirectory());
     sensors.addAll(Collections2.transform(
              Arrays.asList(sensorDir.list())
             ,s -> Iterables.getFirst(Splitter.on('.').split(s), "null")
@@ -114,9 +114,9 @@ public class ConfigurationManagerIntegrationTest {
     Assert.assertTrue("Global config does not exist", globalConfigFile.exists());
     validateConfig("global", ConfigurationType.GLOBAL, new String(Files.readAllBytes(Paths.get(globalConfigFile.toURI()))));
     for(String sensor : sensors) {
-      File sensorFile = new File(configDir, ConfigurationType.SENSOR.getDirectory() + "/" + sensor + ".json");
+      File sensorFile = new File(configDir, ConfigurationType.ENRICHMENT.getDirectory() + "/" + sensor + ".json");
       Assert.assertTrue(sensor + " config does not exist", sensorFile.exists());
-      validateConfig(sensor, ConfigurationType.SENSOR, new String(Files.readAllBytes(Paths.get(sensorFile.toURI()))));
+      validateConfig(sensor, ConfigurationType.ENRICHMENT, new String(Files.readAllBytes(Paths.get(sensorFile.toURI()))));
     }
   }
 
@@ -164,6 +164,7 @@ public class ConfigurationManagerIntegrationTest {
         }
       }
     });
+    Assert.assertEquals(true, foundGlobal.get());
     Assert.assertEquals(sensorsInZookeeper, sensors);
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/df8d682e/metron-platform/metron-common/src/test/java/org/apache/metron/common/cli/ConfigurationsUtilsTest.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-common/src/test/java/org/apache/metron/common/cli/ConfigurationsUtilsTest.java b/metron-platform/metron-common/src/test/java/org/apache/metron/common/cli/ConfigurationsUtilsTest.java
index 3393b41..0645619 100644
--- a/metron-platform/metron-common/src/test/java/org/apache/metron/common/cli/ConfigurationsUtilsTest.java
+++ b/metron-platform/metron-common/src/test/java/org/apache/metron/common/cli/ConfigurationsUtilsTest.java
@@ -37,8 +37,9 @@ public class ConfigurationsUtilsTest {
   private TestingServer testZkServer;
   private String zookeeperUrl;
   private CuratorFramework client;
-  private byte[] testGlobalConfig;
-  private Map<String, byte[]> testSensorConfigMap;
+  private byte[] expectedGlobalConfig;
+  private Map<String, byte[]> expectedSensorParserConfigMap;
+  private Map<String, byte[]> expectedSensorEnrichmentConfigMap;
 
   @Before
   public void setup() throws Exception {
@@ -46,23 +47,31 @@ public class ConfigurationsUtilsTest {
     zookeeperUrl = testZkServer.getConnectString();
     client = ConfigurationsUtils.getClient(zookeeperUrl);
     client.start();
-    testGlobalConfig = ConfigurationsUtils.readGlobalConfigFromFile(TestConstants.SAMPLE_CONFIG_PATH);
-    testSensorConfigMap = ConfigurationsUtils.readSensorEnrichmentConfigsFromFile(TestConstants.SAMPLE_CONFIG_PATH);
+    expectedGlobalConfig = ConfigurationsUtils.readGlobalConfigFromFile(TestConstants.SAMPLE_CONFIG_PATH);
+    expectedSensorParserConfigMap = ConfigurationsUtils.readSensorParserConfigsFromFile(TestConstants.PARSER_CONFIGS_PATH);
+    expectedSensorEnrichmentConfigMap = ConfigurationsUtils.readSensorEnrichmentConfigsFromFile(TestConstants.ENRICHMENTS_CONFIGS_PATH);
   }
 
   @Test
   public void test() throws Exception {
-    Assert.assertTrue(testGlobalConfig.length > 0);
-    ConfigurationsUtils.writeGlobalConfigToZookeeper(testGlobalConfig, zookeeperUrl);
-    byte[] readGlobalConfigBytes = ConfigurationsUtils.readGlobalConfigBytesFromZookeeper(client);
-    Assert.assertTrue(Arrays.equals(testGlobalConfig, readGlobalConfigBytes));
+    Assert.assertTrue(expectedGlobalConfig.length > 0);
+    ConfigurationsUtils.writeGlobalConfigToZookeeper(expectedGlobalConfig, zookeeperUrl);
+    byte[] actualGlobalConfigBytes = ConfigurationsUtils.readGlobalConfigBytesFromZookeeper(client);
+    Assert.assertTrue(Arrays.equals(expectedGlobalConfig, actualGlobalConfigBytes));
 
-    Assert.assertTrue(testSensorConfigMap.size() > 0);
+    Assert.assertTrue(expectedSensorParserConfigMap.size() > 0);
     String testSensorType = "yaf";
-    byte[] testSensorConfigBytes = testSensorConfigMap.get(testSensorType);
-    ConfigurationsUtils.writeSensorEnrichmentConfigToZookeeper(testSensorType, testSensorConfigBytes, zookeeperUrl);
-    byte[] readSensorConfigBytes = ConfigurationsUtils.readSensorEnrichmentConfigBytesFromZookeeper(testSensorType, client);
-    Assert.assertTrue(Arrays.equals(testSensorConfigBytes, readSensorConfigBytes));
+    byte[] expectedSensorParserConfigBytes = expectedSensorParserConfigMap.get(testSensorType);
+    ConfigurationsUtils.writeSensorParserConfigToZookeeper(testSensorType, expectedSensorParserConfigBytes, zookeeperUrl);
+    byte[] actualSensorParserConfigBytes = ConfigurationsUtils.readSensorParserConfigBytesFromZookeeper(testSensorType, client);
+    Assert.assertTrue(Arrays.equals(expectedSensorParserConfigBytes, actualSensorParserConfigBytes));
+
+    Assert.assertTrue(expectedSensorEnrichmentConfigMap.size() > 0);
+    byte[] expectedSensorEnrichmentConfigBytes = expectedSensorEnrichmentConfigMap.get(testSensorType);
+    ConfigurationsUtils.writeSensorEnrichmentConfigToZookeeper(testSensorType, expectedSensorEnrichmentConfigBytes, zookeeperUrl);
+    byte[] actualSensorEnrichmentConfigBytes = ConfigurationsUtils.readSensorEnrichmentConfigBytesFromZookeeper(testSensorType, client);
+    Assert.assertTrue(Arrays.equals(expectedSensorEnrichmentConfigBytes, actualSensorEnrichmentConfigBytes));
+
     String name = "testConfig";
     Map<String, Object> testConfig = new HashMap<>();
     testConfig.put("stringField", "value");
@@ -74,7 +83,6 @@ public class ConfigurationsUtilsTest {
 
   }
 
-
   @After
   public void tearDown() throws IOException {
     client.close();

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/df8d682e/metron-platform/metron-common/src/test/java/org/apache/metron/common/configuration/ConfigurationTest.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-common/src/test/java/org/apache/metron/common/configuration/ConfigurationTest.java b/metron-platform/metron-common/src/test/java/org/apache/metron/common/configuration/ConfigurationTest.java
index fb45ccc..3031ab9 100644
--- a/metron-platform/metron-common/src/test/java/org/apache/metron/common/configuration/ConfigurationTest.java
+++ b/metron-platform/metron-common/src/test/java/org/apache/metron/common/configuration/ConfigurationTest.java
@@ -55,7 +55,7 @@ public class ConfigurationTest {
         GetDataBuilder getDataBuilder = mock(GetDataBuilder.class);
         GetChildrenBuilder getChildrenBuilder = mock(GetChildrenBuilder.class);
 
-        when(getDataBuilder.forPath(Constants.ZOOKEEPER_GLOBAL_ROOT)).thenReturn(mockGlobalData());
+        when(getDataBuilder.forPath(ConfigurationType.GLOBAL.getZookeeperRoot())).thenReturn(mockGlobalData());
         when(curatorFramework.checkExists()).thenReturn(existsBuilder);
         when(curatorFramework.getData()).thenReturn(getDataBuilder);
         when(curatorFramework.getChildren()).thenReturn(getChildrenBuilder);

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/df8d682e/metron-platform/metron-common/src/test/java/org/apache/metron/common/configuration/ConfigurationsTest.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-common/src/test/java/org/apache/metron/common/configuration/ConfigurationsTest.java b/metron-platform/metron-common/src/test/java/org/apache/metron/common/configuration/ConfigurationsTest.java
index 18e6ee8..4720929 100644
--- a/metron-platform/metron-common/src/test/java/org/apache/metron/common/configuration/ConfigurationsTest.java
+++ b/metron-platform/metron-common/src/test/java/org/apache/metron/common/configuration/ConfigurationsTest.java
@@ -20,7 +20,6 @@ package org.apache.metron.common.configuration;
 import junit.framework.Assert;
 import nl.jqno.equalsverifier.EqualsVerifier;
 import nl.jqno.equalsverifier.Warning;
-import org.apache.metron.common.configuration.Configurations;
 import org.junit.Test;
 
 import java.io.IOException;
@@ -32,7 +31,7 @@ public class ConfigurationsTest {
     EqualsVerifier.forClass(Configurations.class).suppress(Warning.NONFINAL_FIELDS, Warning.NULL_FIELDS).usingGetClass().verify();
     Configurations configurations = new Configurations();
     try {
-      configurations.updateConfig("someConfig", (byte[]) null);
+      configurations.updateGlobalConfig((byte[]) null);
       Assert.fail("Updating a config with null should throw an IllegalStateException");
     } catch(IllegalStateException e) {}
     Assert.assertTrue(configurations.toString() != null && configurations.toString().length() > 0);

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/df8d682e/metron-platform/metron-common/src/test/java/org/apache/metron/common/configuration/SensorEnrichmentConfigTest.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-common/src/test/java/org/apache/metron/common/configuration/SensorEnrichmentConfigTest.java b/metron-platform/metron-common/src/test/java/org/apache/metron/common/configuration/SensorEnrichmentConfigTest.java
index 418ee9b..dbf894b 100644
--- a/metron-platform/metron-common/src/test/java/org/apache/metron/common/configuration/SensorEnrichmentConfigTest.java
+++ b/metron-platform/metron-common/src/test/java/org/apache/metron/common/configuration/SensorEnrichmentConfigTest.java
@@ -32,7 +32,7 @@ public class SensorEnrichmentConfigTest {
   @Test
   public void test() throws IOException {
     EqualsVerifier.forClass(SensorEnrichmentConfig.class).suppress(Warning.NONFINAL_FIELDS).usingGetClass().verify();
-    Map<String, byte[]> testSensorConfigMap = ConfigurationsUtils.readSensorEnrichmentConfigsFromFile(TestConstants.SAMPLE_CONFIG_PATH);
+    Map<String, byte[]> testSensorConfigMap = ConfigurationsUtils.readSensorEnrichmentConfigsFromFile(TestConstants.ENRICHMENTS_CONFIGS_PATH);
     byte[] sensorConfigBytes = testSensorConfigMap.get("yaf");
     SensorEnrichmentConfig sensorEnrichmentConfig = SensorEnrichmentConfig.fromBytes(sensorConfigBytes);
     Assert.assertNotNull(sensorEnrichmentConfig);

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/df8d682e/metron-platform/metron-common/src/test/java/org/apache/metron/common/configuration/SensorEnrichmentUpdateConfigTest.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-common/src/test/java/org/apache/metron/common/configuration/SensorEnrichmentUpdateConfigTest.java b/metron-platform/metron-common/src/test/java/org/apache/metron/common/configuration/SensorEnrichmentUpdateConfigTest.java
index 009fc22..77035fb 100644
--- a/metron-platform/metron-common/src/test/java/org/apache/metron/common/configuration/SensorEnrichmentUpdateConfigTest.java
+++ b/metron-platform/metron-common/src/test/java/org/apache/metron/common/configuration/SensorEnrichmentUpdateConfigTest.java
@@ -79,7 +79,7 @@ public class SensorEnrichmentUpdateConfigTest {
   @Test
   public void testThreatIntel() throws Exception {
 
-    SensorEnrichmentConfig broSc = (SensorEnrichmentConfig) ConfigurationType.SENSOR.deserialize(sourceConfigStr);
+    SensorEnrichmentConfig broSc = (SensorEnrichmentConfig) ConfigurationType.ENRICHMENT.deserialize(sourceConfigStr);
 
 
     SensorEnrichmentUpdateConfig config = JSONUtils.INSTANCE.load(threatIntelConfigStr, SensorEnrichmentUpdateConfig.class);

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/df8d682e/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/writer/ElasticsearchWriter.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/writer/ElasticsearchWriter.java b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/writer/ElasticsearchWriter.java
index 989abfb..4aa5995 100644
--- a/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/writer/ElasticsearchWriter.java
+++ b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/writer/ElasticsearchWriter.java
@@ -18,7 +18,7 @@
 package org.apache.metron.elasticsearch.writer;
 
 import backtype.storm.tuple.Tuple;
-import org.apache.metron.common.configuration.Configurations;
+import org.apache.metron.common.configuration.EnrichmentConfigurations;
 import org.apache.metron.common.configuration.enrichment.SensorEnrichmentConfig;
 import org.apache.metron.common.interfaces.BulkMessageWriter;
 import org.elasticsearch.action.bulk.BulkRequestBuilder;
@@ -51,7 +51,7 @@ public class ElasticsearchWriter implements BulkMessageWriter<JSONObject>, Seria
   }
 
   @Override
-  public void init(Map stormConf, Configurations configurations) {
+  public void init(Map stormConf, EnrichmentConfigurations configurations) {
     Map<String, Object> globalConfiguration = configurations.getGlobalConfig();
     ImmutableSettings.Builder builder = ImmutableSettings.settingsBuilder();
     builder.put("cluster.name", globalConfiguration.get("es.clustername"));
@@ -66,7 +66,7 @@ public class ElasticsearchWriter implements BulkMessageWriter<JSONObject>, Seria
   }
 
   @Override
-  public void write(String sensorType, Configurations configurations, List<Tuple> tuples, List<JSONObject> messages) throws Exception {
+  public void write(String sensorType, EnrichmentConfigurations configurations, List<Tuple> tuples, List<JSONObject> messages) throws Exception {
     SensorEnrichmentConfig sensorEnrichmentConfig = configurations.getSensorEnrichmentConfig(sensorType);
     String indexPostfix = dateFormat.format(new Date());
     BulkRequestBuilder bulkRequest = client.prepareBulk();

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/df8d682e/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/integration/ElasticsearchEnrichmentIntegrationTest.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/integration/ElasticsearchEnrichmentIntegrationTest.java b/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/integration/ElasticsearchEnrichmentIntegrationTest.java
index 6931aff..f7d96a4 100644
--- a/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/integration/ElasticsearchEnrichmentIntegrationTest.java
+++ b/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/integration/ElasticsearchEnrichmentIntegrationTest.java
@@ -55,7 +55,7 @@ public class ElasticsearchEnrichmentIntegrationTest extends EnrichmentIntegratio
         if (elasticSearchComponent.hasIndex(index)) {
           List<Map<String, Object>> docsFromDisk;
           try {
-            docs = elasticSearchComponent.getAllIndexedDocs(index, "yaf_doc");
+            docs = elasticSearchComponent.getAllIndexedDocs(index, testSensorType + "_doc");
             docsFromDisk = readDocsFromDisk(hdfsDir);
             System.out.println(docs.size() + " vs " + inputMessages.size() + " vs " + docsFromDisk.size());
           } catch (IOException e) {

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/df8d682e/metron-platform/metron-enrichment/src/main/assembly/assembly.xml
----------------------------------------------------------------------
diff --git a/metron-platform/metron-enrichment/src/main/assembly/assembly.xml b/metron-platform/metron-enrichment/src/main/assembly/assembly.xml
index fdc8f36..6c9165c 100644
--- a/metron-platform/metron-enrichment/src/main/assembly/assembly.xml
+++ b/metron-platform/metron-enrichment/src/main/assembly/assembly.xml
@@ -19,6 +19,18 @@
   <includeBaseDirectory>false</includeBaseDirectory>
   <fileSets>
     <fileSet>
+      <directory>${project.basedir}/src/main/config</directory>
+      <outputDirectory>/config</outputDirectory>
+      <useDefaultExcludes>true</useDefaultExcludes>
+      <excludes>
+        <exclude>**/*.formatted</exclude>
+        <exclude>**/*.filtered</exclude>
+      </excludes>
+      <fileMode>0644</fileMode>
+      <lineEnding>unix</lineEnding>
+      <filtered>true</filtered>
+    </fileSet>
+    <fileSet>
       <directory>${project.basedir}/src/main/scripts</directory>
       <outputDirectory>/bin</outputDirectory>
       <useDefaultExcludes>true</useDefaultExcludes>

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/df8d682e/metron-platform/metron-enrichment/src/main/config/zookeeper/enrichments/bro.json
----------------------------------------------------------------------
diff --git a/metron-platform/metron-enrichment/src/main/config/zookeeper/enrichments/bro.json b/metron-platform/metron-enrichment/src/main/config/zookeeper/enrichments/bro.json
new file mode 100644
index 0000000..0eb34b3
--- /dev/null
+++ b/metron-platform/metron-enrichment/src/main/config/zookeeper/enrichments/bro.json
@@ -0,0 +1,20 @@
+{
+  "index": "bro",
+  "batchSize": 5,
+  "enrichment" : {
+    "fieldMap": {
+      "geo": ["ip_dst_addr", "ip_src_addr"],
+      "host": ["host"]
+    }
+  },
+  "threatIntel": {
+    "fieldMap": {
+      "hbaseThreatIntel": ["ip_src_addr", "ip_dst_addr"]
+    },
+    "fieldToTypeMap": {
+      "ip_src_addr" : ["malicious_ip"],
+      "ip_dst_addr" : ["malicious_ip"]
+    }
+  }
+}
+

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/df8d682e/metron-platform/metron-enrichment/src/main/config/zookeeper/enrichments/snort.json
----------------------------------------------------------------------
diff --git a/metron-platform/metron-enrichment/src/main/config/zookeeper/enrichments/snort.json b/metron-platform/metron-enrichment/src/main/config/zookeeper/enrichments/snort.json
new file mode 100644
index 0000000..9dfc80e
--- /dev/null
+++ b/metron-platform/metron-enrichment/src/main/config/zookeeper/enrichments/snort.json
@@ -0,0 +1,28 @@
+{
+  "index": "snort",
+  "batchSize": 1,
+  "enrichment" : {
+    "fieldMap":
+      {
+      "geo": ["ip_dst_addr", "ip_src_addr"],
+      "host": ["host"]
+    }
+  },
+  "threatIntel" : {
+    "fieldMap":
+      {
+      "hbaseThreatIntel": ["ip_src_addr", "ip_dst_addr"]
+    },
+    "fieldToTypeMap":
+      {
+      "ip_src_addr" : ["malicious_ip"],
+      "ip_dst_addr" : ["malicious_ip"]
+    },
+    "triageConfig" : {
+      "riskLevelRules" : {
+        "not(IN_SUBNET(ip_dst_addr, '192.168.0.0/24'))" : 10
+      },
+      "aggregator" : "MAX"
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/df8d682e/metron-platform/metron-enrichment/src/main/config/zookeeper/enrichments/websphere.json
----------------------------------------------------------------------
diff --git a/metron-platform/metron-enrichment/src/main/config/zookeeper/enrichments/websphere.json b/metron-platform/metron-enrichment/src/main/config/zookeeper/enrichments/websphere.json
new file mode 100644
index 0000000..b765808
--- /dev/null
+++ b/metron-platform/metron-enrichment/src/main/config/zookeeper/enrichments/websphere.json
@@ -0,0 +1,20 @@
+{
+  "index": "websphere",
+  "batchSize": 5,
+  "enrichment": {
+    "fieldMap": {
+      "geo": [
+        "ip_src_addr"
+      ],
+      "host": [
+        "ip_src_addr"
+      ]
+    },
+  "fieldToTypeMap": {
+      "ip_src_addr": [
+        "playful_classification"
+      ]
+    }
+  }
+}
+

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/df8d682e/metron-platform/metron-enrichment/src/main/config/zookeeper/enrichments/yaf.json
----------------------------------------------------------------------
diff --git a/metron-platform/metron-enrichment/src/main/config/zookeeper/enrichments/yaf.json b/metron-platform/metron-enrichment/src/main/config/zookeeper/enrichments/yaf.json
new file mode 100644
index 0000000..4e67748
--- /dev/null
+++ b/metron-platform/metron-enrichment/src/main/config/zookeeper/enrichments/yaf.json
@@ -0,0 +1,22 @@
+{
+  "index": "yaf",
+  "batchSize": 5,
+  "enrichment" : {
+    "fieldMap":
+      {
+      "geo": ["ip_dst_addr", "ip_src_addr"],
+      "host": ["host"]
+    }
+  },
+  "threatIntel": {
+    "fieldMap":
+      {
+      "hbaseThreatIntel": ["ip_src_addr", "ip_dst_addr"]
+    },
+    "fieldToTypeMap":
+      {
+      "ip_src_addr" : ["malicious_ip"],
+      "ip_dst_addr" : ["malicious_ip"]
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/df8d682e/metron-platform/metron-enrichment/src/main/flux/enrichment/test.yaml
----------------------------------------------------------------------
diff --git a/metron-platform/metron-enrichment/src/main/flux/enrichment/test.yaml b/metron-platform/metron-enrichment/src/main/flux/enrichment/test.yaml
index dcc507c..226b686 100644
--- a/metron-platform/metron-enrichment/src/main/flux/enrichment/test.yaml
+++ b/metron-platform/metron-enrichment/src/main/flux/enrichment/test.yaml
@@ -156,16 +156,6 @@ components:
                 value: -2
 
 spouts:
-    -   id: "testingSpout"
-        className: "org.apache.metron.test.spouts.GenericInternalTestSpout"
-        parallelism: 1
-        configMethods:
-            -   name: "withFilename"
-                args:
-                    - "../metron-integration-test/src/main/resources/sample/data/SampleInput/YafExampleOutput"
-            -   name: "withRepeating"
-                args:
-                    - true
     -   id: "kafkaSpout"
         className: "storm.kafka.KafkaSpout"
         constructorArgs:

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/df8d682e/metron-platform/metron-enrichment/src/main/java/org/apache/metron/enrichment/bolt/BulkMessageWriterBolt.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-enrichment/src/main/java/org/apache/metron/enrichment/bolt/BulkMessageWriterBolt.java b/metron-platform/metron-enrichment/src/main/java/org/apache/metron/enrichment/bolt/BulkMessageWriterBolt.java
index 7cfa34d..3e407c7 100644
--- a/metron-platform/metron-enrichment/src/main/java/org/apache/metron/enrichment/bolt/BulkMessageWriterBolt.java
+++ b/metron-platform/metron-enrichment/src/main/java/org/apache/metron/enrichment/bolt/BulkMessageWriterBolt.java
@@ -25,6 +25,7 @@ import backtype.storm.tuple.Tuple;
 import org.apache.metron.common.Constants;
 import org.apache.metron.common.bolt.ConfiguredBolt;
 import org.apache.metron.common.configuration.enrichment.SensorEnrichmentConfig;
+import org.apache.metron.common.bolt.ConfiguredEnrichmentBolt;
 import org.apache.metron.common.utils.ErrorUtils;
 import org.apache.metron.common.utils.MessageUtils;
 import org.apache.metron.common.interfaces.BulkMessageWriter;
@@ -34,7 +35,7 @@ import org.slf4j.LoggerFactory;
 
 import java.util.*;
 
-public class BulkMessageWriterBolt extends ConfiguredBolt {
+public class BulkMessageWriterBolt extends ConfiguredEnrichmentBolt {
 
   private static final Logger LOG = LoggerFactory
           .getLogger(BulkMessageWriterBolt.class);

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/df8d682e/metron-platform/metron-enrichment/src/main/java/org/apache/metron/enrichment/bolt/GenericEnrichmentBolt.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-enrichment/src/main/java/org/apache/metron/enrichment/bolt/GenericEnrichmentBolt.java b/metron-platform/metron-enrichment/src/main/java/org/apache/metron/enrichment/bolt/GenericEnrichmentBolt.java
index b79d6c7..234c795 100644
--- a/metron-platform/metron-enrichment/src/main/java/org/apache/metron/enrichment/bolt/GenericEnrichmentBolt.java
+++ b/metron-platform/metron-enrichment/src/main/java/org/apache/metron/enrichment/bolt/GenericEnrichmentBolt.java
@@ -28,12 +28,12 @@ import com.google.common.cache.CacheBuilder;
 import com.google.common.cache.CacheLoader;
 import com.google.common.cache.LoadingCache;
 import org.apache.metron.common.Constants;
-import org.apache.metron.common.bolt.ConfiguredBolt;
-import org.apache.metron.common.configuration.Configurations;
-import org.apache.metron.enrichment.configuration.Enrichment;
+import org.apache.metron.common.bolt.ConfiguredEnrichmentBolt;
+import org.apache.metron.common.configuration.ConfigurationType;
 import org.apache.metron.common.configuration.enrichment.SensorEnrichmentConfig;
-import org.apache.metron.enrichment.interfaces.EnrichmentAdapter;
 import org.apache.metron.common.utils.ErrorUtils;
+import org.apache.metron.enrichment.configuration.Enrichment;
+import org.apache.metron.enrichment.interfaces.EnrichmentAdapter;
 import org.json.simple.JSONObject;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -61,7 +61,7 @@ import java.util.concurrent.TimeUnit;
  **/
 
 @SuppressWarnings({"rawtypes", "serial"})
-public class GenericEnrichmentBolt extends ConfiguredBolt {
+public class GenericEnrichmentBolt extends ConfiguredEnrichmentBolt {
 
   private static final Logger LOG = LoggerFactory
           .getLogger(GenericEnrichmentBolt.class);
@@ -114,8 +114,9 @@ public class GenericEnrichmentBolt extends ConfiguredBolt {
     this.invalidateCacheOnReload= cacheInvalidationOnReload;
     return this;
   }
+
   @Override
-  public void reloadCallback(String name, Configurations.Type type) {
+  public void reloadCallback(String name, ConfigurationType type) {
     if(invalidateCacheOnReload) {
       if (cache != null) {
         cache.invalidateAll();

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/df8d682e/metron-platform/metron-enrichment/src/main/java/org/apache/metron/enrichment/bolt/JoinBolt.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-enrichment/src/main/java/org/apache/metron/enrichment/bolt/JoinBolt.java b/metron-platform/metron-enrichment/src/main/java/org/apache/metron/enrichment/bolt/JoinBolt.java
index 1964961..771097f 100644
--- a/metron-platform/metron-enrichment/src/main/java/org/apache/metron/enrichment/bolt/JoinBolt.java
+++ b/metron-platform/metron-enrichment/src/main/java/org/apache/metron/enrichment/bolt/JoinBolt.java
@@ -31,6 +31,7 @@ import com.google.common.collect.Sets;
 import org.apache.metron.common.bolt.ConfiguredBolt;
 import org.apache.metron.common.utils.ErrorUtils;
 import org.json.simple.JSONObject;
+import org.apache.metron.common.bolt.ConfiguredEnrichmentBolt;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import java.util.HashMap;
@@ -39,7 +40,7 @@ import java.util.Set;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.TimeUnit;
 
-public abstract class JoinBolt<V> extends ConfiguredBolt {
+public abstract class JoinBolt<V> extends ConfiguredEnrichmentBolt {
 
   private static final Logger LOG = LoggerFactory
           .getLogger(JoinBolt.class);

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/df8d682e/metron-platform/metron-enrichment/src/main/java/org/apache/metron/enrichment/bolt/SplitBolt.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-enrichment/src/main/java/org/apache/metron/enrichment/bolt/SplitBolt.java b/metron-platform/metron-enrichment/src/main/java/org/apache/metron/enrichment/bolt/SplitBolt.java
index 4ff387c..3063b7f 100644
--- a/metron-platform/metron-enrichment/src/main/java/org/apache/metron/enrichment/bolt/SplitBolt.java
+++ b/metron-platform/metron-enrichment/src/main/java/org/apache/metron/enrichment/bolt/SplitBolt.java
@@ -24,12 +24,13 @@ import backtype.storm.tuple.Fields;
 import backtype.storm.tuple.Tuple;
 import backtype.storm.tuple.Values;
 import org.apache.metron.common.bolt.ConfiguredBolt;
+import org.apache.metron.common.bolt.ConfiguredEnrichmentBolt;
 
 import java.util.Map;
 import java.util.Set;
 
 public abstract class SplitBolt<T> extends
-        ConfiguredBolt {
+        ConfiguredEnrichmentBolt {
 
   protected OutputCollector collector;
 

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/df8d682e/metron-platform/metron-enrichment/src/main/java/org/apache/metron/writer/hdfs/HdfsWriter.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-enrichment/src/main/java/org/apache/metron/writer/hdfs/HdfsWriter.java b/metron-platform/metron-enrichment/src/main/java/org/apache/metron/writer/hdfs/HdfsWriter.java
index a364419..d8f0e73 100644
--- a/metron-platform/metron-enrichment/src/main/java/org/apache/metron/writer/hdfs/HdfsWriter.java
+++ b/metron-platform/metron-enrichment/src/main/java/org/apache/metron/writer/hdfs/HdfsWriter.java
@@ -18,7 +18,7 @@
 package org.apache.metron.writer.hdfs;
 
 import backtype.storm.tuple.Tuple;
-import org.apache.metron.common.configuration.Configurations;
+import org.apache.metron.common.configuration.EnrichmentConfigurations;
 import org.apache.metron.common.interfaces.BulkMessageWriter;
 import org.apache.storm.hdfs.bolt.format.FileNameFormat;
 import org.apache.storm.hdfs.bolt.rotation.FileRotationPolicy;
@@ -62,13 +62,13 @@ public class HdfsWriter implements BulkMessageWriter<JSONObject>, Serializable {
   }
 
   @Override
-  public void init(Map stormConfig, Configurations configurations) {
+  public void init(Map stormConfig, EnrichmentConfigurations configurations) {
     this.stormConfig = stormConfig;
   }
 
   @Override
-  public void write( String sourceType
-                   , Configurations configurations
+  public void write(String sourceType
+                   , EnrichmentConfigurations configurations
                    , List<Tuple> tuples
                    , List<JSONObject> messages
                    ) throws Exception

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/df8d682e/metron-platform/metron-enrichment/src/test/java/org/apache/metron/enrichment/bolt/BulkMessageWriterBoltTest.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-enrichment/src/test/java/org/apache/metron/enrichment/bolt/BulkMessageWriterBoltTest.java b/metron-platform/metron-enrichment/src/test/java/org/apache/metron/enrichment/bolt/BulkMessageWriterBoltTest.java
index bcf4d04..61961a7 100644
--- a/metron-platform/metron-enrichment/src/test/java/org/apache/metron/enrichment/bolt/BulkMessageWriterBoltTest.java
+++ b/metron-platform/metron-enrichment/src/test/java/org/apache/metron/enrichment/bolt/BulkMessageWriterBoltTest.java
@@ -21,6 +21,7 @@ import backtype.storm.tuple.Tuple;
 import backtype.storm.tuple.Values;
 import org.adrianwalker.multilinestring.Multiline;
 import org.apache.metron.common.Constants;
+import org.apache.metron.common.configuration.EnrichmentConfigurations;
 import org.apache.metron.test.bolt.BaseEnrichmentBoltTest;
 import org.apache.metron.common.configuration.Configurations;
 import org.apache.metron.common.interfaces.BulkMessageWriter;
@@ -77,7 +78,7 @@ public class BulkMessageWriterBoltTest extends BaseEnrichmentBoltTest {
   /**
    * {
    * "field": "value",
-   * "source.type": "yaf"
+   * "source.type": "test"
    * }
    */
   @Multiline
@@ -116,28 +117,28 @@ public class BulkMessageWriterBoltTest extends BaseEnrichmentBoltTest {
     bulkMessageWriterBolt.declareOutputFields(declarer);
     verify(declarer, times(1)).declareStream(eq("error"), argThat(new FieldsMatcher("message")));
     Map stormConf = new HashMap();
-    doThrow(new Exception()).when(bulkMessageWriter).init(eq(stormConf), any(Configurations.class));
+    doThrow(new Exception()).when(bulkMessageWriter).init(eq(stormConf), any(EnrichmentConfigurations.class));
     try {
       bulkMessageWriterBolt.prepare(stormConf, topologyContext, outputCollector);
       fail("A runtime exception should be thrown when bulkMessageWriter.init throws an exception");
     } catch(RuntimeException e) {}
     reset(bulkMessageWriter);
     bulkMessageWriterBolt.prepare(stormConf, topologyContext, outputCollector);
-    verify(bulkMessageWriter, times(1)).init(eq(stormConf), any(Configurations.class));
+    verify(bulkMessageWriter, times(1)).init(eq(stormConf), any(EnrichmentConfigurations.class));
     tupleList = new ArrayList<>();
     for(int i = 0; i < 4; i++) {
       when(tuple.getValueByField("message")).thenReturn(messageList.get(i));
       tupleList.add(tuple);
       bulkMessageWriterBolt.execute(tuple);
-      verify(bulkMessageWriter, times(0)).write(eq(sensorType), any(Configurations.class), eq(tupleList), eq(messageList));
+      verify(bulkMessageWriter, times(0)).write(eq(sensorType), any(EnrichmentConfigurations.class), eq(tupleList), eq(messageList));
     }
     when(tuple.getValueByField("message")).thenReturn(messageList.get(4));
     tupleList.add(tuple);
     bulkMessageWriterBolt.execute(tuple);
-    verify(bulkMessageWriter, times(1)).write(eq(sensorType), any(Configurations.class), eq(tupleList), argThat(new MessageListMatcher(messageList)));
+    verify(bulkMessageWriter, times(1)).write(eq(sensorType), any(EnrichmentConfigurations.class), eq(tupleList), argThat(new MessageListMatcher(messageList)));
     verify(outputCollector, times(5)).ack(tuple);
     reset(outputCollector);
-    doThrow(new Exception()).when(bulkMessageWriter).write(eq(sensorType), any(Configurations.class), Matchers.anyListOf(Tuple.class), Matchers.anyListOf(JSONObject.class));
+    doThrow(new Exception()).when(bulkMessageWriter).write(eq(sensorType), any(EnrichmentConfigurations.class), Matchers.anyListOf(Tuple.class), Matchers.anyListOf(JSONObject.class));
     when(tuple.getValueByField("message")).thenReturn(messageList.get(0));
     for(int i = 0; i < 5; i++) {
       bulkMessageWriterBolt.execute(tuple);

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/df8d682e/metron-platform/metron-enrichment/src/test/java/org/apache/metron/enrichment/bolt/EnrichmentJoinBoltTest.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-enrichment/src/test/java/org/apache/metron/enrichment/bolt/EnrichmentJoinBoltTest.java b/metron-platform/metron-enrichment/src/test/java/org/apache/metron/enrichment/bolt/EnrichmentJoinBoltTest.java
index f760e5a..d7b3629 100644
--- a/metron-platform/metron-enrichment/src/test/java/org/apache/metron/enrichment/bolt/EnrichmentJoinBoltTest.java
+++ b/metron-platform/metron-enrichment/src/test/java/org/apache/metron/enrichment/bolt/EnrichmentJoinBoltTest.java
@@ -47,7 +47,7 @@ public class EnrichmentJoinBoltTest extends BaseEnrichmentBoltTest {
    * {
    * "ip_src_addr": "ip1",
    * "ip_dst_addr": "ip2",
-   * "source.type": "yaf",
+   * "source.type": "test",
    * "enrichedField": "enrichedValue"
    * }
    */

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/df8d682e/metron-platform/metron-enrichment/src/test/java/org/apache/metron/enrichment/bolt/GenericEnrichmentBoltTest.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-enrichment/src/test/java/org/apache/metron/enrichment/bolt/GenericEnrichmentBoltTest.java b/metron-platform/metron-enrichment/src/test/java/org/apache/metron/enrichment/bolt/GenericEnrichmentBoltTest.java
index 5a09f79..a9a1637 100644
--- a/metron-platform/metron-enrichment/src/test/java/org/apache/metron/enrichment/bolt/GenericEnrichmentBoltTest.java
+++ b/metron-platform/metron-enrichment/src/test/java/org/apache/metron/enrichment/bolt/GenericEnrichmentBoltTest.java
@@ -80,7 +80,7 @@ public class GenericEnrichmentBoltTest extends BaseEnrichmentBoltTest {
    {
    "field1": "value1",
    "field2": "value2",
-   "source.type": "yaf"
+   "source.type": "test"
    }
    */
   @Multiline
@@ -106,7 +106,7 @@ public class GenericEnrichmentBoltTest extends BaseEnrichmentBoltTest {
    {
    "field1.enrichedField1": "enrichedValue1",
    "field2.enrichedField2": "enrichedValue2",
-   "source.type": "yaf"
+   "source.type": "test"
    }
    */
   @Multiline

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/df8d682e/metron-platform/metron-enrichment/src/test/java/org/apache/metron/enrichment/bolt/SplitBoltTest.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-enrichment/src/test/java/org/apache/metron/enrichment/bolt/SplitBoltTest.java b/metron-platform/metron-enrichment/src/test/java/org/apache/metron/enrichment/bolt/SplitBoltTest.java
index 7166b1b..fc18ad0 100644
--- a/metron-platform/metron-enrichment/src/test/java/org/apache/metron/enrichment/bolt/SplitBoltTest.java
+++ b/metron-platform/metron-enrichment/src/test/java/org/apache/metron/enrichment/bolt/SplitBoltTest.java
@@ -22,8 +22,8 @@ import backtype.storm.topology.OutputFieldsDeclarer;
 import backtype.storm.tuple.Tuple;
 import backtype.storm.tuple.Values;
 import junit.framework.Assert;
+import org.apache.metron.common.configuration.ConfigurationType;
 import org.apache.metron.test.bolt.BaseEnrichmentBoltTest;
-import org.apache.metron.common.configuration.Configurations;
 import org.json.simple.JSONObject;
 import org.junit.Test;
 
@@ -35,11 +35,7 @@ import static org.mockito.Matchers.any;
 import static org.mockito.Matchers.anyString;
 import static org.mockito.Matchers.argThat;
 import static org.mockito.Matchers.eq;
-import static org.mockito.Mockito.doCallRealMethod;
-import static org.mockito.Mockito.doReturn;
-import static org.mockito.Mockito.spy;
-import static org.mockito.Mockito.times;
-import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.*;
 
 public class SplitBoltTest extends BaseEnrichmentBoltTest {
 
@@ -91,7 +87,7 @@ public class SplitBoltTest extends BaseEnrichmentBoltTest {
     StandAloneSplitBolt splitBolt = spy(new StandAloneSplitBolt("zookeeperUrl"));
     splitBolt.setCuratorFramework(client);
     splitBolt.setTreeCache(cache);
-    doCallRealMethod().when(splitBolt).reloadCallback(anyString(), any(Configurations.Type.class));
+    doCallRealMethod().when(splitBolt).reloadCallback(anyString(), any(ConfigurationType.class));
     splitBolt.prepare(new HashMap(), topologyContext, outputCollector);
     splitBolt.declareOutputFields(declarer);
     verify(declarer, times(1)).declareStream(eq("message"), argThat(new FieldsMatcher("key", "message")));

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/df8d682e/metron-platform/metron-enrichment/src/test/java/org/apache/metron/enrichment/bolt/ThreatIntelJoinBoltTest.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-enrichment/src/test/java/org/apache/metron/enrichment/bolt/ThreatIntelJoinBoltTest.java b/metron-platform/metron-enrichment/src/test/java/org/apache/metron/enrichment/bolt/ThreatIntelJoinBoltTest.java
index d3a5414..b109960 100644
--- a/metron-platform/metron-enrichment/src/test/java/org/apache/metron/enrichment/bolt/ThreatIntelJoinBoltTest.java
+++ b/metron-platform/metron-enrichment/src/test/java/org/apache/metron/enrichment/bolt/ThreatIntelJoinBoltTest.java
@@ -42,7 +42,7 @@ public class ThreatIntelJoinBoltTest extends BaseEnrichmentBoltTest {
    {
    "field1": "value1",
    "enrichedField1": "enrichedValue1",
-   "source.type": "yaf"
+   "source.type": "test"
    }
    */
   @Multiline
@@ -52,7 +52,7 @@ public class ThreatIntelJoinBoltTest extends BaseEnrichmentBoltTest {
    {
    "field1": "value1",
    "enrichedField1": "enrichedValue1",
-   "source.type": "yaf",
+   "source.type": "test",
    "threatintels.field.end.ts": "timing"
    }
    */
@@ -63,7 +63,7 @@ public class ThreatIntelJoinBoltTest extends BaseEnrichmentBoltTest {
    {
    "field1": "value1",
    "enrichedField1": "enrichedValue1",
-   "source.type": "yaf",
+   "source.type": "test",
    "threatintels.field": "threatIntelValue"
    }
    */

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/df8d682e/metron-platform/metron-integration-test/src/main/config/zookeeper/enrichments/test.json
----------------------------------------------------------------------
diff --git a/metron-platform/metron-integration-test/src/main/config/zookeeper/enrichments/test.json b/metron-platform/metron-integration-test/src/main/config/zookeeper/enrichments/test.json
new file mode 100644
index 0000000..d72851e
--- /dev/null
+++ b/metron-platform/metron-integration-test/src/main/config/zookeeper/enrichments/test.json
@@ -0,0 +1,51 @@
+{
+  "index": "yaf",
+  "batchSize": 5,
+  "enrichment": {
+    "fieldMap": {
+      "geo": [
+        "ip_src_addr",
+        "ip_dst_addr"
+      ],
+      "host": [
+        "ip_src_addr",
+        "ip_dst_addr"
+      ],
+      "hbaseEnrichment": [
+        "ip_src_addr",
+        "ip_dst_addr"
+      ]
+    }
+  ,"fieldToTypeMap": {
+      "ip_src_addr": [
+        "playful_classification"
+      ],
+      "ip_dst_addr": [
+        "playful_classification"
+      ]
+    }
+  },
+  "threatIntel": {
+    "fieldMap": {
+      "hbaseThreatIntel": [
+        "ip_src_addr",
+        "ip_dst_addr"
+      ]
+    },
+    "fieldToTypeMap": {
+      "ip_src_addr": [
+        "malicious_ip"
+      ],
+      "ip_dst_addr": [
+        "malicious_ip"
+      ]
+    },
+    "triageConfig" : {
+      "riskLevelRules" : {
+        "ip_src_addr == '10.0.2.3' or ip_dst_addr == '10.0.2.3'" : 10
+      },
+      "aggregator" : "MAX"
+    }
+  }
+}
+

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/df8d682e/metron-platform/metron-integration-test/src/main/config/zookeeper/global.json
----------------------------------------------------------------------
diff --git a/metron-platform/metron-integration-test/src/main/config/zookeeper/global.json b/metron-platform/metron-integration-test/src/main/config/zookeeper/global.json
new file mode 100644
index 0000000..721f70f
--- /dev/null
+++ b/metron-platform/metron-integration-test/src/main/config/zookeeper/global.json
@@ -0,0 +1,10 @@
+{
+  "es.clustername": "metron",
+  "es.ip": "localhost",
+  "es.port": 9300,
+  "es.date.format": "yyyy.MM.dd.HH",
+  "solr.zookeeper": "localhost:2181",
+  "solr.collection": "metron",
+  "solr.numShards": 1,
+  "solr.replicationFactor": 1
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/df8d682e/metron-platform/metron-integration-test/src/main/java/org/apache/metron/integration/BaseIntegrationTest.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-integration-test/src/main/java/org/apache/metron/integration/BaseIntegrationTest.java b/metron-platform/metron-integration-test/src/main/java/org/apache/metron/integration/BaseIntegrationTest.java
index ec04fe0..1a3247c 100644
--- a/metron-platform/metron-integration-test/src/main/java/org/apache/metron/integration/BaseIntegrationTest.java
+++ b/metron-platform/metron-integration-test/src/main/java/org/apache/metron/integration/BaseIntegrationTest.java
@@ -34,12 +34,7 @@ public abstract class BaseIntegrationTest {
               @Nullable
               @Override
               public Void apply(@Nullable KafkaWithZKComponent kafkaWithZKComponent) {
-                topologyProperties.setProperty("kafka.zk", kafkaWithZKComponent.getZookeeperConnect());
-                try {
-                  ConfigurationsUtils.uploadConfigsToZookeeper(TestConstants.SAMPLE_CONFIG_PATH, kafkaWithZKComponent.getZookeeperConnect());
-                } catch (Exception e) {
-                  throw new IllegalStateException(e);
-                }
+                topologyProperties.setProperty(KafkaWithZKComponent.ZOOKEEPER_PROPERTY, kafkaWithZKComponent.getZookeeperConnect());
                 return null;
               }
             });

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/df8d682e/metron-platform/metron-integration-test/src/main/java/org/apache/metron/integration/EnrichmentIntegrationTest.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-integration-test/src/main/java/org/apache/metron/integration/EnrichmentIntegrationTest.java b/metron-platform/metron-integration-test/src/main/java/org/apache/metron/integration/EnrichmentIntegrationTest.java
index b392448..818dae0 100644
--- a/metron-platform/metron-integration-test/src/main/java/org/apache/metron/integration/EnrichmentIntegrationTest.java
+++ b/metron-platform/metron-integration-test/src/main/java/org/apache/metron/integration/EnrichmentIntegrationTest.java
@@ -26,10 +26,12 @@ import org.apache.hadoop.hbase.client.HTableInterface;
 import org.apache.metron.common.Constants;
 import org.apache.metron.TestConstants;
 import org.apache.metron.common.configuration.Configurations;
+import org.apache.metron.common.configuration.EnrichmentConfigurations;
 import org.apache.metron.hbase.TableProvider;
 import org.apache.metron.enrichment.converter.EnrichmentKey;
 import org.apache.metron.enrichment.converter.EnrichmentValue;
 import org.apache.metron.enrichment.converter.EnrichmentHelper;
+import org.apache.metron.integration.components.ConfigUploadComponent;
 import org.apache.metron.integration.utils.TestUtils;
 import org.apache.metron.test.utils.UnitTestHelper;
 import org.apache.metron.integration.components.FluxTopologyComponent;
@@ -57,6 +59,7 @@ import java.util.Set;
 import java.util.Stack;
 
 public abstract class EnrichmentIntegrationTest extends BaseIntegrationTest {
+
   private static final String SRC_IP = "ip_src_addr";
   private static final String DST_IP = "ip_dst_addr";
   private static final String MALICIOUS_IP_TYPE = "malicious_ip";
@@ -65,9 +68,10 @@ public abstract class EnrichmentIntegrationTest extends BaseIntegrationTest {
     put("orientation", "north");
   }};
   private String fluxPath = "../metron-enrichment/src/main/flux/enrichment/test.yaml";
+  protected String testSensorType = "test";
   protected String hdfsDir = "target/enrichmentIntegrationTest/hdfs";
-  private String sampleParsedPath = TestConstants.SAMPLE_DATA_PARSED_PATH + "YafExampleParsed";
-  private String sampleIndexedPath = TestConstants.SAMPLE_DATA_INDEXED_PATH + "YafIndexed";
+  private String sampleParsedPath = TestConstants.SAMPLE_DATA_PARSED_PATH + "TestExampleParsed";
+  private String sampleIndexedPath = TestConstants.SAMPLE_DATA_INDEXED_PATH + "TestIndexed";
 
 
   public static class Provider implements TableProvider, Serializable {
@@ -140,7 +144,7 @@ public abstract class EnrichmentIntegrationTest extends BaseIntegrationTest {
   @Test
   public void test() throws Exception {
     cleanHdfsDir(hdfsDir);
-    final Configurations configurations = SampleUtil.getSampleConfigs();
+    final EnrichmentConfigurations configurations = SampleUtil.getSampleEnrichmentConfigs();
     final String dateFormat = "yyyy.MM.dd.HH";
     final List<byte[]> inputMessages = TestUtils.readSampleData(sampleParsedPath);
     final String cf = "cf";
@@ -170,6 +174,11 @@ public abstract class EnrichmentIntegrationTest extends BaseIntegrationTest {
       add(new KafkaWithZKComponent.Topic(Constants.ENRICHMENT_TOPIC, 1));
     }});
 
+    ConfigUploadComponent configUploadComponent = new ConfigUploadComponent()
+            .withTopologyProperties(topologyProperties)
+            .withGlobalConfigsPath(TestConstants.SAMPLE_CONFIG_PATH)
+            .withEnrichmentConfigsPath(TestConstants.SAMPLE_CONFIG_PATH);
+
     //create MockHBaseTables
     final MockHTable trackerTable = (MockHTable)MockHTable.Provider.addToCache(trackerHBaseTableName, cf);
     final MockHTable threatIntelTable = (MockHTable)MockHTable.Provider.addToCache(threatIntelTableName, cf);
@@ -194,6 +203,7 @@ public abstract class EnrichmentIntegrationTest extends BaseIntegrationTest {
     UnitTestHelper.verboseLogging();
     ComponentRunner runner = new ComponentRunner.Builder()
             .withComponent("kafka", kafkaComponent)
+            .withComponent("config", configUploadComponent)
             .withComponent("search", searchComponent)
             .withComponent("storm", fluxComponent)
             .withMillisecondsBetweenAttempts(10000)
@@ -214,7 +224,7 @@ public abstract class EnrichmentIntegrationTest extends BaseIntegrationTest {
       List<Map<String, Object>> docsFromDisk = readDocsFromDisk(hdfsDir);
       Assert.assertEquals(docsFromDisk.size(), docs.size()) ;
       Assert.assertEquals(new File(hdfsDir).list().length, 1);
-      Assert.assertEquals(new File(hdfsDir).list()[0], "yaf");
+      Assert.assertEquals(new File(hdfsDir).list()[0], testSensorType);
       validateAll(docsFromDisk);
     }
     finally {

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/df8d682e/metron-platform/metron-integration-test/src/main/java/org/apache/metron/integration/components/ConfigUploadComponent.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-integration-test/src/main/java/org/apache/metron/integration/components/ConfigUploadComponent.java b/metron-platform/metron-integration-test/src/main/java/org/apache/metron/integration/components/ConfigUploadComponent.java
new file mode 100644
index 0000000..0a088cb
--- /dev/null
+++ b/metron-platform/metron-integration-test/src/main/java/org/apache/metron/integration/components/ConfigUploadComponent.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.integration.components;
+
+import org.apache.curator.framework.CuratorFramework;
+import org.apache.metron.common.configuration.ConfigurationsUtils;
+import org.apache.metron.common.configuration.SensorParserConfig;
+import org.apache.metron.integration.InMemoryComponent;
+import org.apache.metron.integration.UnableToStartException;
+
+import java.util.Properties;
+
+public class ConfigUploadComponent implements InMemoryComponent {
+
+  private Properties topologyProperties;
+  private String globalConfigPath;
+  private String parserConfigsPath;
+  private String enrichmentConfigsPath;
+
+  public ConfigUploadComponent withTopologyProperties(Properties topologyProperties) {
+    this.topologyProperties = topologyProperties;
+    return this;
+  }
+
+  public ConfigUploadComponent withGlobalConfigsPath(String globalConfigPath) {
+    this.globalConfigPath = globalConfigPath;
+    return this;
+  }
+
+  public ConfigUploadComponent withParserConfigsPath(String parserConfigsPath) {
+    this.parserConfigsPath = parserConfigsPath;
+    return this;
+  }
+  public ConfigUploadComponent withEnrichmentConfigsPath(String enrichmentConfigsPath) {
+    this.enrichmentConfigsPath = enrichmentConfigsPath;
+    return this;
+  }
+
+
+  @Override
+  public void start() throws UnableToStartException {
+    try {
+      ConfigurationsUtils.uploadConfigsToZookeeper(globalConfigPath, parserConfigsPath, enrichmentConfigsPath, topologyProperties.getProperty(KafkaWithZKComponent.ZOOKEEPER_PROPERTY));
+    } catch (Exception e) {
+      throw new UnableToStartException(e.getMessage(), e);
+    }
+  }
+
+  public SensorParserConfig getSensorParserConfig(String sensorType) {
+    SensorParserConfig sensorParserConfig = new SensorParserConfig();
+    CuratorFramework client = ConfigurationsUtils.getClient(topologyProperties.getProperty(KafkaWithZKComponent.ZOOKEEPER_PROPERTY));
+    client.start();
+    try {
+      sensorParserConfig = ConfigurationsUtils.readSensorParserConfigFromZookeeper(sensorType, client);
+    } catch (Exception e) {
+      e.printStackTrace();
+    } finally {
+      client.close();
+    }
+    return sensorParserConfig;
+  }
+
+  @Override
+  public void stop() {
+
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/df8d682e/metron-platform/metron-integration-test/src/main/java/org/apache/metron/integration/components/KafkaWithZKComponent.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-integration-test/src/main/java/org/apache/metron/integration/components/KafkaWithZKComponent.java b/metron-platform/metron-integration-test/src/main/java/org/apache/metron/integration/components/KafkaWithZKComponent.java
index 42d7567..f991fe2 100644
--- a/metron-platform/metron-integration-test/src/main/java/org/apache/metron/integration/components/KafkaWithZKComponent.java
+++ b/metron-platform/metron-integration-test/src/main/java/org/apache/metron/integration/components/KafkaWithZKComponent.java
@@ -44,6 +44,7 @@ import java.util.*;
 
 public class KafkaWithZKComponent implements InMemoryComponent {
 
+  public static final String ZOOKEEPER_PROPERTY = "kafka.zk";
 
   public static class Topic {
     public int numPartitions;
@@ -177,6 +178,7 @@ public class KafkaWithZKComponent implements InMemoryComponent {
       payload.get(bytes);
       messages.add(bytes);
     }
+    consumer.close();
     return messages;
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/df8d682e/metron-platform/metron-integration-test/src/main/java/org/apache/metron/integration/utils/SampleUtil.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-integration-test/src/main/java/org/apache/metron/integration/utils/SampleUtil.java b/metron-platform/metron-integration-test/src/main/java/org/apache/metron/integration/utils/SampleUtil.java
index ace428d..8e293c0 100644
--- a/metron-platform/metron-integration-test/src/main/java/org/apache/metron/integration/utils/SampleUtil.java
+++ b/metron-platform/metron-integration-test/src/main/java/org/apache/metron/integration/utils/SampleUtil.java
@@ -20,16 +20,33 @@ package org.apache.metron.integration.utils;
 import org.apache.metron.TestConstants;
 import org.apache.metron.common.configuration.Configurations;
 import org.apache.metron.common.configuration.ConfigurationsUtils;
+import org.apache.metron.common.configuration.EnrichmentConfigurations;
+import org.apache.metron.common.configuration.ParserConfigurations;
 
 import java.io.IOException;
 import java.util.Map;
 
 public class SampleUtil {
 
-  public static
-  Configurations getSampleConfigs() throws IOException {
+  public static Configurations getSampleConfigs() throws IOException {
     Configurations configurations = new Configurations();
     configurations.updateGlobalConfig(ConfigurationsUtils.readGlobalConfigFromFile(TestConstants.SAMPLE_CONFIG_PATH));
+    return configurations;
+  }
+
+  public static ParserConfigurations getSampleParserConfigs() throws IOException {
+    ParserConfigurations configurations = new ParserConfigurations();
+    configurations.updateGlobalConfig(ConfigurationsUtils.readGlobalConfigFromFile(TestConstants.SAMPLE_CONFIG_PATH));
+    Map<String, byte[]> sensorParserConfigs = ConfigurationsUtils.readSensorParserConfigsFromFile(TestConstants.PARSER_CONFIGS_PATH);
+    for(String sensorType: sensorParserConfigs.keySet()) {
+      configurations.updateSensorParserConfig(sensorType, sensorParserConfigs.get(sensorType));
+    }
+    return configurations;
+  }
+
+  public static EnrichmentConfigurations getSampleEnrichmentConfigs() throws IOException {
+    EnrichmentConfigurations configurations = new EnrichmentConfigurations();
+    configurations.updateGlobalConfig(ConfigurationsUtils.readGlobalConfigFromFile(TestConstants.SAMPLE_CONFIG_PATH));
     Map<String, byte[]> sensorEnrichmentConfigs = ConfigurationsUtils.readSensorEnrichmentConfigsFromFile(TestConstants.SAMPLE_CONFIG_PATH);
     for(String sensorType: sensorEnrichmentConfigs.keySet()) {
       configurations.updateSensorEnrichmentConfig(sensorType, sensorEnrichmentConfigs.get(sensorType));

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/df8d682e/metron-platform/metron-integration-test/src/main/resources/sample/config/global.json
----------------------------------------------------------------------
diff --git a/metron-platform/metron-integration-test/src/main/resources/sample/config/global.json b/metron-platform/metron-integration-test/src/main/resources/sample/config/global.json
deleted file mode 100644
index 721f70f..0000000
--- a/metron-platform/metron-integration-test/src/main/resources/sample/config/global.json
+++ /dev/null
@@ -1,10 +0,0 @@
-{
-  "es.clustername": "metron",
-  "es.ip": "localhost",
-  "es.port": 9300,
-  "es.date.format": "yyyy.MM.dd.HH",
-  "solr.zookeeper": "localhost:2181",
-  "solr.collection": "metron",
-  "solr.numShards": 1,
-  "solr.replicationFactor": 1
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/df8d682e/metron-platform/metron-integration-test/src/main/resources/sample/config/sensors/bro.json
----------------------------------------------------------------------
diff --git a/metron-platform/metron-integration-test/src/main/resources/sample/config/sensors/bro.json b/metron-platform/metron-integration-test/src/main/resources/sample/config/sensors/bro.json
deleted file mode 100644
index 422200c..0000000
--- a/metron-platform/metron-integration-test/src/main/resources/sample/config/sensors/bro.json
+++ /dev/null
@@ -1,29 +0,0 @@
-{
-  "index": "bro",
-  "batchSize": 5,
-  "enrichment" : {
-    "fieldMap": {
-      "geo": [
-        "ip_dst_addr",
-        "ip_src_addr"
-      ],
-      "host": [
-        "host"
-      ]
-    }
-  },
-  "threatIntel" : {
-    "fieldMap":
-    {
-      "hbaseThreatIntel": [
-        "ip_dst_addr",
-        "ip_src_addr"
-      ]
-    },
-    "fieldToTypeMap": {
-      "ip_dst_addr": ["malicious_ip"]
-    , "ip_src_addr": ["malicious_ip"]
-    }
-  }
-}
-

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/df8d682e/metron-platform/metron-integration-test/src/main/resources/sample/config/sensors/pcap.json
----------------------------------------------------------------------
diff --git a/metron-platform/metron-integration-test/src/main/resources/sample/config/sensors/pcap.json b/metron-platform/metron-integration-test/src/main/resources/sample/config/sensors/pcap.json
deleted file mode 100644
index b4308ca..0000000
--- a/metron-platform/metron-integration-test/src/main/resources/sample/config/sensors/pcap.json
+++ /dev/null
@@ -1,24 +0,0 @@
-{
-  "index": "pcap",
-  "batchSize": 5,
-  "enrichment": {
-    "fieldMap": {
-      "geo": [
-        "ip_src_addr",
-        "ip_dst_addr"
-      ],
-      "host": [
-        "ip_src_addr",
-        "ip_dst_addr"
-      ]
-    }
-  },
-  "threatIntel": {
-    "fieldMap": {
-      "ip": [
-        "ip_src_addr",
-        "ip_dst_addr"
-      ]
-    }
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/df8d682e/metron-platform/metron-integration-test/src/main/resources/sample/config/sensors/snort.json
----------------------------------------------------------------------
diff --git a/metron-platform/metron-integration-test/src/main/resources/sample/config/sensors/snort.json b/metron-platform/metron-integration-test/src/main/resources/sample/config/sensors/snort.json
deleted file mode 100644
index 4d0d366..0000000
--- a/metron-platform/metron-integration-test/src/main/resources/sample/config/sensors/snort.json
+++ /dev/null
@@ -1,32 +0,0 @@
-{
-  "index": "snort",
-  "batchSize": 1,
-  "enrichment": {
-    "fieldMap": {
-      "geo": [
-        "ip_dst_addr",
-        "ip_src_addr"
-      ],
-      "host": [
-        "host"
-      ]
-    }
-  },
-  "threatIntel": {
-    "fieldMap": {
-      "hbaseThreatIntel": [
-        "ip_dst_addr",
-        "ip_src_addr"
-      ]
-    },
-    "fieldToTypeMap": {
-      "ip_dst_addr": [
-        "malicious_ip"
-      ],
-      "ip_src_addr": [
-        "malicious_ip"
-      ]
-    }
-  }
-}
-

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/df8d682e/metron-platform/metron-integration-test/src/main/resources/sample/config/sensors/websphere.json
----------------------------------------------------------------------
diff --git a/metron-platform/metron-integration-test/src/main/resources/sample/config/sensors/websphere.json b/metron-platform/metron-integration-test/src/main/resources/sample/config/sensors/websphere.json
deleted file mode 100644
index b765808..0000000
--- a/metron-platform/metron-integration-test/src/main/resources/sample/config/sensors/websphere.json
+++ /dev/null
@@ -1,20 +0,0 @@
-{
-  "index": "websphere",
-  "batchSize": 5,
-  "enrichment": {
-    "fieldMap": {
-      "geo": [
-        "ip_src_addr"
-      ],
-      "host": [
-        "ip_src_addr"
-      ]
-    },
-  "fieldToTypeMap": {
-      "ip_src_addr": [
-        "playful_classification"
-      ]
-    }
-  }
-}
-

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/df8d682e/metron-platform/metron-integration-test/src/main/resources/sample/config/sensors/yaf.json
----------------------------------------------------------------------
diff --git a/metron-platform/metron-integration-test/src/main/resources/sample/config/sensors/yaf.json b/metron-platform/metron-integration-test/src/main/resources/sample/config/sensors/yaf.json
deleted file mode 100644
index d72851e..0000000
--- a/metron-platform/metron-integration-test/src/main/resources/sample/config/sensors/yaf.json
+++ /dev/null
@@ -1,51 +0,0 @@
-{
-  "index": "yaf",
-  "batchSize": 5,
-  "enrichment": {
-    "fieldMap": {
-      "geo": [
-        "ip_src_addr",
-        "ip_dst_addr"
-      ],
-      "host": [
-        "ip_src_addr",
-        "ip_dst_addr"
-      ],
-      "hbaseEnrichment": [
-        "ip_src_addr",
-        "ip_dst_addr"
-      ]
-    }
-  ,"fieldToTypeMap": {
-      "ip_src_addr": [
-        "playful_classification"
-      ],
-      "ip_dst_addr": [
-        "playful_classification"
-      ]
-    }
-  },
-  "threatIntel": {
-    "fieldMap": {
-      "hbaseThreatIntel": [
-        "ip_src_addr",
-        "ip_dst_addr"
-      ]
-    },
-    "fieldToTypeMap": {
-      "ip_src_addr": [
-        "malicious_ip"
-      ],
-      "ip_dst_addr": [
-        "malicious_ip"
-      ]
-    },
-    "triageConfig" : {
-      "riskLevelRules" : {
-        "ip_src_addr == '10.0.2.3' or ip_dst_addr == '10.0.2.3'" : 10
-      },
-      "aggregator" : "MAX"
-    }
-  }
-}
-


[08/15] incubator-metron git commit: METRON-142 Simplify Parser configuration (merrimanr via cestella) closes apache/incubator-metron#120

Posted by ce...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/df8d682e/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
deleted file mode 100644
index 201c972..0000000
--- a/metron-platform/metron-integration-test/src/main/resources/sample/data/SampleParsed/BluecoatParsed
+++ /dev/null
@@ -1,144 +0,0 @@
-{"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/df8d682e/metron-platform/metron-integration-test/src/main/resources/sample/data/SampleParsed/SnortParsed
----------------------------------------------------------------------
diff --git a/metron-platform/metron-integration-test/src/main/resources/sample/data/SampleParsed/SnortParsed b/metron-platform/metron-integration-test/src/main/resources/sample/data/SampleParsed/SnortParsed
deleted file mode 100644
index 318b158..0000000
--- a/metron-platform/metron-integration-test/src/main/resources/sample/data/SampleParsed/SnortParsed
+++ /dev/null
@@ -1,3 +0,0 @@
-{"msg":"\"Consecutive TCP small segments exceeding threshold\"","sig_rev":"1","ip_dst_addr":"10.0.2.15","ip_dst_port":"22","ethsrc":"52:54:00:12:35:02","tcpseq":"0x9AFF3D7","dgmlen":"64","icmpid":"","tcplen":"","tcpwindow":"0xFFFF","icmpseq":"","tcpack":"0xC8761D52","original_string":"01\/27-16:01:04.877970 ,129,12,1,\"Consecutive TCP small segments exceeding threshold\",TCP,10.0.2.2,56642,10.0.2.15,22,52:54:00:12:35:02,08:00:27:7F:93:2D,0x4E,***AP***,0x9AFF3D7,0xC8761D52,,0xFFFF,64,0,59677,64,65536,,,,","icmpcode":"","tos":"0","id":"59677","timestamp":1453932941970,"ethdst":"08:00:27:7F:93:2D","ip_src_addr":"10.0.2.2","ttl":"64","source.type":"snort","ethlen":"0x4E","iplen":"65536","icmptype":"","protocol":"TCP","ip_src_port":"56642","tcpflags":"***AP***","sig_id":"12","sig_generator":"129", "is_alert" : "true"}
-{"msg":"\"Consecutive TCP small segments exceeding threshold\"","sig_rev":"1","ip_dst_addr":"10.0.2.15","ip_dst_port":"50895","ethsrc":"52:54:00:12:35:02","tcpseq":"0xDB45F7A","dgmlen":"96","icmpid":"","tcplen":"","tcpwindow":"0xFFFF","icmpseq":"","tcpack":"0x7701DD5B","original_string":"02\/22-15:56:48.612494 ,129,12,1,\"Consecutive TCP small segments exceeding threshold\",TCP,96.44.142.5,80,10.0.2.15,50895,52:54:00:12:35:02,08:00:27:7F:93:2D,0x6E,***AP***,0xDB45F7A,0x7701DD5B,,0xFFFF,64,0,16785,96,98304,,,,","icmpcode":"","tos":"0","id":"16785","timestamp":1456178820494,"ethdst":"08:00:27:7F:93:2D","ip_src_addr":"96.44.142.5","ttl":"64","source.type":"snort","ethlen":"0x6E","iplen":"98304","icmptype":"","protocol":"TCP","ip_src_port":"80","tcpflags":"***AP***","sig_id":"12","sig_generator":"129", "is_alert" : "true"}
-{"msg":"\"Consecutive TCP small segments exceeding threshold\"","sig_rev":"1","ip_dst_addr":"10.0.2.15","ip_dst_port":"50895","ethsrc":"52:54:00:12:35:02","tcpseq":"0xDB508F2","dgmlen":"152","icmpid":"","tcplen":"","tcpwindow":"0xFFFF","icmpseq":"","tcpack":"0x7701DD5B","original_string":"02\/22-15:56:48.616775 ,129,12,1,\"Consecutive TCP small segments exceeding threshold\",TCP,96.44.142.5,80,10.0.2.15,50895,52:54:00:12:35:02,08:00:27:7F:93:2D,0xA6,***AP***,0xDB508F2,0x7701DD5B,,0xFFFF,64,0,16824,152,155648,,,,","icmpcode":"","tos":"0","id":"16824","timestamp":1456178824775,"ethdst":"08:00:27:7F:93:2D","ip_src_addr":"96.44.142.5","ttl":"64","source.type":"snort","ethlen":"0xA6","iplen":"155648","icmptype":"","protocol":"TCP","ip_src_port":"80","tcpflags":"***AP***","sig_id":"12","sig_generator":"129", "is_alert" : "true"}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/df8d682e/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
deleted file mode 100644
index 9643c25..0000000
--- a/metron-platform/metron-integration-test/src/main/resources/sample/data/SampleParsed/SquidExampleParsed
+++ /dev/null
@@ -1,2 +0,0 @@
-{"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/df8d682e/metron-platform/metron-integration-test/src/main/resources/sample/data/SampleParsed/WebsphereParsed
----------------------------------------------------------------------
diff --git a/metron-platform/metron-integration-test/src/main/resources/sample/data/SampleParsed/WebsphereParsed b/metron-platform/metron-integration-test/src/main/resources/sample/data/SampleParsed/WebsphereParsed
deleted file mode 100644
index 86e3895..0000000
--- a/metron-platform/metron-integration-test/src/main/resources/sample/data/SampleParsed/WebsphereParsed
+++ /dev/null
@@ -1,5 +0,0 @@
-{"severity":"notice","hostname":"ABCXML1413","event_type":"auth","original_string":"<133>Apr 15 17:47:28 ABCXML1413 [rojOut][0x81000033][auth][notice] user(rick007): [120.43.200.6]: User logged into 'cohlOut'.","event_code":"0x81000033","security_domain":"rojOut","event_subtype":"login","priority":133,"ip_src_addr":"120.43.200.6","timestamp":1460742448000,"username":"rick007","source.type":"websphere"}
-{"severity":"info","hostname":"PHIXML3RWD","event_type":"auth","original_string":"<134>Apr 15 18:02:27 PHIXML3RWD [0x81000019][auth][info] [14.122.2.201]: User 'hjpotter' logged out from 'default'.","event_code":"0x81000019","security_domain":"default","event_subtype":"logout","priority":134,"ip_src_addr":"14.122.2.201","timestamp":1460743347000,"username":"hjpotter","source.type":"websphere"}
-{"severity":"error","hostname":"ROBXML3QRS","process":"rbm","event_type":"auth","original_string":"<131>Apr 15 17:36:35 ROBXML3QRS [0x80800018][auth][error] rbm(RBM-Settings): trans(3502888135)[request] gtid(3502888135): RBM: Resource access denied.","event_code":"0x80800018","message":"trans(3502888135)[request] gtid(3502888135): RBM: Resource access denied.","priority":131,"timestamp":1460741795000,"source.type":"websphere"}
-{"severity":"info","hostname":"SAGPXMLQA333","process":"trans","event_type":"audit","original_string":"<134>Apr 15 17:17:34 SAGPXMLQA333 [0x8240001c][audit][info] trans(191): (admin:default:system:*): ntp-service 'NTP Service' - Operational state down","event_code":"0x8240001c","message":"(admin:default:system:*): ntp-service 'NTP Service' - Operational state down","priority":134,"timestamp":1460740654000,"source.type":"websphere"}
-{"severity":"info","hostname":"DOMXML3PUZ","event_type":"auth","original_string":"<134>Apr 15 17:46:52 DOMXML3PUZ [0x8100448e][auth][info] CLI timeout occurred.","event_code":"0x8100448e","message":"CLI timeout occurred.","priority":134,"timestamp":1460742412000,"source.type":"websphere"}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/df8d682e/metron-platform/metron-integration-test/src/main/resources/sample/data/SampleParsed/YafExampleParsed
----------------------------------------------------------------------
diff --git a/metron-platform/metron-integration-test/src/main/resources/sample/data/SampleParsed/YafExampleParsed b/metron-platform/metron-integration-test/src/main/resources/sample/data/SampleParsed/YafExampleParsed
deleted file mode 100644
index 6155e98..0000000
--- a/metron-platform/metron-integration-test/src/main/resources/sample/data/SampleParsed/YafExampleParsed
+++ /dev/null
@@ -1,10 +0,0 @@
-{"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,"source.type":"yaf","start_time":1453994988512,"riflags":0,"rtt":"0.000","protocol":6}
-{"iflags":"A","uflags":0,"isn":10000000,"ip_dst_addr":"10.0.2.3","ip_dst_port":53,"duration":"0.000","rpkt":0,"original_string":"2016-01-28 15:29:48.502|2016-01-28 15:29:48.502|   0.000|   0.000| 17|                               10.0.2.15|37299|                                10.0.2.3|   53|       A|       0|       0|       0|10000000|00000000|000|000|       1|      56|       0|       0|    0|idle","pkt":1,"ruflags":0,"roct":0,"ip_src_addr":"10.0.2.15","tag":0,"rtag":0,"ip_src_port":37299,"timestamp":1453994988502,"app":0,"oct":56,"end_reason":"idle","risn":0,"end_time":1453994988502,"source.type":"yaf","start_time":1453994988502,"riflags":0,"rtt":"0.000","protocol":17}
-{"iflags":"A","uflags":0,"isn":0,"ip_dst_addr":"10.0.2.15","ip_dst_port":37299,"duration":"0.000","rpkt":0,"original_string":"2016-01-28 15:29:48.504|2016-01-28 15:29:48.504|   0.000|   0.000| 17|                                10.0.2.3|   53|                               10.0.2.15|37299|       A|       0|       0|       0|00000000|00000000|000|000|       1|     312|       0|       0|    0|idle","pkt":1,"ruflags":0,"roct":0,"ip_src_addr":"10.0.2.3","tag":0,"rtag":0,"ip_src_port":53,"timestamp":1453994988504,"app":0,"oct":312,"end_reason":"idle","risn":0,"end_time":1453994988504,"source.type":"yaf","start_time":1453994988504,"riflags":0,"rtt":"0.000","protocol":17}
-{"iflags":"A","uflags":0,"isn":0,"ip_dst_addr":"10.0.2.3","ip_dst_port":53,"duration":"0.000","rpkt":0,"original_string":"2016-01-28 15:29:48.504|2016-01-28 15:29:48.504|   0.000|   0.000| 17|                               10.0.2.15|56303|                                10.0.2.3|   53|       A|       0|       0|       0|00000000|00000000|000|000|       1|      56|       0|       0|    0|idle","pkt":1,"ruflags":0,"roct":0,"ip_src_addr":"10.0.2.15","tag":0,"rtag":0,"ip_src_port":56303,"timestamp":1453994988504,"app":0,"oct":56,"end_reason":"idle","risn":0,"end_time":1453994988504,"source.type":"yaf","start_time":1453994988504,"riflags":0,"rtt":"0.000","protocol":17}
-{"iflags":"A","uflags":0,"isn":0,"ip_dst_addr":"10.0.2.15","ip_dst_port":56303,"duration":"0.000","rpkt":0,"original_string":"2016-01-28 15:29:48.506|2016-01-28 15:29:48.506|   0.000|   0.000| 17|                                10.0.2.3|   53|                               10.0.2.15|56303|       A|       0|       0|       0|00000000|00000000|000|000|       1|      84|       0|       0|    0|idle","pkt":1,"ruflags":0,"roct":0,"ip_src_addr":"10.0.2.3","tag":0,"rtag":0,"ip_src_port":53,"timestamp":1453994988506,"app":0,"oct":84,"end_reason":"idle","risn":0,"end_time":1453994988506,"source.type":"yaf","start_time":1453994988506,"riflags":0,"rtt":"0.000","protocol":17}
-{"iflags":"S","uflags":0,"isn":"58c52fca","ip_dst_addr":"216.21.170.221","ip_dst_port":80,"duration":"0.000","rpkt":0,"original_string":"2016-01-28 15:29:48.508|2016-01-28 15:29:48.508|   0.000|   0.000|  6|                               10.0.2.15|39468|                          216.21.170.221|   80|       S|       0|       0|       0|58c52fca|00000000|000|000|       1|      60|       0|       0|    0|idle","pkt":1,"ruflags":0,"roct":0,"ip_src_addr":"10.0.2.15","tag":0,"rtag":0,"ip_src_port":39468,"timestamp":1453994988508,"app":0,"oct":60,"end_reason":"idle","risn":0,"end_time":1453994988508,"source.type":"yaf","start_time":1453994988508,"riflags":0,"rtt":"0.000","protocol":6}
-{"iflags":"A","uflags":0,"isn":"58c52fcb","ip_dst_addr":"216.21.170.221","ip_dst_port":80,"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|                               10.0.2.15|39468|                          216.21.170.221|   80|       A|       0|       0|       0|58c52fcb|00000000|000|000|       1|      40|       0|       0|    0|idle ","pkt":1,"ruflags":0,"roct":0,"ip_src_addr":"10.0.2.15","tag":0,"rtag":0,"ip_src_port":39468,"timestamp":1453994988512,"app":0,"oct":40,"end_reason":"idle ","risn":0,"end_time":1453994988512,"source.type":"yaf","start_time":1453994988512,"riflags":0,"rtt":"0.000","protocol":6}
-{"iflags":"AP","uflags":0,"isn":"58c52fcb","ip_dst_addr":"216.21.170.221","ip_dst_port":80,"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|                               10.0.2.15|39468|                          216.21.170.221|   80|      AP|       0|       0|       0|58c52fcb|00000000|000|000|       1|     148|       0|       0|    0|idle ","pkt":1,"ruflags":0,"roct":0,"ip_src_addr":"10.0.2.15","tag":0,"rtag":0,"ip_src_port":39468,"timestamp":1453994988512,"app":0,"oct":148,"end_reason":"idle ","risn":0,"end_time":1453994988512,"source.type":"yaf","start_time":1453994988512,"riflags":0,"rtt":"0.000","protocol":6}
-{"iflags":"A","uflags":0,"isn":"22efa002","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|       A|       0|       0|       0|22efa002|00000000|000|000|       1|      40|       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":40,"end_reason":"idle ","risn":0,"end_time":1453994988512,"source.type":"yaf","start_time":1453994988512,"riflags":0,"rtt":"0.000","protocol":6}
-{"iflags":"AP","uflags":0,"isn":"22efa002","ip_dst_addr":"10.0.2.15","ip_dst_port":39468,"duration":"0.000","rpkt":0,"original_string":"2016-01-28 15:29:48.562|2016-01-28 15:29:48.562|   0.000|   0.000|  6|                          216.21.170.221|   80|                               10.0.2.15|39468|      AP|       0|       0|       0|22efa002|00000000|000|000|       1|     604|       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":1453994988562,"app":0,"oct":604,"end_reason":"idle","risn":0,"end_time":1453994988562,"source.type":"yaf","start_time":1453994988562,"riflags":0,"rtt":"0.000","protocol":6}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/df8d682e/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
deleted file mode 100644
index a88a255..0000000
--- a/metron-platform/metron-integration-test/src/main/resources/sample/patterns/test
+++ /dev/null
@@ -1,2 +0,0 @@
-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/df8d682e/metron-platform/metron-integration-test/src/main/sample/data/SampleInput/.PCAPExampleOutput.crc
----------------------------------------------------------------------
diff --git a/metron-platform/metron-integration-test/src/main/sample/data/SampleInput/.PCAPExampleOutput.crc b/metron-platform/metron-integration-test/src/main/sample/data/SampleInput/.PCAPExampleOutput.crc
new file mode 100644
index 0000000..6e53497
Binary files /dev/null and b/metron-platform/metron-integration-test/src/main/sample/data/SampleInput/.PCAPExampleOutput.crc differ



[13/15] incubator-metron git commit: METRON-142 Simplify Parser configuration (merrimanr via cestella) closes apache/incubator-metron#120

Posted by ce...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/df8d682e/metron-platform/metron-integration-test/src/main/resources/sample/data/SampleIndexed/YafIndexed
----------------------------------------------------------------------
diff --git a/metron-platform/metron-integration-test/src/main/resources/sample/data/SampleIndexed/YafIndexed b/metron-platform/metron-integration-test/src/main/resources/sample/data/SampleIndexed/YafIndexed
deleted file mode 100644
index 1c38406..0000000
--- a/metron-platform/metron-integration-test/src/main/resources/sample/data/SampleIndexed/YafIndexed
+++ /dev/null
@@ -1,10 +0,0 @@
-{"adapter.threatinteladapter.end.ts":"1457102731219","enrichments.geo.dip.location_point":"test longitude,test latitude","isn":"22efa001","index.elasticsearchwriter.ts":"1457102731220","dip":"10.0.2.15","dp":39468,"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","enrichments.geo.dip.locID":"1","enrichments.geo.sip.city":"test city","enrichmentjoinbolt.joiner.ts":"1457102731206","adapter.hostfromjsonlistadapter.begin.ts":"1457102731185","tag":0,"enrichments.geo.dip.dmaCode":"test dmaCode","app":0,"oct":44,"end_reason":"idle","enrichments.geo.sip.locID":"1","adapter.mockgeoadapter.begin.ts":"1457102731185","threatintelsplitterbolt.splitter.ts":"1457102731207","enrichments.geo.dip.postalCode":"test postalCode","start_time":1453994988512,
 "adapter.threatinteladapter.begin.ts":"1457102731210","riflags":0,"proto":6,"enrichments.host.dip.known_info.local":"YES","enrichments.geo.dip.longitude":"test longitude","iflags":"AS","uflags":0,"adapter.mockgeoadapter.end.ts":"1457102731198","adapter.hostfromjsonlistadapter.end.ts":"1457102731197","enrichments.geo.sip.postalCode":"test postalCode","duration":"0.000","enrichments.geo.dip.country":"test country","threatinteljoinbolt.joiner.ts":"1457102731220","enrichments.geo.dip.latitude":"test latitude","enrichments.geo.sip.country":"test country","enrichments.geo.dip.city":"test city","enrichments.geo.sip.dmaCode":"test dmaCode","pkt":1,"enrichments.geo.sip.location_point":"test longitude,test latitude","ruflags":0,"roct":0,"sip":"216.21.170.221","rtag":0,"sp":80,"enrichments.geo.sip.longitude":"test longitude","enrichments.geo.sip.latitude":"test latitude","timestamp":1453994988512,"risn":0,"enrichments.host.dip.known_info.type":"printer","end_time":1453994988512,"enrichments.ho
 st.dip.known_info.asset_value":"important","source.type":"yaf","rtt":"0.000"}
-{"adapter.threatinteladapter.end.ts":"1457102731221","enrichments.geo.dip.location_point":"test longitude,test latitude","enrichments.host.sip.known_info.asset_value":"important","isn":10000000,"index.elasticsearchwriter.ts":"1457102731221","dip":"10.0.2.3","dp":53,"rpkt":0,"original_string":"2016-01-28 15:29:48.502|2016-01-28 15:29:48.502|   0.000|   0.000| 17|                               10.0.2.15|37299|                                10.0.2.3|   53|       A|       0|       0|       0|10000000|00000000|000|000|       1|      56|       0|       0|    0|idle","enrichments.geo.dip.locID":"1","enrichments.geo.sip.city":"test city","enrichments.host.sip.known_info.type":"printer","enrichmentjoinbolt.joiner.ts":"1457102731208","adapter.hostfromjsonlistadapter.begin.ts":"1457102731197","tag":0,"enrichments.geo.dip.dmaCode":"test dmaCode","app":0,"oct":56,"end_reason":"idle","enrichments.geo.sip.locID":"1","adapter.mockgeoadapter.begin.ts":"1457102731198","threatintelsplitterbolt.splitt
 er.ts":"1457102731210","enrichments.geo.dip.postalCode":"test postalCode","start_time":1453994988502,"adapter.threatinteladapter.begin.ts":"1457102731219","riflags":0,"proto":17,"enrichments.geo.dip.longitude":"test longitude","iflags":"A","uflags":0,"adapter.mockgeoadapter.end.ts":"1457102731198","adapter.hostfromjsonlistadapter.end.ts":"1457102731197","enrichments.host.sip.known_info.local":"YES","threatintels.ip.dip.ip_threat_intel":"alert","enrichments.geo.sip.postalCode":"test postalCode","duration":"0.000","enrichments.geo.dip.country":"test country","threatinteljoinbolt.joiner.ts":"1457102731221","enrichments.geo.dip.latitude":"test latitude","enrichments.geo.sip.country":"test country","enrichments.geo.dip.city":"test city","enrichments.geo.sip.dmaCode":"test dmaCode","pkt":1,"enrichments.geo.sip.location_point":"test longitude,test latitude","ruflags":0,"roct":0,"sip":"10.0.2.15","rtag":0,"sp":37299,"enrichments.geo.sip.longitude":"test longitude","enrichments.geo.sip.latit
 ude":"test latitude","timestamp":1453994988502,"risn":0,"end_time":1453994988502,"is_alert":"true","source.type":"yaf","rtt":"0.000"}
-{"adapter.threatinteladapter.end.ts":"1457102731221","enrichments.geo.dip.location_point":"test longitude,test latitude","isn":0,"index.elasticsearchwriter.ts":"1457102731222","dip":"10.0.2.15","dp":37299,"rpkt":0,"original_string":"2016-01-28 15:29:48.504|2016-01-28 15:29:48.504|   0.000|   0.000| 17|                                10.0.2.3|   53|                               10.0.2.15|37299|       A|       0|       0|       0|00000000|00000000|000|000|       1|     312|       0|       0|    0|idle","enrichments.geo.dip.locID":"1","enrichments.geo.sip.city":"test city","enrichmentjoinbolt.joiner.ts":"1457102731209","adapter.hostfromjsonlistadapter.begin.ts":"1457102731197","tag":0,"enrichments.geo.dip.dmaCode":"test dmaCode","app":0,"oct":312,"end_reason":"idle","enrichments.geo.sip.locID":"1","adapter.mockgeoadapter.begin.ts":"1457102731198","threatintelsplitterbolt.splitter.ts":"1457102731210","enrichments.geo.dip.postalCode":"test postalCode","start_time":1453994988504,"adapter
 .threatinteladapter.begin.ts":"1457102731221","riflags":0,"proto":17,"enrichments.host.dip.known_info.local":"YES","enrichments.geo.dip.longitude":"test longitude","iflags":"A","uflags":0,"adapter.mockgeoadapter.end.ts":"1457102731199","adapter.hostfromjsonlistadapter.end.ts":"1457102731198","enrichments.geo.sip.postalCode":"test postalCode","duration":"0.000","enrichments.geo.dip.country":"test country","threatinteljoinbolt.joiner.ts":"1457102731222","enrichments.geo.dip.latitude":"test latitude","enrichments.geo.sip.country":"test country","enrichments.geo.dip.city":"test city","enrichments.geo.sip.dmaCode":"test dmaCode","pkt":1,"enrichments.geo.sip.location_point":"test longitude,test latitude","ruflags":0,"roct":0,"sip":"10.0.2.3","rtag":0,"sp":53,"enrichments.geo.sip.longitude":"test longitude","enrichments.geo.sip.latitude":"test latitude","timestamp":1453994988504,"risn":0,"enrichments.host.dip.known_info.type":"printer","end_time":1453994988504,"enrichments.host.dip.known_i
 nfo.asset_value":"important","is_alert":"true","source.type":"yaf","threatintels.ip.sip.ip_threat_intel":"alert","rtt":"0.000"}
-{"adapter.threatinteladapter.end.ts":"1457102731222","enrichments.geo.dip.location_point":"test longitude,test latitude","enrichments.host.sip.known_info.asset_value":"important","isn":0,"index.elasticsearchwriter.ts":"1457102731222","dip":"10.0.2.3","dp":53,"rpkt":0,"original_string":"2016-01-28 15:29:48.504|2016-01-28 15:29:48.504|   0.000|   0.000| 17|                               10.0.2.15|56303|                                10.0.2.3|   53|       A|       0|       0|       0|00000000|00000000|000|000|       1|      56|       0|       0|    0|idle","enrichments.geo.dip.locID":"1","enrichments.geo.sip.city":"test city","enrichments.host.sip.known_info.type":"printer","enrichmentjoinbolt.joiner.ts":"1457102731209","adapter.hostfromjsonlistadapter.begin.ts":"1457102731198","tag":0,"enrichments.geo.dip.dmaCode":"test dmaCode","app":0,"oct":56,"end_reason":"idle","enrichments.geo.sip.locID":"1","adapter.mockgeoadapter.begin.ts":"1457102731199","threatintelsplitterbolt.splitter.ts":
 "1457102731211","enrichments.geo.dip.postalCode":"test postalCode","start_time":1453994988504,"adapter.threatinteladapter.begin.ts":"1457102731221","riflags":0,"proto":17,"enrichments.geo.dip.longitude":"test longitude","iflags":"A","uflags":0,"adapter.mockgeoadapter.end.ts":"1457102731199","adapter.hostfromjsonlistadapter.end.ts":"1457102731198","enrichments.host.sip.known_info.local":"YES","threatintels.ip.dip.ip_threat_intel":"alert","enrichments.geo.sip.postalCode":"test postalCode","duration":"0.000","enrichments.geo.dip.country":"test country","threatinteljoinbolt.joiner.ts":"1457102731222","enrichments.geo.dip.latitude":"test latitude","enrichments.geo.sip.country":"test country","enrichments.geo.dip.city":"test city","enrichments.geo.sip.dmaCode":"test dmaCode","pkt":1,"enrichments.geo.sip.location_point":"test longitude,test latitude","ruflags":0,"roct":0,"sip":"10.0.2.15","rtag":0,"sp":56303,"enrichments.geo.sip.longitude":"test longitude","enrichments.geo.sip.latitude":"t
 est latitude","timestamp":1453994988504,"risn":0,"end_time":1453994988504,"is_alert":"true","source.type":"yaf","rtt":"0.000"}
-{"adapter.threatinteladapter.end.ts":"1457102731222","enrichments.geo.dip.location_point":"test longitude,test latitude","isn":0,"index.elasticsearchwriter.ts":"1457102731222","dip":"10.0.2.15","dp":56303,"rpkt":0,"original_string":"2016-01-28 15:29:48.506|2016-01-28 15:29:48.506|   0.000|   0.000| 17|                                10.0.2.3|   53|                               10.0.2.15|56303|       A|       0|       0|       0|00000000|00000000|000|000|       1|      84|       0|       0|    0|idle","enrichments.geo.dip.locID":"1","enrichments.geo.sip.city":"test city","enrichmentjoinbolt.joiner.ts":"1457102731210","adapter.hostfromjsonlistadapter.begin.ts":"1457102731198","tag":0,"enrichments.geo.dip.dmaCode":"test dmaCode","app":0,"oct":84,"end_reason":"idle","enrichments.geo.sip.locID":"1","adapter.mockgeoadapter.begin.ts":"1457102731199","threatintelsplitterbolt.splitter.ts":"1457102731212","enrichments.geo.dip.postalCode":"test postalCode","start_time":1453994988506,"adapter.
 threatinteladapter.begin.ts":"1457102731222","riflags":0,"proto":17,"enrichments.host.dip.known_info.local":"YES","enrichments.geo.dip.longitude":"test longitude","iflags":"A","uflags":0,"adapter.mockgeoadapter.end.ts":"1457102731199","adapter.hostfromjsonlistadapter.end.ts":"1457102731198","enrichments.geo.sip.postalCode":"test postalCode","duration":"0.000","enrichments.geo.dip.country":"test country","threatinteljoinbolt.joiner.ts":"1457102731222","enrichments.geo.dip.latitude":"test latitude","enrichments.geo.sip.country":"test country","enrichments.geo.dip.city":"test city","enrichments.geo.sip.dmaCode":"test dmaCode","pkt":1,"enrichments.geo.sip.location_point":"test longitude,test latitude","ruflags":0,"roct":0,"sip":"10.0.2.3","rtag":0,"sp":53,"enrichments.geo.sip.longitude":"test longitude","enrichments.geo.sip.latitude":"test latitude","timestamp":1453994988506,"risn":0,"enrichments.host.dip.known_info.type":"printer","end_time":1453994988506,"enrichments.host.dip.known_in
 fo.asset_value":"important","is_alert":"true","source.type":"yaf","threatintels.ip.sip.ip_threat_intel":"alert","rtt":"0.000"}
-{"adapter.threatinteladapter.end.ts":"1457102731222","enrichments.geo.dip.location_point":"test longitude,test latitude","enrichments.host.sip.known_info.asset_value":"important","isn":"58c52fca","index.elasticsearchwriter.ts":"1457102732038","dip":"216.21.170.221","dp":80,"rpkt":0,"original_string":"2016-01-28 15:29:48.508|2016-01-28 15:29:48.508|   0.000|   0.000|  6|                               10.0.2.15|39468|                          216.21.170.221|   80|       S|       0|       0|       0|58c52fca|00000000|000|000|       1|      60|       0|       0|    0|idle","enrichments.geo.dip.locID":"1","enrichments.geo.sip.city":"test city","enrichments.host.sip.known_info.type":"printer","enrichmentjoinbolt.joiner.ts":"1457102731210","adapter.hostfromjsonlistadapter.begin.ts":"1457102731198","tag":0,"enrichments.geo.dip.dmaCode":"test dmaCode","app":0,"oct":60,"end_reason":"idle","enrichments.geo.sip.locID":"1","adapter.mockgeoadapter.begin.ts":"1457102731199","threatintelsplitterbol
 t.splitter.ts":"1457102731212","enrichments.geo.dip.postalCode":"test postalCode","start_time":1453994988508,"adapter.threatinteladapter.begin.ts":"1457102731222","riflags":0,"proto":6,"enrichments.geo.dip.longitude":"test longitude","iflags":"S","uflags":0,"adapter.mockgeoadapter.end.ts":"1457102731199","adapter.hostfromjsonlistadapter.end.ts":"1457102731198","enrichments.host.sip.known_info.local":"YES","enrichments.geo.sip.postalCode":"test postalCode","duration":"0.000","enrichments.geo.dip.country":"test country","threatinteljoinbolt.joiner.ts":"1457102731223","enrichments.geo.dip.latitude":"test latitude","enrichments.geo.sip.country":"test country","enrichments.geo.dip.city":"test city","enrichments.geo.sip.dmaCode":"test dmaCode","pkt":1,"enrichments.geo.sip.location_point":"test longitude,test latitude","ruflags":0,"roct":0,"sip":"10.0.2.15","rtag":0,"sp":39468,"enrichments.geo.sip.longitude":"test longitude","enrichments.geo.sip.latitude":"test latitude","timestamp":145399
 4988508,"risn":0,"end_time":1453994988508,"source.type":"yaf","rtt":"0.000"}
-{"adapter.threatinteladapter.end.ts":"1457102731223","enrichments.geo.dip.location_point":"test longitude,test latitude","enrichments.host.sip.known_info.asset_value":"important","isn":"58c52fcb","index.elasticsearchwriter.ts":"1457102732038","dip":"216.21.170.221","dp":80,"rpkt":0,"original_string":"2016-01-28 15:29:48.512|2016-01-28 15:29:48.512|   0.000|   0.000|  6|                               10.0.2.15|39468|                          216.21.170.221|   80|       A|       0|       0|       0|58c52fcb|00000000|000|000|       1|      40|       0|       0|    0|idle ","enrichments.geo.dip.locID":"1","enrichments.geo.sip.city":"test city","enrichments.host.sip.known_info.type":"printer","enrichmentjoinbolt.joiner.ts":"1457102731210","adapter.hostfromjsonlistadapter.begin.ts":"1457102731198","tag":0,"enrichments.geo.dip.dmaCode":"test dmaCode","app":0,"oct":40,"end_reason":"idle ","enrichments.geo.sip.locID":"1","adapter.mockgeoadapter.begin.ts":"1457102731199","threatintelsplitterb
 olt.splitter.ts":"1457102731212","enrichments.geo.dip.postalCode":"test postalCode","start_time":1453994988512,"adapter.threatinteladapter.begin.ts":"1457102731223","riflags":0,"proto":6,"enrichments.geo.dip.longitude":"test longitude","iflags":"A","uflags":0,"adapter.mockgeoadapter.end.ts":"1457102731199","adapter.hostfromjsonlistadapter.end.ts":"1457102731198","enrichments.host.sip.known_info.local":"YES","enrichments.geo.sip.postalCode":"test postalCode","duration":"0.000","enrichments.geo.dip.country":"test country","threatinteljoinbolt.joiner.ts":"1457102731223","enrichments.geo.dip.latitude":"test latitude","enrichments.geo.sip.country":"test country","enrichments.geo.dip.city":"test city","enrichments.geo.sip.dmaCode":"test dmaCode","pkt":1,"enrichments.geo.sip.location_point":"test longitude,test latitude","ruflags":0,"roct":0,"sip":"10.0.2.15","rtag":0,"sp":39468,"enrichments.geo.sip.longitude":"test longitude","enrichments.geo.sip.latitude":"test latitude","timestamp":1453
 994988512,"risn":0,"end_time":1453994988512,"source.type":"yaf","rtt":"0.000"}
-{"adapter.threatinteladapter.end.ts":"1457102731223","enrichments.geo.dip.location_point":"test longitude,test latitude","enrichments.host.sip.known_info.asset_value":"important","isn":"58c52fcb","index.elasticsearchwriter.ts":"1457102732038","dip":"216.21.170.221","dp":80,"rpkt":0,"original_string":"2016-01-28 15:29:48.512|2016-01-28 15:29:48.512|   0.000|   0.000|  6|                               10.0.2.15|39468|                          216.21.170.221|   80|      AP|       0|       0|       0|58c52fcb|00000000|000|000|       1|     148|       0|       0|    0|idle ","enrichments.geo.dip.locID":"1","enrichments.geo.sip.city":"test city","enrichments.host.sip.known_info.type":"printer","enrichmentjoinbolt.joiner.ts":"1457102731210","adapter.hostfromjsonlistadapter.begin.ts":"1457102731198","tag":0,"enrichments.geo.dip.dmaCode":"test dmaCode","app":0,"oct":148,"end_reason":"idle ","enrichments.geo.sip.locID":"1","adapter.mockgeoadapter.begin.ts":"1457102731199","threatintelsplitter
 bolt.splitter.ts":"1457102731212","enrichments.geo.dip.postalCode":"test postalCode","start_time":1453994988512,"adapter.threatinteladapter.begin.ts":"1457102731223","riflags":0,"proto":6,"enrichments.geo.dip.longitude":"test longitude","iflags":"AP","uflags":0,"adapter.mockgeoadapter.end.ts":"1457102731199","adapter.hostfromjsonlistadapter.end.ts":"1457102731198","enrichments.host.sip.known_info.local":"YES","enrichments.geo.sip.postalCode":"test postalCode","duration":"0.000","enrichments.geo.dip.country":"test country","threatinteljoinbolt.joiner.ts":"1457102731225","enrichments.geo.dip.latitude":"test latitude","enrichments.geo.sip.country":"test country","enrichments.geo.dip.city":"test city","enrichments.geo.sip.dmaCode":"test dmaCode","pkt":1,"enrichments.geo.sip.location_point":"test longitude,test latitude","ruflags":0,"roct":0,"sip":"10.0.2.15","rtag":0,"sp":39468,"enrichments.geo.sip.longitude":"test longitude","enrichments.geo.sip.latitude":"test latitude","timestamp":14
 53994988512,"risn":0,"end_time":1453994988512,"source.type":"yaf","rtt":"0.000"}
-{"adapter.threatinteladapter.end.ts":"1457102731225","enrichments.geo.dip.location_point":"test longitude,test latitude","isn":"22efa002","index.elasticsearchwriter.ts":"1457102732038","dip":"10.0.2.15","dp":39468,"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|       A|       0|       0|       0|22efa002|00000000|000|000|       1|      40|       0|       0|    0|idle ","enrichments.geo.dip.locID":"1","enrichments.geo.sip.city":"test city","enrichmentjoinbolt.joiner.ts":"1457102731211","adapter.hostfromjsonlistadapter.begin.ts":"1457102731198","tag":0,"enrichments.geo.dip.dmaCode":"test dmaCode","app":0,"oct":40,"end_reason":"idle ","enrichments.geo.sip.locID":"1","adapter.mockgeoadapter.begin.ts":"1457102731199","threatintelsplitterbolt.splitter.ts":"1457102731212","enrichments.geo.dip.postalCode":"test postalCode","start_time":145399498851
 2,"adapter.threatinteladapter.begin.ts":"1457102731223","riflags":0,"proto":6,"enrichments.host.dip.known_info.local":"YES","enrichments.geo.dip.longitude":"test longitude","iflags":"A","uflags":0,"adapter.mockgeoadapter.end.ts":"1457102731199","adapter.hostfromjsonlistadapter.end.ts":"1457102731198","enrichments.geo.sip.postalCode":"test postalCode","duration":"0.000","enrichments.geo.dip.country":"test country","threatinteljoinbolt.joiner.ts":"1457102731225","enrichments.geo.dip.latitude":"test latitude","enrichments.geo.sip.country":"test country","enrichments.geo.dip.city":"test city","enrichments.geo.sip.dmaCode":"test dmaCode","pkt":1,"enrichments.geo.sip.location_point":"test longitude,test latitude","ruflags":0,"roct":0,"sip":"216.21.170.221","rtag":0,"sp":80,"enrichments.geo.sip.longitude":"test longitude","enrichments.geo.sip.latitude":"test latitude","timestamp":1453994988512,"risn":0,"enrichments.host.dip.known_info.type":"printer","end_time":1453994988512,"enrichments.h
 ost.dip.known_info.asset_value":"important","source.type":"yaf","rtt":"0.000"}
-{"adapter.threatinteladapter.end.ts":"1457102731226","enrichments.geo.dip.location_point":"test longitude,test latitude","isn":"22efa002","index.elasticsearchwriter.ts":"1457102732038","dip":"10.0.2.15","dp":39468,"rpkt":0,"original_string":"2016-01-28 15:29:48.562|2016-01-28 15:29:48.562|   0.000|   0.000|  6|                          216.21.170.221|   80|                               10.0.2.15|39468|      AP|       0|       0|       0|22efa002|00000000|000|000|       1|     604|       0|       0|    0|idle","enrichments.geo.dip.locID":"1","enrichments.geo.sip.city":"test city","enrichmentjoinbolt.joiner.ts":"1457102731211","adapter.hostfromjsonlistadapter.begin.ts":"1457102731198","tag":0,"enrichments.geo.dip.dmaCode":"test dmaCode","app":0,"oct":604,"end_reason":"idle","enrichments.geo.sip.locID":"1","adapter.mockgeoadapter.begin.ts":"1457102731199","threatintelsplitterbolt.splitter.ts":"1457102731213","enrichments.geo.dip.postalCode":"test postalCode","start_time":1453994988562
 ,"adapter.threatinteladapter.begin.ts":"1457102731226","riflags":0,"proto":6,"enrichments.host.dip.known_info.local":"YES","enrichments.geo.dip.longitude":"test longitude","iflags":"AP","uflags":0,"adapter.mockgeoadapter.end.ts":"1457102731199","adapter.hostfromjsonlistadapter.end.ts":"1457102731198","enrichments.geo.sip.postalCode":"test postalCode","duration":"0.000","enrichments.geo.dip.country":"test country","threatinteljoinbolt.joiner.ts":"1457102731226","enrichments.geo.dip.latitude":"test latitude","enrichments.geo.sip.country":"test country","enrichments.geo.dip.city":"test city","enrichments.geo.sip.dmaCode":"test dmaCode","pkt":1,"enrichments.geo.sip.location_point":"test longitude,test latitude","ruflags":0,"roct":0,"sip":"216.21.170.221","rtag":0,"sp":80,"enrichments.geo.sip.longitude":"test longitude","enrichments.geo.sip.latitude":"test latitude","timestamp":1453994988562,"risn":0,"enrichments.host.dip.known_info.type":"printer","end_time":1453994988562,"enrichments.h
 ost.dip.known_info.asset_value":"important","source.type":"yaf","rtt":"0.000"}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/df8d682e/metron-platform/metron-integration-test/src/main/resources/sample/data/SampleInput/.PCAPExampleOutput.crc
----------------------------------------------------------------------
diff --git a/metron-platform/metron-integration-test/src/main/resources/sample/data/SampleInput/.PCAPExampleOutput.crc b/metron-platform/metron-integration-test/src/main/resources/sample/data/SampleInput/.PCAPExampleOutput.crc
deleted file mode 100644
index 6e53497..0000000
Binary files a/metron-platform/metron-integration-test/src/main/resources/sample/data/SampleInput/.PCAPExampleOutput.crc and /dev/null differ

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/df8d682e/metron-platform/metron-integration-test/src/main/resources/sample/data/SampleInput/AsaOutput
----------------------------------------------------------------------
diff --git a/metron-platform/metron-integration-test/src/main/resources/sample/data/SampleInput/AsaOutput b/metron-platform/metron-integration-test/src/main/resources/sample/data/SampleInput/AsaOutput
deleted file mode 100644
index 6009d48..0000000
--- a/metron-platform/metron-integration-test/src/main/resources/sample/data/SampleInput/AsaOutput
+++ /dev/null
@@ -1,100 +0,0 @@
-<167>Jan  5 08:52:35 10.22.8.216 %ASA-7-609001: Built local-host inside:10.22.8.205
-<166>Jan  5 08:52:35 10.22.8.216 %ASA-6-302021: Teardown ICMP connection for faddr 10.22.8.74/0(LOCAL\user.name) gaddr 10.22.8.205/0 laddr 10.22.8.205/0
-<167>Jan  5 08:52:35 10.22.8.216 %ASA-7-609002: Teardown local-host inside:10.22.8.205 duration 0:00:00
-<142>Jan  5 08:52:35 10.22.8.201 %ASA-6-302014: Teardown TCP connection 488167725 for Outside_VPN:147.111.72.16/26436 to DMZ-Inside:10.22.8.53/443 duration 0:00:00 bytes 9687 TCP FINs
-<166>Jan  5 08:52:35 10.22.8.216 %ASA-6-302014: Teardown TCP connection 212805593 for outside:10.22.8.223/59614(LOCAL\user.name) to inside:10.22.8.78/8102 duration 0:00:07 bytes 3433 TCP FINs (user.name)
-<174>Jan  5 14:52:35 10.22.8.212 %ASA-6-302013: Built inbound TCP connection 76245503 for outside:10.22.8.233/54209 (10.22.8.233/54209) to inside:198.111.72.238/443 (198.111.72.238/443) (user.name)
-<166>Jan  5 08:52:35 10.22.8.216 %ASA-6-302013: Built inbound TCP connection 212806031 for outside:10.22.8.17/58633 (10.22.8.17/58633)(LOCAL\user.name) to inside:10.22.8.12/389 (10.22.8.12/389) (user.name)
-<142>Jan  5 08:52:35 10.22.8.201 %ASA-6-302014: Teardown TCP connection 488168292 for DMZ-Inside:10.22.8.51/51231 to Inside-Trunk:10.22.8.174/40004 duration 0:00:00 bytes 2103 TCP FINs
-<142>Jan  5 08:52:35 10.22.8.201 %ASA-6-106015: Deny TCP (no connection) from 186.111.72.11/80 to 204.111.72.226/45019 flags SYN ACK  on interface Outside_VPN
-<166>Jan  5 09:52:35 10.22.8.12 %ASA-6-302014: Teardown TCP connection 17604987 for outside:209.111.72.151/443 to inside:10.22.8.188/64306 duration 0:00:31 bytes 10128 TCP FINs
-<166>Jan  5 09:52:35 10.22.8.12 %ASA-6-302014: Teardown TCP connection 17604999 for outside:209.111.72.151/443 to inside:10.22.8.188/64307 duration 0:00:30 bytes 6370 TCP FINs
-<142>Jan  5 08:52:35 10.22.8.201 %ASA-6-302014: Teardown TCP connection 488167347 for Outside_VPN:198.111.72.24/2134 to DMZ-Inside:10.22.8.53/443 duration 0:00:01 bytes 9785 TCP FINs
-<174>Jan  5 14:52:35 10.22.8.212 %ASA-6-302015: Built inbound UDP connection 76245506 for outside:10.22.8.110/49886 (10.22.8.110/49886) to inside:192.111.72.8/8612 (192.111.72.8/8612) (user.name)
-<166>Jan  5 08:52:35 10.22.8.216 %ASA-6-302014: Teardown TCP connection 212805993 for outside:10.22.8.89/56917(LOCAL\user.name) to inside:216.111.72.126/443 duration 0:00:00 bytes 0 TCP FINs (user.name)
-<167>Jan  5 08:52:35 10.22.8.216 %ASA-7-710005: UDP request discarded from 10.22.8.223/49192 to outside:224.111.72.252/5355
-<142>Jan  5 08:52:35 10.22.8.201 %ASA-6-302014: Teardown TCP connection 488166143 for Outside_VPN:198.111.72.64/80 to Inside-Trunk:10.22.8.39/54883 duration 0:00:04 bytes 1148 TCP FINs
-<166>Jan  5 08:52:35 10.22.8.216 %ASA-6-106015: Deny TCP (no connection) from 10.22.8.84/445 to 10.22.8.219/60726 flags ACK  on interface inside
-<142>Jan  5 08:52:35 10.22.8.201 %ASA-6-302014: Teardown TCP connection 488168344 for DMZ-Inside:10.22.8.53/61682 to Inside-Trunk:10.22.8.174/40004 duration 0:00:00 bytes 5648 TCP FINs
-<142>Jan  5 08:52:35 10.22.8.201 %ASA-6-302014: Teardown TCP connection 488168345 for DMZ-Inside:10.22.8.16/31454 to Inside-Trunk:10.22.8.21/443 duration 0:00:00 bytes 756 TCP FINs
-<182>Jan  5 20:22:35 10.22.8.4 %ASA-6-302020: Built inbound ICMP connection for faddr 10.22.8.12/0 gaddr 10.22.8.45/1 laddr 10.22.8.45/1
-<142>Jan  5 08:52:35 10.22.8.201 %ASA-6-106015: Deny TCP (no connection) from 50.111.72.230/80 to 204.111.72.254/53077 flags RST  on interface Outside_VPN
-<166>Jan  5 09:52:35 10.22.8.12 %ASA-6-302016: Teardown UDP connection 17603649 for outside:206.111.72.2/161 to inside:10.22.8.48/63297 duration 0:02:01 bytes 209
-<166>Jan  5 09:52:35 10.22.8.12 %ASA-6-302016: Teardown UDP connection 17603650 for outside:207.111.72.122/161 to inside:10.22.8.48/63298 duration 0:02:01 bytes 209
-<166>Jan  5 09:52:35 10.22.8.12 %ASA-6-302016: Teardown UDP connection 17603652 for outside:206.111.72.2/161 to inside:10.22.8.48/63300 duration 0:02:01 bytes 115
-<166>Jan  5 09:52:35 10.22.8.12 %ASA-6-302016: Teardown UDP connection 17603657 for outside:206.111.72.2/161 to inside:10.22.8.48/63306 duration 0:02:01 bytes 115
-<142>Jan  5 08:52:35 10.22.8.201 %ASA-6-302014: Teardown TCP connection 488168436 for DMZ-Inside:10.22.8.51/51235 to Inside-Trunk:10.22.8.174/40004 duration 0:00:00 bytes 2497 TCP FINs
-<142>Jan  5 08:52:35 10.22.8.201 %ASA-6-302014: Teardown TCP connection 488167656 for Outside_VPN:69.111.72.70/21560 to DMZ-Inside:10.22.8.53/443 duration 0:00:01 bytes 11410 TCP FINs
-<166>Jan  5 08:52:35 10.22.8.216 %ASA-6-302015: Built inbound UDP connection 212806050 for outside:10.22.8.62/53965 (10.22.8.62/53965)(LOCAL\user.name) to inside:10.22.8.85/53 (10.22.8.85/53) (user.name)
-<166>Jan  5 08:52:35 10.22.8.216 %ASA-6-302013: Built inbound TCP connection 212806052 for outside:10.22.8.62/56500 (10.22.8.62/56500)(LOCAL\user.name) to inside:198.111.72.83/443 (198.111.72.83/443) (user.name)
-<166>Jan  5 08:52:35 10.22.8.216 %ASA-6-302013: Built inbound TCP connection 212806054 for outside:10.22.8.62/56502 (10.22.8.62/56502)(LOCAL\user.name) to inside:50.111.72.252/443 (50.111.72.252/443) (user.name)
-<166>Jan  5 09:52:35 10.22.8.12 %ASA-6-305011: Built dynamic TCP translation from inside:10.22.8.188/64340 to outside:206.111.72.41/2013
-<166>Jan  5 15:52:35 10.22.8.33 %ASA-6-305012: Teardown dynamic UDP translation from inside:192.111.72.2/62251 to outside:79.111.72.174/21311 duration 0:02:30
-<166>Jan  5 08:52:35 10.22.8.216 %ASA-6-302015: Built inbound UDP connection 212806058 for outside:10.22.8.221/56631 (10.22.8.221/56631)(LOCAL\user.name) to inside:10.22.8.26/389 (10.22.8.26/389) (user.name)
-<142>Jan  5 08:52:35 10.22.8.201 %ASA-6-302014: Teardown TCP connection 488168189 for Outside_VPN:209.111.72.10/56619 to DMZ-Inside:10.22.8.53/443 duration 0:00:00 bytes 2477 TCP FINs
-<142>Jan  5 08:52:35 10.22.8.201 %ASA-6-106015: Deny TCP (no connection) from 10.22.8.112/52235 to 198.111.72.227/80 flags ACK  on interface Inside-Trunk
-<142>Jan  5 08:52:35 10.22.8.201 %ASA-6-302014: Teardown TCP connection 488167192 for Outside_VPN:115.111.72.7/49196 to DMZ-Inside:10.22.8.57/443 duration 0:00:02 bytes 20588 TCP Reset-O
-<166>Jan  5 08:52:35 10.22.8.216 %ASA-6-302016: Teardown UDP connection 212806055 for outside:10.22.8.62/55383(LOCAL\user.name) to inside:10.22.8.85/53 duration 0:00:00 bytes 349 (user.name)
-<142>Jan  5 08:52:35 10.22.8.201 %ASA-6-302014: Teardown TCP connection 488168380 for Outside_VPN:74.111.72.12/443 to Inside-Trunk:10.22.8.39/54894 duration 0:00:00 bytes 5701 TCP FINs
-<174>Jan  5 14:52:35 10.22.8.212 %ASA-6-302013: Built inbound TCP connection 76245522 for outside:10.22.8.147/56343 (10.22.8.147/56343) to inside:209.111.72.151/443 (209.111.72.151/443) (user.name)
-<142>Jan  5 08:52:35 10.22.8.201 %ASA-6-302014: Teardown TCP connection 488168443 for Outside_VPN:23.111.72.27/80 to Inside-Trunk:10.22.8.81/64713 duration 0:00:00 bytes 2426 TCP FINs
-<142>Jan  5 08:52:35 10.22.8.201 %ASA-6-302014: Teardown TCP connection 488111566 for Outside_VPN:131.111.72.49/443 to Inside-Trunk:10.22.8.127/56558 duration 0:01:57 bytes 3614 TCP Reset-O
-<166>Jan  5 08:52:35 10.22.8.216 %ASA-6-302013: Built inbound TCP connection 212806061 for outside:10.22.8.17/58635 (10.22.8.17/58635)(LOCAL\user.name) to inside:10.22.8.12/389 (10.22.8.12/389) (user.name)
-<166>Jan  5 08:52:35 10.22.8.216 %ASA-6-302014: Teardown TCP connection 212806010 for outside:10.22.8.33/60223(LOCAL\user.name) to inside:10.22.8.86/389 duration 0:00:00 bytes 416 TCP Reset-I (user.name)
-<166>Jan  5 08:52:35 10.22.8.216 %ASA-6-302015: Built inbound UDP connection 212806062 for outside:10.22.8.221/56632 (10.22.8.221/56632)(LOCAL\user.name) to inside:10.22.8.73/389 (10.22.8.73/389) (user.name)
-<167>Jan  5 08:52:35 10.22.8.216 %ASA-7-609002: Teardown local-host inside:10.22.8.205 duration 0:00:00
-<142>Jan  5 08:52:35 10.22.8.201 %ASA-6-302014: Teardown TCP connection 488168231 for Outside_VPN:204.111.72.243/3011 to Inside-Trunk:10.22.8.208/60037 duration 0:00:00 bytes 19415 TCP FINs
-<166>Jan  5 16:52:35 10.22.8.41 %ASA-6-302013: Built inbound TCP connection 45476108 for Outside:10.22.8.97/53484 (10.22.8.97/53484)(LOCAL\user.name) to Inside:141.111.72.70/7576 (141.111.72.70/7576) (user.name)
-<174>Jan  5 14:52:35 10.22.8.212 %ASA-6-302013: Built inbound TCP connection 76245527 for outside:10.22.8.97/65195 (10.22.8.97/65195) to inside:17.111.72.212/5223 (17.111.72.212/5223) (user.name)
-<166>Jan  5 08:52:35 10.22.8.216 %ASA-6-302014: Teardown TCP connection 212806018 for outside:10.22.8.17/58632(LOCAL\user.name) to inside:10.22.8.12/389 duration 0:00:00 bytes 0 TCP FINs (user.name)
-<142>Jan  5 08:52:35 10.22.8.201 %ASA-6-302014: Teardown TCP connection 488168562 for DMZ-Inside:10.22.8.51/51236 to Inside-Trunk:10.22.8.174/40004 duration 0:00:00 bytes 2273 TCP FINs
-<166>Jan  5 08:52:35 10.22.8.216 %ASA-6-302015: Built inbound UDP connection 212806065 for outside:10.22.8.62/59829 (10.22.8.62/59829)(LOCAL\user.name) to inside:10.22.8.85/53 (10.22.8.85/53) (user.name)
-<166>Jan  5 08:52:35 10.22.8.216 %ASA-6-302013: Built inbound TCP connection 212806067 for outside:10.22.8.143/62675 (10.22.8.143/62675)(LOCAL\user.name) to inside:141.111.72.12/389 (141.111.72.12/389) (user.name)
-<167>Jan  5 08:52:35 10.22.8.216 %ASA-7-710005: UDP request discarded from 10.22.8.223/61122 to outside:224.111.72.252/5355
-<166>Jan  5 08:52:35 10.22.8.216 %ASA-6-302020: Built inbound ICMP connection for faddr 10.22.8.143/0(LOCAL\user.name) gaddr 141.111.72.12/0 laddr 141.111.72.12/0 (user.name)
-<142>Jan  5 08:52:35 10.22.8.201 %ASA-6-302014: Teardown TCP connection 488168547 for Outside_VPN:107.111.72.102/80 to Inside-Trunk:10.22.8.54/61676 duration 0:00:00 bytes 1030 TCP FINs
-<166>Jan  5 08:52:35 10.22.8.216 %ASA-6-302015: Built inbound UDP connection 212806078 for outside:10.22.8.221/56633 (10.22.8.221/56633)(LOCAL\user.name) to inside:10.22.8.20/389 (10.22.8.20/389) (user.name)
-<166>Jan  5 09:52:35 10.22.8.12 %ASA-6-305011: Built dynamic TCP translation from inside:10.22.8.83/59915 to outside:206.111.72.41/22776
-<142>Jan  5 08:52:36 10.22.8.201 %ASA-6-302014: Teardown TCP connection 488168044 for Outside_VPN:50.111.72.39/80 to Inside-Trunk:10.22.8.75/60877 duration 0:00:01 bytes 13304 TCP FINs
-<142>Jan  5 08:52:36 10.22.8.201 %ASA-6-302014: Teardown TCP connection 488118326 for Outside_VPN:23.111.72.27/80 to Inside-Trunk:10.22.8.229/57901 duration 0:01:45 bytes 1942 TCP FINs
-<142>Jan  5 08:52:36 10.22.8.201 %ASA-6-302014: Teardown TCP connection 488160565 for Outside_VPN:72.111.72.29/80 to Inside-Trunk:10.22.8.42/57520 duration 0:00:15 bytes 1025 TCP FINs
-<142>Jan  5 08:52:36 10.22.8.201 %ASA-6-302014: Teardown TCP connection 488096423 for Outside_VPN:72.111.72.43/80 to Inside-Trunk:10.22.8.127/59096 duration 0:02:27 bytes 99347 TCP Reset-O
-<142>Jan  5 08:52:36 10.22.8.201 %ASA-6-302014: Teardown TCP connection 488095522 for Outside_VPN:72.111.72.43/80 to Inside-Trunk:10.22.8.127/59087 duration 0:02:29 bytes 154785 TCP Reset-O
-<142>Jan  5 08:52:36 10.22.8.201 %ASA-6-302014: Teardown TCP connection 488106557 for Outside_VPN:72.111.72.43/80 to Inside-Trunk:10.22.8.127/59134 duration 0:02:09 bytes 25319 TCP Reset-O
-<142>Jan  5 08:52:36 10.22.8.201 %ASA-6-302014: Teardown TCP connection 488096426 for Outside_VPN:72.111.72.43/80 to Inside-Trunk:10.22.8.127/59099 duration 0:02:27 bytes 26171 TCP Reset-O
-<166>Jan  5 08:52:36 10.22.8.216 %ASA-6-302014: Teardown TCP connection 212806005 for outside:10.22.8.17/58630(LOCAL\user.name) to inside:10.22.8.12/389 duration 0:00:00 bytes 3942 TCP FINs (user.name)
-<166>Jan  5 08:52:36 10.22.8.216 %ASA-6-302015: Built inbound UDP connection 212806085 for outside:10.22.8.143/54018 (10.22.8.143/54018)(LOCAL\user.name) to inside:10.22.8.85/53 (10.22.8.85/53) (user.name)
-<174>Jan  5 14:52:36 10.22.8.212 %ASA-6-302020: Built inbound ICMP connection for faddr 10.22.8.96/2708 gaddr 10.22.8.30/0 laddr 10.22.8.30/0 (user.name)
-<174>Jan  5 14:52:36 10.22.8.212 %ASA-6-302015: Built inbound UDP connection 76245537 for outside:10.22.8.110/49886 (10.22.8.110/49886) to inside:192.111.72.11/8612 (192.111.72.11/8612) (user.name)
-<166>Jan  5 16:52:36 10.22.8.41 %ASA-6-106015: Deny TCP (no connection) from 10.22.8.85/58359 to 10.22.8.11/88 flags RST ACK  on interface Outside
-<166>Jan  5 08:52:36 10.22.8.216 %ASA-6-302021: Teardown ICMP connection for faddr 10.22.8.82/0(LOCAL\user.name) gaddr 10.22.8.205/0 laddr 10.22.8.205/0
-<166>Jan  5 08:52:36 10.22.8.216 %ASA-6-302016: Teardown UDP connection 212799832 for outside:10.22.8.230/55549(LOCAL\user.name) to inside:10.22.8.11/389 duration 0:02:01 bytes 354 (user.name)
-<166>Jan  5 08:52:36 10.22.8.216 %ASA-6-302016: Teardown UDP connection 212799867 for outside:10.22.8.240/138(LOCAL\user.name) to inside:10.22.8.255/138 duration 0:02:01 bytes 214 (user.name)
-<167>Jan  5 08:52:36 10.22.8.216 %ASA-7-609001: Built local-host inside:67.111.72.204
-<174>Jan  5 14:52:36 10.22.8.212 %ASA-6-302013: Built inbound TCP connection 76245544 for outside:10.22.8.227/54540 (10.22.8.227/54540) to inside:63.111.72.124/80 (63.111.72.124/80) (user.name)
-<142>Jan  5 08:52:36 10.22.8.201 %ASA-6-302014: Teardown TCP connection 488168135 for Outside_VPN:198.111.72.66/36797 to DMZ-Inside:10.22.8.53/80 duration 0:00:01 bytes 89039 TCP FINs
-<166>Jan  5 08:52:36 10.22.8.216 %ASA-6-302014: Teardown TCP connection 212805836 for outside:10.22.8.62/56471(LOCAL\user.name) to inside:208.111.72.1/443 duration 0:00:04 bytes 1700 TCP FINs (user.name)
-<174>Jan  5 14:52:36 10.22.8.212 %ASA-6-302013: Built inbound TCP connection 76245546 for outside:10.22.8.227/54542 (10.22.8.227/54542) to inside:63.111.72.124/80 (63.111.72.124/80) (user.name)
-<166>Jan  5 08:52:36 10.22.8.216 %ASA-6-302021: Teardown ICMP connection for faddr 10.22.8.74/0(LOCAL\user.name) gaddr 10.22.8.205/0 laddr 10.22.8.205/0
-<174>Jan  5 14:52:36 10.22.8.212 %ASA-6-302020: Built outbound ICMP connection for faddr 10.22.8.96/2708 gaddr 10.22.8.30/0 laddr 10.22.8.30/0
-<142>Jan  5 08:52:36 10.22.8.201 %ASA-6-302014: Teardown TCP connection 488168388 for DMZ-Inside:10.22.8.10/49771 to Inside-Trunk:10.22.8.128/443 duration 0:00:00 bytes 19132 TCP Reset-O
-<142>Jan  5 08:52:36 10.22.8.201 %ASA-6-302014: Teardown TCP connection 488168692 for DMZ-Inside:10.22.8.53/61694 to Inside-Trunk:10.22.8.174/40004 duration 0:00:00 bytes 5660 TCP FINs
-<174>Jan  5 14:52:36 10.22.8.212 %ASA-6-302013: Built inbound TCP connection 76245552 for outside:10.22.8.92/51042 (10.22.8.92/51042) to inside:10.22.8.193/9100 (10.22.8.193/9100) (user.name)
-<166>Jan  5 16:52:36 10.22.8.41 %ASA-6-302016: Teardown UDP connection 45474680 for Outside:10.22.8.49/137(LOCAL\user.name) to Inside:10.22.8.12/137 duration 0:02:03 bytes 486 (user.name)
-<166>Jan  5 16:52:36 10.22.8.41 %ASA-6-302016: Teardown UDP connection 45474694 for Outside:10.22.8.49/138(LOCAL\user.name) to Inside:10.22.8.12/138 duration 0:02:01 bytes 184 (user.name)
-<142>Jan  5 08:52:36 10.22.8.201 %ASA-6-302014: Teardown TCP connection 488167720 for Outside_VPN:198.111.72.75/1033 to DMZ-Inside:10.22.8.53/443 duration 0:00:01 bytes 9634 TCP FINs
-<142>Jan  5 08:52:32 10.22.8.201 %ASA-6-302014: Teardown TCP connection 488165627 for Outside_VPN:170.111.72.22/27463 to DMZ-Inside:10.22.8.53/443 duration 0:00:01 bytes 9756 TCP FINs
-<166>Jan  5 08:52:32 10.22.8.216 %ASA-6-302016: Teardown UDP connection 212805854 for outside:10.22.8.62/54704(LOCAL\user.name) to inside:10.22.8.85/53 duration 0:00:00 bytes 114 (user.name)
-<166>Jan  5 09:52:32 10.22.8.12 %ASA-6-302020: Built inbound ICMP connection for faddr 207.111.72.122/0 gaddr 206.111.72.24/512 laddr 10.22.8.57/512
-<166>Jan  5 09:52:32 10.22.8.12 %ASA-6-302013: Built outbound TCP connection 17605397 for outside:69.111.72.0/80 (69.111.72.0/80) to inside:10.22.8.102/55659 (206.111.72.41/40627)
-<174>Jan  5 14:52:32 10.22.8.212 %ASA-6-302015: Built inbound UDP connection 76245230 for outside:10.22.8.96/123 (10.22.8.96/123) to inside:10.22.8.12/123 (10.22.8.12/123) (user.name)
-<142>Jan  5 08:52:32 10.22.8.201 %ASA-6-302014: Teardown TCP connection 488031413 for Outside_VPN:184.111.72.216/50341 to DMZ-Inside:10.22.8.57/443 duration 0:05:01 bytes 13543 TCP Reset-O
-<166>Jan  5 16:52:32 10.22.8.41 %ASA-6-302020: Built inbound ICMP connection for faddr 10.22.8.95/1(LOCAL\user.name) gaddr 10.22.8.12/0 laddr 10.22.8.12/0 (user.name)
-<142>Jan  5 08:52:32 10.22.8.201 %ASA-6-302014: Teardown TCP connection 488030393 for DMZ-Inside:[10.22.8.10/57109 to Inside-Trunk:10.22.8.128/443 duration 0:05:04 bytes 13541 TCP Reset-O
-<166>Jan  5 09:52:32 10.22.8.12 %ASA-6-305012: Teardown dynamic TCP translation from inside:10.22.8.149/62156 to outside:206.111.72.41/19576 duration 0:00:44
-<166>Jan  5 09:52:32 10.22.8.12 %ASA-6-305012: Teardown dynamic TCP translation from inside:10.22.8.149/62159 to outside:206.111.72.41/39634 duration 0:00:44
-<142>Jan  5 08:52:32 10.22.8.201 %ASA-6-302014: Teardown TCP connection 488031793 for Outside_VPN:198.111.72.146/28026 to DMZ-Inside:10.22.8.53/443 duration 0:05:00 bytes 119 TCP FINs
-<142>Jan  5 08:52:32 10.22.8.201 %ASA-6-302014: Teardown TCP connection 488030810 for DMZ-Inside:10.22.8.10/56930 to Inside-Trunk:10.22.8.128/443 duration 0:05:03 bytes 13543 TCP Reset-O
-<142>Jan  5 08:52:32 10.22.8.201 %ASA-6-106015: Deny TCP (no connection) from 186.111.72.11/80 to 204.111.72.199/61438 flags SYN ACK  on interface Outside_VPN
-<166>Jan  5 08:52:32 10.22.8.216 %ASA-6-302013: Built inbound TCP connection 212805863 for outside:10.22.8.144/61999 (10.22.8.144/61999)(LOCAL\user.name) to inside:10.22.8.163/80 (10.22.8.163/80) (user.name)
-<167>Jan  5 08:52:32 10.22.8.216 %ASA-7-609002: Teardown local-host inside:10.22.8.205 duration 0:00:00
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/df8d682e/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
deleted file mode 100644
index 9738691..0000000
--- a/metron-platform/metron-integration-test/src/main/resources/sample/data/SampleInput/BluecoatSyslog.txt
+++ /dev/null
@@ -1,144 +0,0 @@
-<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


[03/15] incubator-metron git commit: METRON-142 Simplify Parser configuration (merrimanr via cestella) closes apache/incubator-metron#120

Posted by ce...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/df8d682e/metron-platform/metron-integration-test/src/main/sample/data/bluecoat/raw/BluecoatSyslog.txt
----------------------------------------------------------------------
diff --git a/metron-platform/metron-integration-test/src/main/sample/data/bluecoat/raw/BluecoatSyslog.txt b/metron-platform/metron-integration-test/src/main/sample/data/bluecoat/raw/BluecoatSyslog.txt
new file mode 100644
index 0000000..9738691
--- /dev/null
+++ b/metron-platform/metron-integration-test/src/main/sample/data/bluecoat/raw/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/df8d682e/metron-platform/metron-integration-test/src/main/sample/data/bro/parsed/BroExampleParsed
----------------------------------------------------------------------
diff --git a/metron-platform/metron-integration-test/src/main/sample/data/bro/parsed/BroExampleParsed b/metron-platform/metron-integration-test/src/main/sample/data/bro/parsed/BroExampleParsed
new file mode 100644
index 0000000..d9ddb0d
--- /dev/null
+++ b/metron-platform/metron-integration-test/src/main/sample/data/bro/parsed/BroExampleParsed
@@ -0,0 +1,10 @@
+{"bro_timestamp":"1402307733473","status_code":200,"method":"GET","ip_dst_port":80,"request_body_len":0,"uri":"\/","tags":[],"source.type":"bro","uid":"CTo78A11g7CYbbOHvj","resp_mime_types":["text\/html"],"trans_depth":1,"protocol":"http","original_string":"HTTP | id.orig_p:58808 status_code:200 method:GET request_body_len:0 id.resp_p:80 uri:\/ tags:[] uid:CTo78A11g7CYbbOHvj resp_mime_types:[\"text\\\/html\"] trans_depth:1 host:www.cisco.com status_msg:OK id.orig_h:192.249.113.37 response_body_len:25523 user_agent:curl\/7.22.0 (x86_64-pc-linux-gnu) libcurl\/7.22.0 OpenSSL\/1.0.1 zlib\/1.2.3.4 libidn\/1.23 librtmp\/2.3 ts:1402307733473 id.resp_h:72.163.4.161 resp_fuids:[\"FJDyMC15lxUn5ngPfd\"]","ip_dst_addr":"72.163.4.161","ip_src_port":58808,"host":"www.cisco.com","status_msg":"OK","response_body_len":25523,"ip_src_addr":"192.249.113.37","user_agent":"curl\/7.22.0 (x86_64-pc-linux-gnu) libcurl\/7.22.0 OpenSSL\/1.0.1 zlib\/1.2.3.4 libidn\/1.23 librtmp\/2.3","resp_fuids":["FJDyMC15lxU
 n5ngPfd"],"timestamp":1402307733473}
+{"TTLs":[3600.0,289.0,14.0],"qclass_name":"C_INTERNET","bro_timestamp":"1402308259609","qtype_name":"AAAA","ip_dst_port":53,"qtype":28,"rejected":false,"answers":["www.cisco.com.akadns.net","origin-www.cisco.com","2001:420:1201:2::a"],"trans_id":62418,"uid":"CuJT272SKaJSuqO0Ia","protocol":"dns","original_string":"DNS | AA:true TTLs:[3600.0,289.0,14.0] qclass_name:C_INTERNET id.orig_p:33976 qtype_name:AAAA qtype:28 rejected:false id.resp_p:53 query:www.cisco.com answers:[\"www.cisco.com.akadns.net\",\"origin-www.cisco.com\",\"2001:420:1201:2::a\"] trans_id:62418 rcode:0 rcode_name:NOERROR TC:false RA:true uid:CuJT272SKaJSuqO0Ia RD:true proto:udp id.orig_h:10.122.196.204 Z:0 qclass:1 ts:1402308259609 id.resp_h:144.254.71.184","ip_dst_addr":"144.254.71.184","Z":0,"ip_src_addr":"10.122.196.204","qclass":1,"timestamp":1402308259609,"AA":true,"query":"www.cisco.com","rcode":0,"rcode_name":"NOERROR","TC":false,"RA":true,"source.type":"bro","RD":true,"ip_src_port":33976,"proto":"udp"}
+{"bro_timestamp":"1402307733473","status_code":200,"method":"GET","ip_dst_port":80,"request_body_len":0,"uri":"\/","tags":[],"source.type":"bro","uid":"KIRAN","resp_mime_types":["text\/html"],"trans_depth":1,"protocol":"http","original_string":"HTTP | id.orig_p:58808 status_code:200 method:GET request_body_len:0 id.resp_p:80 uri:\/ tags:[] uid:KIRAN resp_mime_types:[\"text\\\/html\"] trans_depth:1 host:www.cisco.com status_msg:OK id.orig_h:10.122.196.204 response_body_len:25523 user_agent:curl\/7.22.0 (x86_64-pc-linux-gnu) libcurl\/7.22.0 OpenSSL\/1.0.1 zlib\/1.2.3.4 libidn\/1.23 librtmp\/2.3 ts:1402307733473 id.resp_h:72.163.4.161 resp_fuids:[\"FJDyMC15lxUn5ngPfd\"]","ip_dst_addr":"72.163.4.161","ip_src_port":58808,"host":"www.cisco.com","status_msg":"OK","response_body_len":25523,"ip_src_addr":"10.122.196.204","user_agent":"curl\/7.22.0 (x86_64-pc-linux-gnu) libcurl\/7.22.0 OpenSSL\/1.0.1 zlib\/1.2.3.4 libidn\/1.23 librtmp\/2.3","resp_fuids":["FJDyMC15lxUn5ngPfd"],"timestamp":1402
 307733473}
+{"bro_timestamp":"1402307733473","status_code":200,"method":"GET","ip_dst_port":80,"request_body_len":0,"uri":"\/","tags":[],"source.type":"bro","uid":"KIRAN12312312","resp_mime_types":["text\/html"],"trans_depth":1,"protocol":"http","original_string":"HTTP | id.orig_p:58808 status_code:200 method:GET request_body_len:0 id.resp_p:80 uri:\/ tags:[] uid:KIRAN12312312 resp_mime_types:[\"text\\\/html\"] trans_depth:1 host:www.cisco.com status_msg:OK id.orig_h:192.249.113.37 response_body_len:25523 user_agent:curl\/7.22.0 (x86_64-pc-linux-gnu) libcurl\/7.22.0 OpenSSL\/1.0.1 zlib\/1.2.3.4 libidn\/1.23 librtmp\/2.3 ts:1402307733473 id.resp_h:72.163.4.161 resp_fuids:[\"FJDyMC15lxUn5ngPfd\"]","ip_dst_addr":"72.163.4.161","ip_src_port":58808,"host":"www.cisco.com","status_msg":"OK","response_body_len":25523,"ip_src_addr":"192.249.113.37","user_agent":"curl\/7.22.0 (x86_64-pc-linux-gnu) libcurl\/7.22.0 OpenSSL\/1.0.1 zlib\/1.2.3.4 libidn\/1.23 librtmp\/2.3","resp_fuids":["FJDyMC15lxUn5ngPfd"],
 "timestamp":1402307733473}
+{"bro_timestamp":"1402307733473","status_code":200,"method":"GET","ip_dst_port":80,"request_body_len":0,"uri":"\/","tags":[],"source.type":"bro","uid":"KIRAN12312312","resp_mime_types":["text\/html"],"trans_depth":1,"protocol":"http","original_string":"HTTP | id.orig_p:58808 status_code:200 method:GET request_body_len:0 id.resp_p:80 uri:\/ tags:[] uid:KIRAN12312312 resp_mime_types:[\"text\\\/html\"] trans_depth:1 host:www.cisco.com status_msg:OK id.orig_h:192.249.113.37 response_body_len:25523 user_agent:curl\/7.22.0 (x86_64-pc-linux-gnu) libcurl\/7.22.0 OpenSSL\/1.0.1 zlib\/1.2.3.4 libidn\/1.23 librtmp\/2.3 ts:1402307733473 id.resp_h:72.163.4.161 resp_fuids:[\"FJDyMC15lxUn5ngPfd\"]","ip_dst_addr":"72.163.4.161","ip_src_port":58808,"host":"www.cisco.com","status_msg":"OK","response_body_len":25523,"ip_src_addr":"192.249.113.37","user_agent":"curl\/7.22.0 (x86_64-pc-linux-gnu) libcurl\/7.22.0 OpenSSL\/1.0.1 zlib\/1.2.3.4 libidn\/1.23 librtmp\/2.3","resp_fuids":["FJDyMC15lxUn5ngPfd"],
 "timestamp":1402307733473}
+{"bro_timestamp":"1402307733473","status_code":200,"method":"GET","ip_dst_port":80,"request_body_len":0,"uri":"\/","tags":[],"source.type":"bro","uid":"CTo78A11g7CYbbOHvj","resp_mime_types":["text\/html"],"trans_depth":1,"protocol":"http","original_string":"HTTP | id.orig_p:58808 status_code:200 method:GET request_body_len:0 id.resp_p:80 uri:\/ tags:[] uid:CTo78A11g7CYbbOHvj resp_mime_types:[\"text\\\/html\"] trans_depth:1 host:gabacentre.pw status_msg:OK id.orig_h:10.122.196.204 response_body_len:25523 email:abullis@mail.csuchico.edu user_agent:curl\/7.22.0 (x86_64-pc-linux-gnu) libcurl\/7.22.0 OpenSSL\/1.0.1 zlib\/1.2.3.4 libidn\/1.23 librtmp\/2.3 ts:1402307733473 id.resp_h:72.163.4.161 resp_fuids:[\"FJDyMC15lxUn5ngPfd\"]","ip_dst_addr":"72.163.4.161","ip_src_port":58808,"host":"gabacentre.pw","status_msg":"OK","response_body_len":25523,"ip_src_addr":"10.122.196.204","email":"abullis@mail.csuchico.edu","user_agent":"curl\/7.22.0 (x86_64-pc-linux-gnu) libcurl\/7.22.0 OpenSSL\/1.0.1
  zlib\/1.2.3.4 libidn\/1.23 librtmp\/2.3","resp_fuids":["FJDyMC15lxUn5ngPfd"],"timestamp":1402307733473}
+{"TTLs":[3600.0,289.0,14.0],"qclass_name":"C_INTERNET","bro_timestamp":"1402308259609","qtype_name":"AAAA","ip_dst_port":53,"qtype":28,"rejected":false,"answers":["gabacentre.pw","www.cisco.com.akadns.net","origin-www.cisco.com","2001:420:1201:2::a"],"trans_id":62418,"uid":"CYbbOHvj","protocol":"dns","original_string":"DNS | AA:true TTLs:[3600.0,289.0,14.0] qclass_name:C_INTERNET id.orig_p:33976 qtype_name:AAAA qtype:28 rejected:false id.resp_p:53 query:www.cisco.com answers:[\"gabacentre.pw\",\"www.cisco.com.akadns.net\",\"origin-www.cisco.com\",\"2001:420:1201:2::a\"] trans_id:62418 rcode:0 rcode_name:NOERROR TC:false RA:true uid:CYbbOHvj RD:true proto:udp id.orig_h:93.188.160.43 Z:0 qclass:1 ts:1402308259609 id.resp_h:144.254.71.184","ip_dst_addr":"144.254.71.184","Z":0,"ip_src_addr":"93.188.160.43","qclass":1,"timestamp":1402308259609,"AA":true,"query":"www.cisco.com","rcode":0,"rcode_name":"NOERROR","TC":false,"RA":true,"source.type":"bro","RD":true,"ip_src_port":33976,"proto":
 "udp"}
+{"bro_timestamp":"1402307733473","status_code":200,"method":"GET","ip_dst_port":80,"request_body_len":0,"uri":"\/","tags":[],"source.type":"bro","uid":"CTo78A11g7CYbbOHvj","resp_mime_types":["text\/html"],"trans_depth":1,"protocol":"http","original_string":"HTTP | id.orig_p:58808 status_code:200 method:GET request_body_len:0 id.resp_p:80 uri:\/ tags:[] uid:CTo78A11g7CYbbOHvj resp_mime_types:[\"text\\\/html\"] trans_depth:1 host:www.cisco.com status_msg:OK id.orig_h:192.249.113.37 response_body_len:25523 user_agent:curl\/7.22.0 (x86_64-pc-linux-gnu) libcurl\/7.22.0 OpenSSL\/1.0.1 zlib\/1.2.3.4 libidn\/1.23 librtmp\/2.3 ts:1402307733473 id.resp_h:72.163.4.161 resp_fuids:[\"FJDyMC15lxUn5ngPfd\"]","ip_dst_addr":"72.163.4.161","ip_src_port":58808,"host":"www.cisco.com","status_msg":"OK","response_body_len":25523,"ip_src_addr":"192.249.113.37","user_agent":"curl\/7.22.0 (x86_64-pc-linux-gnu) libcurl\/7.22.0 OpenSSL\/1.0.1 zlib\/1.2.3.4 libidn\/1.23 librtmp\/2.3","resp_fuids":["FJDyMC15lxU
 n5ngPfd"],"timestamp":1402307733473}
+{"TTLs":[3600.0,289.0,14.0],"qclass_name":"C_INTERNET","bro_timestamp":"1402308259609","qtype_name":"AAAA","ip_dst_port":53,"qtype":28,"rejected":false,"answers":["www.cisco.com.akadns.net","origin-www.cisco.com","2001:420:1201:2::a"],"trans_id":62418,"uid":"CuJT272SKaJSuqO0Ia","protocol":"dns","original_string":"DNS | AA:true TTLs:[3600.0,289.0,14.0] qclass_name:C_INTERNET id.orig_p:33976 qtype_name:AAAA qtype:28 rejected:false id.resp_p:53 query:www.cisco.com answers:[\"www.cisco.com.akadns.net\",\"origin-www.cisco.com\",\"2001:420:1201:2::a\"] trans_id:62418 rcode:0 rcode_name:NOERROR TC:false RA:true uid:CuJT272SKaJSuqO0Ia RD:true proto:udp id.orig_h:10.122.196.204 Z:0 qclass:1 ts:1402308259609 id.resp_h:144.254.71.184","ip_dst_addr":"144.254.71.184","Z":0,"ip_src_addr":"10.122.196.204","qclass":1,"timestamp":1402308259609,"AA":true,"query":"www.cisco.com","rcode":0,"rcode_name":"NOERROR","TC":false,"RA":true,"source.type":"bro","RD":true,"ip_src_port":33976,"proto":"udp"}
+{"bro_timestamp":"1402307733473","status_code":200,"method":"GET","ip_dst_port":80,"request_body_len":0,"uri":"\/","tags":[],"source.type":"bro","uid":"KIRAN","resp_mime_types":["text\/html"],"trans_depth":1,"protocol":"http","original_string":"HTTP | id.orig_p:58808 status_code:200 method:GET request_body_len:0 id.resp_p:80 uri:\/ tags:[] uid:KIRAN resp_mime_types:[\"text\\\/html\"] trans_depth:1 host:www.cisco.com status_msg:OK id.orig_h:10.122.196.204 response_body_len:25523 user_agent:curl\/7.22.0 (x86_64-pc-linux-gnu) libcurl\/7.22.0 OpenSSL\/1.0.1 zlib\/1.2.3.4 libidn\/1.23 librtmp\/2.3 ts:1402307733473 id.resp_h:72.163.4.161 resp_fuids:[\"FJDyMC15lxUn5ngPfd\"]","ip_dst_addr":"72.163.4.161","ip_src_port":58808,"host":"www.cisco.com","status_msg":"OK","response_body_len":25523,"ip_src_addr":"10.122.196.204","user_agent":"curl\/7.22.0 (x86_64-pc-linux-gnu) libcurl\/7.22.0 OpenSSL\/1.0.1 zlib\/1.2.3.4 libidn\/1.23 librtmp\/2.3","resp_fuids":["FJDyMC15lxUn5ngPfd"],"timestamp":1402
 307733473}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/df8d682e/metron-platform/metron-integration-test/src/main/sample/data/bro/raw/BroExampleOutput
----------------------------------------------------------------------
diff --git a/metron-platform/metron-integration-test/src/main/sample/data/bro/raw/BroExampleOutput b/metron-platform/metron-integration-test/src/main/sample/data/bro/raw/BroExampleOutput
new file mode 100644
index 0000000..efd576a
--- /dev/null
+++ b/metron-platform/metron-integration-test/src/main/sample/data/bro/raw/BroExampleOutput
@@ -0,0 +1,10 @@
+{"http":{"ts":1402307733473,"uid":"CTo78A11g7CYbbOHvj","id.orig_h":"192.249.113.37","id.orig_p":58808,"id.resp_h":"72.163.4.161","id.resp_p":80,"trans_depth":1,"method":"GET","host":"www.cisco.com","uri":"/","user_agent":"curl/7.22.0 (x86_64-pc-linux-gnu) libcurl/7.22.0 OpenSSL/1.0.1 zlib/1.2.3.4 libidn/1.23 librtmp/2.3","request_body_len":0,"response_body_len":25523,"status_code":200,"status_msg":"OK","tags":[],"resp_fuids":["FJDyMC15lxUn5ngPfd"],"resp_mime_types":["text/html"]}}
+{"dns":{"ts":1402308259609,"uid":"CuJT272SKaJSuqO0Ia","id.orig_h":"10.122.196.204","id.orig_p":33976,"id.resp_h":"144.254.71.184","id.resp_p":53,"proto":"udp","trans_id":62418,"query":"www.cisco.com","qclass":1,"qclass_name":"C_INTERNET","qtype":28,"qtype_name":"AAAA","rcode":0,"rcode_name":"NOERROR","AA":true,"TC":false,"RD":true,"RA":true,"Z":0,"answers":["www.cisco.com.akadns.net","origin-www.cisco.com","2001:420:1201:2::a"],"TTLs":[3600.0,289.0,14.0],"rejected":false}}
+{"http":{"ts":1402307733473,"uid":"KIRAN","id.orig_h":"10.122.196.204","id.orig_p":58808,"id.resp_h":"72.163.4.161","id.resp_p":80,"trans_depth":1,"method":"GET","host":"www.cisco.com","uri":"/","user_agent":"curl/7.22.0 (x86_64-pc-linux-gnu) libcurl/7.22.0 OpenSSL/1.0.1 zlib/1.2.3.4 libidn/1.23 librtmp/2.3","request_body_len":0,"response_body_len":25523,"status_code":200,"status_msg":"OK","tags":[],"resp_fuids":["FJDyMC15lxUn5ngPfd"],"resp_mime_types":["text/html"]}}
+{"http":{"ts":1402307733473,"uid":"KIRAN12312312","id.orig_h":"192.249.113.37","id.orig_p":58808,"id.resp_h":"72.163.4.161","id.resp_p":80,"trans_depth":1,"method":"GET","host":"www.cisco.com","uri":"/","user_agent":"curl/7.22.0 (x86_64-pc-linux-gnu) libcurl/7.22.0 OpenSSL/1.0.1 zlib/1.2.3.4 libidn/1.23 librtmp/2.3","request_body_len":0,"response_body_len":25523,"status_code":200,"status_msg":"OK","tags":[],"resp_fuids":["FJDyMC15lxUn5ngPfd"],"resp_mime_types":["text/html"]}}
+{"http":{"ts":1402307733473,"uid":"KIRAN12312312","id.orig_h":"192.249.113.37","id.orig_p":58808,"id.resp_h":"72.163.4.161","id.resp_p":80,"trans_depth":1,"method":"GET","host":"www.cisco.com","uri":"/","user_agent":"curl/7.22.0 (x86_64-pc-linux-gnu) libcurl/7.22.0 OpenSSL/1.0.1 zlib/1.2.3.4 libidn/1.23 librtmp/2.3","request_body_len":0,"response_body_len":25523,"status_code":200,"status_msg":"OK","tags":[],"resp_fuids":["FJDyMC15lxUn5ngPfd"],"resp_mime_types":["text/html"]}}
+{"http":{"ts":1402307733473,"uid":"CTo78A11g7CYbbOHvj","id.orig_h":"10.122.196.204","id.orig_p":58808,"id.resp_h":"72.163.4.161","id.resp_p":80,"trans_depth":1,"email":"abullis@mail.csuchico.edu","method":"GET","host":"gabacentre.pw","uri":"/","user_agent":"curl/7.22.0 (x86_64-pc-linux-gnu) libcurl/7.22.0 OpenSSL/1.0.1 zlib/1.2.3.4 libidn/1.23 librtmp/2.3","request_body_len":0,"response_body_len":25523,"status_code":200,"status_msg":"OK","tags":[],"resp_fuids":["FJDyMC15lxUn5ngPfd"],"resp_mime_types":["text/html"]}}
+{"dns":{"ts":1402308259609,"uid":"CYbbOHvj","id.orig_h":"93.188.160.43","id.orig_p":33976,"id.resp_h":"144.254.71.184","id.resp_p":53,"proto":"udp","trans_id":62418,"query":"www.cisco.com","qclass":1,"qclass_name":"C_INTERNET","qtype":28,"qtype_name":"AAAA","rcode":0,"rcode_name":"NOERROR","AA":true,"TC":false,"RD":true,"RA":true,"Z":0,"answers":["gabacentre.pw","www.cisco.com.akadns.net","origin-www.cisco.com","2001:420:1201:2::a"],"TTLs":[3600.0,289.0,14.0],"rejected":false}}
+{"http":{"ts":1402307733473,"uid":"CTo78A11g7CYbbOHvj","id.orig_h":"192.249.113.37","id.orig_p":58808,"id.resp_h":"72.163.4.161","id.resp_p":80,"trans_depth":1,"method":"GET","host":"www.cisco.com","uri":"/","user_agent":"curl/7.22.0 (x86_64-pc-linux-gnu) libcurl/7.22.0 OpenSSL/1.0.1 zlib/1.2.3.4 libidn/1.23 librtmp/2.3","request_body_len":0,"response_body_len":25523,"status_code":200,"status_msg":"OK","tags":[],"resp_fuids":["FJDyMC15lxUn5ngPfd"],"resp_mime_types":["text/html"]}}
+{"dns":{"ts":1402308259609,"uid":"CuJT272SKaJSuqO0Ia","id.orig_h":"10.122.196.204","id.orig_p":33976,"id.resp_h":"144.254.71.184","id.resp_p":53,"proto":"udp","trans_id":62418,"query":"www.cisco.com","qclass":1,"qclass_name":"C_INTERNET","qtype":28,"qtype_name":"AAAA","rcode":0,"rcode_name":"NOERROR","AA":true,"TC":false,"RD":true,"RA":true,"Z":0,"answers":["www.cisco.com.akadns.net","origin-www.cisco.com","2001:420:1201:2::a"],"TTLs":[3600.0,289.0,14.0],"rejected":false}}
+{"http":{"ts":1402307733473,"uid":"KIRAN","id.orig_h":"10.122.196.204","id.orig_p":58808,"id.resp_h":"72.163.4.161","id.resp_p":80,"trans_depth":1,"method":"GET","host":"www.cisco.com","uri":"/","user_agent":"curl/7.22.0 (x86_64-pc-linux-gnu) libcurl/7.22.0 OpenSSL/1.0.1 zlib/1.2.3.4 libidn/1.23 librtmp/2.3","request_body_len":0,"response_body_len":25523,"status_code":200,"status_msg":"OK","tags":[],"resp_fuids":["FJDyMC15lxUn5ngPfd"],"resp_mime_types":["text/html"]}}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/df8d682e/metron-platform/metron-integration-test/src/main/sample/data/snort/parsed/SnortParsed
----------------------------------------------------------------------
diff --git a/metron-platform/metron-integration-test/src/main/sample/data/snort/parsed/SnortParsed b/metron-platform/metron-integration-test/src/main/sample/data/snort/parsed/SnortParsed
new file mode 100644
index 0000000..318b158
--- /dev/null
+++ b/metron-platform/metron-integration-test/src/main/sample/data/snort/parsed/SnortParsed
@@ -0,0 +1,3 @@
+{"msg":"\"Consecutive TCP small segments exceeding threshold\"","sig_rev":"1","ip_dst_addr":"10.0.2.15","ip_dst_port":"22","ethsrc":"52:54:00:12:35:02","tcpseq":"0x9AFF3D7","dgmlen":"64","icmpid":"","tcplen":"","tcpwindow":"0xFFFF","icmpseq":"","tcpack":"0xC8761D52","original_string":"01\/27-16:01:04.877970 ,129,12,1,\"Consecutive TCP small segments exceeding threshold\",TCP,10.0.2.2,56642,10.0.2.15,22,52:54:00:12:35:02,08:00:27:7F:93:2D,0x4E,***AP***,0x9AFF3D7,0xC8761D52,,0xFFFF,64,0,59677,64,65536,,,,","icmpcode":"","tos":"0","id":"59677","timestamp":1453932941970,"ethdst":"08:00:27:7F:93:2D","ip_src_addr":"10.0.2.2","ttl":"64","source.type":"snort","ethlen":"0x4E","iplen":"65536","icmptype":"","protocol":"TCP","ip_src_port":"56642","tcpflags":"***AP***","sig_id":"12","sig_generator":"129", "is_alert" : "true"}
+{"msg":"\"Consecutive TCP small segments exceeding threshold\"","sig_rev":"1","ip_dst_addr":"10.0.2.15","ip_dst_port":"50895","ethsrc":"52:54:00:12:35:02","tcpseq":"0xDB45F7A","dgmlen":"96","icmpid":"","tcplen":"","tcpwindow":"0xFFFF","icmpseq":"","tcpack":"0x7701DD5B","original_string":"02\/22-15:56:48.612494 ,129,12,1,\"Consecutive TCP small segments exceeding threshold\",TCP,96.44.142.5,80,10.0.2.15,50895,52:54:00:12:35:02,08:00:27:7F:93:2D,0x6E,***AP***,0xDB45F7A,0x7701DD5B,,0xFFFF,64,0,16785,96,98304,,,,","icmpcode":"","tos":"0","id":"16785","timestamp":1456178820494,"ethdst":"08:00:27:7F:93:2D","ip_src_addr":"96.44.142.5","ttl":"64","source.type":"snort","ethlen":"0x6E","iplen":"98304","icmptype":"","protocol":"TCP","ip_src_port":"80","tcpflags":"***AP***","sig_id":"12","sig_generator":"129", "is_alert" : "true"}
+{"msg":"\"Consecutive TCP small segments exceeding threshold\"","sig_rev":"1","ip_dst_addr":"10.0.2.15","ip_dst_port":"50895","ethsrc":"52:54:00:12:35:02","tcpseq":"0xDB508F2","dgmlen":"152","icmpid":"","tcplen":"","tcpwindow":"0xFFFF","icmpseq":"","tcpack":"0x7701DD5B","original_string":"02\/22-15:56:48.616775 ,129,12,1,\"Consecutive TCP small segments exceeding threshold\",TCP,96.44.142.5,80,10.0.2.15,50895,52:54:00:12:35:02,08:00:27:7F:93:2D,0xA6,***AP***,0xDB508F2,0x7701DD5B,,0xFFFF,64,0,16824,152,155648,,,,","icmpcode":"","tos":"0","id":"16824","timestamp":1456178824775,"ethdst":"08:00:27:7F:93:2D","ip_src_addr":"96.44.142.5","ttl":"64","source.type":"snort","ethlen":"0xA6","iplen":"155648","icmptype":"","protocol":"TCP","ip_src_port":"80","tcpflags":"***AP***","sig_id":"12","sig_generator":"129", "is_alert" : "true"}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/df8d682e/metron-platform/metron-integration-test/src/main/sample/data/snort/raw/SnortOutput
----------------------------------------------------------------------
diff --git a/metron-platform/metron-integration-test/src/main/sample/data/snort/raw/SnortOutput b/metron-platform/metron-integration-test/src/main/sample/data/snort/raw/SnortOutput
new file mode 100644
index 0000000..0497b0f
--- /dev/null
+++ b/metron-platform/metron-integration-test/src/main/sample/data/snort/raw/SnortOutput
@@ -0,0 +1,3 @@
+01/27-16:01:04.877970 ,129,12,1,"Consecutive TCP small segments exceeding threshold",TCP,10.0.2.2,56642,10.0.2.15,22,52:54:00:12:35:02,08:00:27:7F:93:2D,0x4E,***AP***,0x9AFF3D7,0xC8761D52,,0xFFFF,64,0,59677,64,65536,,,,
+02/22-15:56:48.612494 ,129,12,1,"Consecutive TCP small segments exceeding threshold",TCP,96.44.142.5,80,10.0.2.15,50895,52:54:00:12:35:02,08:00:27:7F:93:2D,0x6E,***AP***,0xDB45F7A,0x7701DD5B,,0xFFFF,64,0,16785,96,98304,,,,
+02/22-15:56:48.616775 ,129,12,1,"Consecutive TCP small segments exceeding threshold",TCP,96.44.142.5,80,10.0.2.15,50895,52:54:00:12:35:02,08:00:27:7F:93:2D,0xA6,***AP***,0xDB508F2,0x7701DD5B,,0xFFFF,64,0,16824,152,155648,,,,
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/df8d682e/metron-platform/metron-integration-test/src/main/sample/data/squid/parsed/SquidExampleParsed
----------------------------------------------------------------------
diff --git a/metron-platform/metron-integration-test/src/main/sample/data/squid/parsed/SquidExampleParsed b/metron-platform/metron-integration-test/src/main/sample/data/squid/parsed/SquidExampleParsed
new file mode 100644
index 0000000..9643c25
--- /dev/null
+++ b/metron-platform/metron-integration-test/src/main/sample/data/squid/parsed/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/df8d682e/metron-platform/metron-integration-test/src/main/sample/data/squid/raw/SquidExampleOutput
----------------------------------------------------------------------
diff --git a/metron-platform/metron-integration-test/src/main/sample/data/squid/raw/SquidExampleOutput b/metron-platform/metron-integration-test/src/main/sample/data/squid/raw/SquidExampleOutput
new file mode 100644
index 0000000..ae70fb9
--- /dev/null
+++ b/metron-platform/metron-integration-test/src/main/sample/data/squid/raw/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

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/df8d682e/metron-platform/metron-integration-test/src/main/sample/data/test/indexed/TestIndexed
----------------------------------------------------------------------
diff --git a/metron-platform/metron-integration-test/src/main/sample/data/test/indexed/TestIndexed b/metron-platform/metron-integration-test/src/main/sample/data/test/indexed/TestIndexed
new file mode 100644
index 0000000..0dfbe43
--- /dev/null
+++ b/metron-platform/metron-integration-test/src/main/sample/data/test/indexed/TestIndexed
@@ -0,0 +1,10 @@
+{"adapter.threatinteladapter.end.ts":"1457102731219","enrichments.geo.dip.location_point":"test longitude,test latitude","isn":"22efa001","index.elasticsearchwriter.ts":"1457102731220","dip":"10.0.2.15","dp":39468,"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","enrichments.geo.dip.locID":"1","enrichments.geo.sip.city":"test city","enrichmentjoinbolt.joiner.ts":"1457102731206","adapter.hostfromjsonlistadapter.begin.ts":"1457102731185","tag":0,"enrichments.geo.dip.dmaCode":"test dmaCode","app":0,"oct":44,"end_reason":"idle","enrichments.geo.sip.locID":"1","adapter.mockgeoadapter.begin.ts":"1457102731185","threatintelsplitterbolt.splitter.ts":"1457102731207","enrichments.geo.dip.postalCode":"test postalCode","start_time":1453994988512,
 "adapter.threatinteladapter.begin.ts":"1457102731210","riflags":0,"proto":6,"enrichments.host.dip.known_info.local":"YES","enrichments.geo.dip.longitude":"test longitude","iflags":"AS","uflags":0,"adapter.mockgeoadapter.end.ts":"1457102731198","adapter.hostfromjsonlistadapter.end.ts":"1457102731197","enrichments.geo.sip.postalCode":"test postalCode","duration":"0.000","enrichments.geo.dip.country":"test country","threatinteljoinbolt.joiner.ts":"1457102731220","enrichments.geo.dip.latitude":"test latitude","enrichments.geo.sip.country":"test country","enrichments.geo.dip.city":"test city","enrichments.geo.sip.dmaCode":"test dmaCode","pkt":1,"enrichments.geo.sip.location_point":"test longitude,test latitude","ruflags":0,"roct":0,"sip":"216.21.170.221","rtag":0,"sp":80,"enrichments.geo.sip.longitude":"test longitude","enrichments.geo.sip.latitude":"test latitude","timestamp":1453994988512,"risn":0,"enrichments.host.dip.known_info.type":"printer","end_time":1453994988512,"enrichments.ho
 st.dip.known_info.asset_value":"important","source.type":"test","rtt":"0.000"}
+{"adapter.threatinteladapter.end.ts":"1457102731221","enrichments.geo.dip.location_point":"test longitude,test latitude","enrichments.host.sip.known_info.asset_value":"important","isn":10000000,"index.elasticsearchwriter.ts":"1457102731221","dip":"10.0.2.3","dp":53,"rpkt":0,"original_string":"2016-01-28 15:29:48.502|2016-01-28 15:29:48.502|   0.000|   0.000| 17|                               10.0.2.15|37299|                                10.0.2.3|   53|       A|       0|       0|       0|10000000|00000000|000|000|       1|      56|       0|       0|    0|idle","enrichments.geo.dip.locID":"1","enrichments.geo.sip.city":"test city","enrichments.host.sip.known_info.type":"printer","enrichmentjoinbolt.joiner.ts":"1457102731208","adapter.hostfromjsonlistadapter.begin.ts":"1457102731197","tag":0,"enrichments.geo.dip.dmaCode":"test dmaCode","app":0,"oct":56,"end_reason":"idle","enrichments.geo.sip.locID":"1","adapter.mockgeoadapter.begin.ts":"1457102731198","threatintelsplitterbolt.splitt
 er.ts":"1457102731210","enrichments.geo.dip.postalCode":"test postalCode","start_time":1453994988502,"adapter.threatinteladapter.begin.ts":"1457102731219","riflags":0,"proto":17,"enrichments.geo.dip.longitude":"test longitude","iflags":"A","uflags":0,"adapter.mockgeoadapter.end.ts":"1457102731198","adapter.hostfromjsonlistadapter.end.ts":"1457102731197","enrichments.host.sip.known_info.local":"YES","threatintels.ip.dip.ip_threat_intel":"alert","enrichments.geo.sip.postalCode":"test postalCode","duration":"0.000","enrichments.geo.dip.country":"test country","threatinteljoinbolt.joiner.ts":"1457102731221","enrichments.geo.dip.latitude":"test latitude","enrichments.geo.sip.country":"test country","enrichments.geo.dip.city":"test city","enrichments.geo.sip.dmaCode":"test dmaCode","pkt":1,"enrichments.geo.sip.location_point":"test longitude,test latitude","ruflags":0,"roct":0,"sip":"10.0.2.15","rtag":0,"sp":37299,"enrichments.geo.sip.longitude":"test longitude","enrichments.geo.sip.latit
 ude":"test latitude","timestamp":1453994988502,"risn":0,"end_time":1453994988502,"is_alert":"true","source.type":"test","rtt":"0.000"}
+{"adapter.threatinteladapter.end.ts":"1457102731221","enrichments.geo.dip.location_point":"test longitude,test latitude","isn":0,"index.elasticsearchwriter.ts":"1457102731222","dip":"10.0.2.15","dp":37299,"rpkt":0,"original_string":"2016-01-28 15:29:48.504|2016-01-28 15:29:48.504|   0.000|   0.000| 17|                                10.0.2.3|   53|                               10.0.2.15|37299|       A|       0|       0|       0|00000000|00000000|000|000|       1|     312|       0|       0|    0|idle","enrichments.geo.dip.locID":"1","enrichments.geo.sip.city":"test city","enrichmentjoinbolt.joiner.ts":"1457102731209","adapter.hostfromjsonlistadapter.begin.ts":"1457102731197","tag":0,"enrichments.geo.dip.dmaCode":"test dmaCode","app":0,"oct":312,"end_reason":"idle","enrichments.geo.sip.locID":"1","adapter.mockgeoadapter.begin.ts":"1457102731198","threatintelsplitterbolt.splitter.ts":"1457102731210","enrichments.geo.dip.postalCode":"test postalCode","start_time":1453994988504,"adapter
 .threatinteladapter.begin.ts":"1457102731221","riflags":0,"proto":17,"enrichments.host.dip.known_info.local":"YES","enrichments.geo.dip.longitude":"test longitude","iflags":"A","uflags":0,"adapter.mockgeoadapter.end.ts":"1457102731199","adapter.hostfromjsonlistadapter.end.ts":"1457102731198","enrichments.geo.sip.postalCode":"test postalCode","duration":"0.000","enrichments.geo.dip.country":"test country","threatinteljoinbolt.joiner.ts":"1457102731222","enrichments.geo.dip.latitude":"test latitude","enrichments.geo.sip.country":"test country","enrichments.geo.dip.city":"test city","enrichments.geo.sip.dmaCode":"test dmaCode","pkt":1,"enrichments.geo.sip.location_point":"test longitude,test latitude","ruflags":0,"roct":0,"sip":"10.0.2.3","rtag":0,"sp":53,"enrichments.geo.sip.longitude":"test longitude","enrichments.geo.sip.latitude":"test latitude","timestamp":1453994988504,"risn":0,"enrichments.host.dip.known_info.type":"printer","end_time":1453994988504,"enrichments.host.dip.known_i
 nfo.asset_value":"important","is_alert":"true","source.type":"test","threatintels.ip.sip.ip_threat_intel":"alert","rtt":"0.000"}
+{"adapter.threatinteladapter.end.ts":"1457102731222","enrichments.geo.dip.location_point":"test longitude,test latitude","enrichments.host.sip.known_info.asset_value":"important","isn":0,"index.elasticsearchwriter.ts":"1457102731222","dip":"10.0.2.3","dp":53,"rpkt":0,"original_string":"2016-01-28 15:29:48.504|2016-01-28 15:29:48.504|   0.000|   0.000| 17|                               10.0.2.15|56303|                                10.0.2.3|   53|       A|       0|       0|       0|00000000|00000000|000|000|       1|      56|       0|       0|    0|idle","enrichments.geo.dip.locID":"1","enrichments.geo.sip.city":"test city","enrichments.host.sip.known_info.type":"printer","enrichmentjoinbolt.joiner.ts":"1457102731209","adapter.hostfromjsonlistadapter.begin.ts":"1457102731198","tag":0,"enrichments.geo.dip.dmaCode":"test dmaCode","app":0,"oct":56,"end_reason":"idle","enrichments.geo.sip.locID":"1","adapter.mockgeoadapter.begin.ts":"1457102731199","threatintelsplitterbolt.splitter.ts":
 "1457102731211","enrichments.geo.dip.postalCode":"test postalCode","start_time":1453994988504,"adapter.threatinteladapter.begin.ts":"1457102731221","riflags":0,"proto":17,"enrichments.geo.dip.longitude":"test longitude","iflags":"A","uflags":0,"adapter.mockgeoadapter.end.ts":"1457102731199","adapter.hostfromjsonlistadapter.end.ts":"1457102731198","enrichments.host.sip.known_info.local":"YES","threatintels.ip.dip.ip_threat_intel":"alert","enrichments.geo.sip.postalCode":"test postalCode","duration":"0.000","enrichments.geo.dip.country":"test country","threatinteljoinbolt.joiner.ts":"1457102731222","enrichments.geo.dip.latitude":"test latitude","enrichments.geo.sip.country":"test country","enrichments.geo.dip.city":"test city","enrichments.geo.sip.dmaCode":"test dmaCode","pkt":1,"enrichments.geo.sip.location_point":"test longitude,test latitude","ruflags":0,"roct":0,"sip":"10.0.2.15","rtag":0,"sp":56303,"enrichments.geo.sip.longitude":"test longitude","enrichments.geo.sip.latitude":"t
 est latitude","timestamp":1453994988504,"risn":0,"end_time":1453994988504,"is_alert":"true","source.type":"test","rtt":"0.000"}
+{"adapter.threatinteladapter.end.ts":"1457102731222","enrichments.geo.dip.location_point":"test longitude,test latitude","isn":0,"index.elasticsearchwriter.ts":"1457102731222","dip":"10.0.2.15","dp":56303,"rpkt":0,"original_string":"2016-01-28 15:29:48.506|2016-01-28 15:29:48.506|   0.000|   0.000| 17|                                10.0.2.3|   53|                               10.0.2.15|56303|       A|       0|       0|       0|00000000|00000000|000|000|       1|      84|       0|       0|    0|idle","enrichments.geo.dip.locID":"1","enrichments.geo.sip.city":"test city","enrichmentjoinbolt.joiner.ts":"1457102731210","adapter.hostfromjsonlistadapter.begin.ts":"1457102731198","tag":0,"enrichments.geo.dip.dmaCode":"test dmaCode","app":0,"oct":84,"end_reason":"idle","enrichments.geo.sip.locID":"1","adapter.mockgeoadapter.begin.ts":"1457102731199","threatintelsplitterbolt.splitter.ts":"1457102731212","enrichments.geo.dip.postalCode":"test postalCode","start_time":1453994988506,"adapter.
 threatinteladapter.begin.ts":"1457102731222","riflags":0,"proto":17,"enrichments.host.dip.known_info.local":"YES","enrichments.geo.dip.longitude":"test longitude","iflags":"A","uflags":0,"adapter.mockgeoadapter.end.ts":"1457102731199","adapter.hostfromjsonlistadapter.end.ts":"1457102731198","enrichments.geo.sip.postalCode":"test postalCode","duration":"0.000","enrichments.geo.dip.country":"test country","threatinteljoinbolt.joiner.ts":"1457102731222","enrichments.geo.dip.latitude":"test latitude","enrichments.geo.sip.country":"test country","enrichments.geo.dip.city":"test city","enrichments.geo.sip.dmaCode":"test dmaCode","pkt":1,"enrichments.geo.sip.location_point":"test longitude,test latitude","ruflags":0,"roct":0,"sip":"10.0.2.3","rtag":0,"sp":53,"enrichments.geo.sip.longitude":"test longitude","enrichments.geo.sip.latitude":"test latitude","timestamp":1453994988506,"risn":0,"enrichments.host.dip.known_info.type":"printer","end_time":1453994988506,"enrichments.host.dip.known_in
 fo.asset_value":"important","is_alert":"true","source.type":"test","threatintels.ip.sip.ip_threat_intel":"alert","rtt":"0.000"}
+{"adapter.threatinteladapter.end.ts":"1457102731222","enrichments.geo.dip.location_point":"test longitude,test latitude","enrichments.host.sip.known_info.asset_value":"important","isn":"58c52fca","index.elasticsearchwriter.ts":"1457102732038","dip":"216.21.170.221","dp":80,"rpkt":0,"original_string":"2016-01-28 15:29:48.508|2016-01-28 15:29:48.508|   0.000|   0.000|  6|                               10.0.2.15|39468|                          216.21.170.221|   80|       S|       0|       0|       0|58c52fca|00000000|000|000|       1|      60|       0|       0|    0|idle","enrichments.geo.dip.locID":"1","enrichments.geo.sip.city":"test city","enrichments.host.sip.known_info.type":"printer","enrichmentjoinbolt.joiner.ts":"1457102731210","adapter.hostfromjsonlistadapter.begin.ts":"1457102731198","tag":0,"enrichments.geo.dip.dmaCode":"test dmaCode","app":0,"oct":60,"end_reason":"idle","enrichments.geo.sip.locID":"1","adapter.mockgeoadapter.begin.ts":"1457102731199","threatintelsplitterbol
 t.splitter.ts":"1457102731212","enrichments.geo.dip.postalCode":"test postalCode","start_time":1453994988508,"adapter.threatinteladapter.begin.ts":"1457102731222","riflags":0,"proto":6,"enrichments.geo.dip.longitude":"test longitude","iflags":"S","uflags":0,"adapter.mockgeoadapter.end.ts":"1457102731199","adapter.hostfromjsonlistadapter.end.ts":"1457102731198","enrichments.host.sip.known_info.local":"YES","enrichments.geo.sip.postalCode":"test postalCode","duration":"0.000","enrichments.geo.dip.country":"test country","threatinteljoinbolt.joiner.ts":"1457102731223","enrichments.geo.dip.latitude":"test latitude","enrichments.geo.sip.country":"test country","enrichments.geo.dip.city":"test city","enrichments.geo.sip.dmaCode":"test dmaCode","pkt":1,"enrichments.geo.sip.location_point":"test longitude,test latitude","ruflags":0,"roct":0,"sip":"10.0.2.15","rtag":0,"sp":39468,"enrichments.geo.sip.longitude":"test longitude","enrichments.geo.sip.latitude":"test latitude","timestamp":145399
 4988508,"risn":0,"end_time":1453994988508,"source.type":"test","rtt":"0.000"}
+{"adapter.threatinteladapter.end.ts":"1457102731223","enrichments.geo.dip.location_point":"test longitude,test latitude","enrichments.host.sip.known_info.asset_value":"important","isn":"58c52fcb","index.elasticsearchwriter.ts":"1457102732038","dip":"216.21.170.221","dp":80,"rpkt":0,"original_string":"2016-01-28 15:29:48.512|2016-01-28 15:29:48.512|   0.000|   0.000|  6|                               10.0.2.15|39468|                          216.21.170.221|   80|       A|       0|       0|       0|58c52fcb|00000000|000|000|       1|      40|       0|       0|    0|idle ","enrichments.geo.dip.locID":"1","enrichments.geo.sip.city":"test city","enrichments.host.sip.known_info.type":"printer","enrichmentjoinbolt.joiner.ts":"1457102731210","adapter.hostfromjsonlistadapter.begin.ts":"1457102731198","tag":0,"enrichments.geo.dip.dmaCode":"test dmaCode","app":0,"oct":40,"end_reason":"idle ","enrichments.geo.sip.locID":"1","adapter.mockgeoadapter.begin.ts":"1457102731199","threatintelsplitterb
 olt.splitter.ts":"1457102731212","enrichments.geo.dip.postalCode":"test postalCode","start_time":1453994988512,"adapter.threatinteladapter.begin.ts":"1457102731223","riflags":0,"proto":6,"enrichments.geo.dip.longitude":"test longitude","iflags":"A","uflags":0,"adapter.mockgeoadapter.end.ts":"1457102731199","adapter.hostfromjsonlistadapter.end.ts":"1457102731198","enrichments.host.sip.known_info.local":"YES","enrichments.geo.sip.postalCode":"test postalCode","duration":"0.000","enrichments.geo.dip.country":"test country","threatinteljoinbolt.joiner.ts":"1457102731223","enrichments.geo.dip.latitude":"test latitude","enrichments.geo.sip.country":"test country","enrichments.geo.dip.city":"test city","enrichments.geo.sip.dmaCode":"test dmaCode","pkt":1,"enrichments.geo.sip.location_point":"test longitude,test latitude","ruflags":0,"roct":0,"sip":"10.0.2.15","rtag":0,"sp":39468,"enrichments.geo.sip.longitude":"test longitude","enrichments.geo.sip.latitude":"test latitude","timestamp":1453
 994988512,"risn":0,"end_time":1453994988512,"source.type":"test","rtt":"0.000"}
+{"adapter.threatinteladapter.end.ts":"1457102731223","enrichments.geo.dip.location_point":"test longitude,test latitude","enrichments.host.sip.known_info.asset_value":"important","isn":"58c52fcb","index.elasticsearchwriter.ts":"1457102732038","dip":"216.21.170.221","dp":80,"rpkt":0,"original_string":"2016-01-28 15:29:48.512|2016-01-28 15:29:48.512|   0.000|   0.000|  6|                               10.0.2.15|39468|                          216.21.170.221|   80|      AP|       0|       0|       0|58c52fcb|00000000|000|000|       1|     148|       0|       0|    0|idle ","enrichments.geo.dip.locID":"1","enrichments.geo.sip.city":"test city","enrichments.host.sip.known_info.type":"printer","enrichmentjoinbolt.joiner.ts":"1457102731210","adapter.hostfromjsonlistadapter.begin.ts":"1457102731198","tag":0,"enrichments.geo.dip.dmaCode":"test dmaCode","app":0,"oct":148,"end_reason":"idle ","enrichments.geo.sip.locID":"1","adapter.mockgeoadapter.begin.ts":"1457102731199","threatintelsplitter
 bolt.splitter.ts":"1457102731212","enrichments.geo.dip.postalCode":"test postalCode","start_time":1453994988512,"adapter.threatinteladapter.begin.ts":"1457102731223","riflags":0,"proto":6,"enrichments.geo.dip.longitude":"test longitude","iflags":"AP","uflags":0,"adapter.mockgeoadapter.end.ts":"1457102731199","adapter.hostfromjsonlistadapter.end.ts":"1457102731198","enrichments.host.sip.known_info.local":"YES","enrichments.geo.sip.postalCode":"test postalCode","duration":"0.000","enrichments.geo.dip.country":"test country","threatinteljoinbolt.joiner.ts":"1457102731225","enrichments.geo.dip.latitude":"test latitude","enrichments.geo.sip.country":"test country","enrichments.geo.dip.city":"test city","enrichments.geo.sip.dmaCode":"test dmaCode","pkt":1,"enrichments.geo.sip.location_point":"test longitude,test latitude","ruflags":0,"roct":0,"sip":"10.0.2.15","rtag":0,"sp":39468,"enrichments.geo.sip.longitude":"test longitude","enrichments.geo.sip.latitude":"test latitude","timestamp":14
 53994988512,"risn":0,"end_time":1453994988512,"source.type":"test","rtt":"0.000"}
+{"adapter.threatinteladapter.end.ts":"1457102731225","enrichments.geo.dip.location_point":"test longitude,test latitude","isn":"22efa002","index.elasticsearchwriter.ts":"1457102732038","dip":"10.0.2.15","dp":39468,"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|       A|       0|       0|       0|22efa002|00000000|000|000|       1|      40|       0|       0|    0|idle ","enrichments.geo.dip.locID":"1","enrichments.geo.sip.city":"test city","enrichmentjoinbolt.joiner.ts":"1457102731211","adapter.hostfromjsonlistadapter.begin.ts":"1457102731198","tag":0,"enrichments.geo.dip.dmaCode":"test dmaCode","app":0,"oct":40,"end_reason":"idle ","enrichments.geo.sip.locID":"1","adapter.mockgeoadapter.begin.ts":"1457102731199","threatintelsplitterbolt.splitter.ts":"1457102731212","enrichments.geo.dip.postalCode":"test postalCode","start_time":145399498851
 2,"adapter.threatinteladapter.begin.ts":"1457102731223","riflags":0,"proto":6,"enrichments.host.dip.known_info.local":"YES","enrichments.geo.dip.longitude":"test longitude","iflags":"A","uflags":0,"adapter.mockgeoadapter.end.ts":"1457102731199","adapter.hostfromjsonlistadapter.end.ts":"1457102731198","enrichments.geo.sip.postalCode":"test postalCode","duration":"0.000","enrichments.geo.dip.country":"test country","threatinteljoinbolt.joiner.ts":"1457102731225","enrichments.geo.dip.latitude":"test latitude","enrichments.geo.sip.country":"test country","enrichments.geo.dip.city":"test city","enrichments.geo.sip.dmaCode":"test dmaCode","pkt":1,"enrichments.geo.sip.location_point":"test longitude,test latitude","ruflags":0,"roct":0,"sip":"216.21.170.221","rtag":0,"sp":80,"enrichments.geo.sip.longitude":"test longitude","enrichments.geo.sip.latitude":"test latitude","timestamp":1453994988512,"risn":0,"enrichments.host.dip.known_info.type":"printer","end_time":1453994988512,"enrichments.h
 ost.dip.known_info.asset_value":"important","source.type":"test","rtt":"0.000"}
+{"adapter.threatinteladapter.end.ts":"1457102731226","enrichments.geo.dip.location_point":"test longitude,test latitude","isn":"22efa002","index.elasticsearchwriter.ts":"1457102732038","dip":"10.0.2.15","dp":39468,"rpkt":0,"original_string":"2016-01-28 15:29:48.562|2016-01-28 15:29:48.562|   0.000|   0.000|  6|                          216.21.170.221|   80|                               10.0.2.15|39468|      AP|       0|       0|       0|22efa002|00000000|000|000|       1|     604|       0|       0|    0|idle","enrichments.geo.dip.locID":"1","enrichments.geo.sip.city":"test city","enrichmentjoinbolt.joiner.ts":"1457102731211","adapter.hostfromjsonlistadapter.begin.ts":"1457102731198","tag":0,"enrichments.geo.dip.dmaCode":"test dmaCode","app":0,"oct":604,"end_reason":"idle","enrichments.geo.sip.locID":"1","adapter.mockgeoadapter.begin.ts":"1457102731199","threatintelsplitterbolt.splitter.ts":"1457102731213","enrichments.geo.dip.postalCode":"test postalCode","start_time":1453994988562
 ,"adapter.threatinteladapter.begin.ts":"1457102731226","riflags":0,"proto":6,"enrichments.host.dip.known_info.local":"YES","enrichments.geo.dip.longitude":"test longitude","iflags":"AP","uflags":0,"adapter.mockgeoadapter.end.ts":"1457102731199","adapter.hostfromjsonlistadapter.end.ts":"1457102731198","enrichments.geo.sip.postalCode":"test postalCode","duration":"0.000","enrichments.geo.dip.country":"test country","threatinteljoinbolt.joiner.ts":"1457102731226","enrichments.geo.dip.latitude":"test latitude","enrichments.geo.sip.country":"test country","enrichments.geo.dip.city":"test city","enrichments.geo.sip.dmaCode":"test dmaCode","pkt":1,"enrichments.geo.sip.location_point":"test longitude,test latitude","ruflags":0,"roct":0,"sip":"216.21.170.221","rtag":0,"sp":80,"enrichments.geo.sip.longitude":"test longitude","enrichments.geo.sip.latitude":"test latitude","timestamp":1453994988562,"risn":0,"enrichments.host.dip.known_info.type":"printer","end_time":1453994988562,"enrichments.h
 ost.dip.known_info.asset_value":"important","source.type":"test","rtt":"0.000"}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/df8d682e/metron-platform/metron-integration-test/src/main/sample/data/test/parsed/TestExampleParsed
----------------------------------------------------------------------
diff --git a/metron-platform/metron-integration-test/src/main/sample/data/test/parsed/TestExampleParsed b/metron-platform/metron-integration-test/src/main/sample/data/test/parsed/TestExampleParsed
new file mode 100644
index 0000000..bbb90f3
--- /dev/null
+++ b/metron-platform/metron-integration-test/src/main/sample/data/test/parsed/TestExampleParsed
@@ -0,0 +1,10 @@
+{"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,"source.type":"test","start_time":1453994988512,"riflags":0,"rtt":"0.000","protocol":6}
+{"iflags":"A","uflags":0,"isn":10000000,"ip_dst_addr":"10.0.2.3","ip_dst_port":53,"duration":"0.000","rpkt":0,"original_string":"2016-01-28 15:29:48.502|2016-01-28 15:29:48.502|   0.000|   0.000| 17|                               10.0.2.15|37299|                                10.0.2.3|   53|       A|       0|       0|       0|10000000|00000000|000|000|       1|      56|       0|       0|    0|idle","pkt":1,"ruflags":0,"roct":0,"ip_src_addr":"10.0.2.15","tag":0,"rtag":0,"ip_src_port":37299,"timestamp":1453994988502,"app":0,"oct":56,"end_reason":"idle","risn":0,"end_time":1453994988502,"source.type":"test","start_time":1453994988502,"riflags":0,"rtt":"0.000","protocol":17}
+{"iflags":"A","uflags":0,"isn":0,"ip_dst_addr":"10.0.2.15","ip_dst_port":37299,"duration":"0.000","rpkt":0,"original_string":"2016-01-28 15:29:48.504|2016-01-28 15:29:48.504|   0.000|   0.000| 17|                                10.0.2.3|   53|                               10.0.2.15|37299|       A|       0|       0|       0|00000000|00000000|000|000|       1|     312|       0|       0|    0|idle","pkt":1,"ruflags":0,"roct":0,"ip_src_addr":"10.0.2.3","tag":0,"rtag":0,"ip_src_port":53,"timestamp":1453994988504,"app":0,"oct":312,"end_reason":"idle","risn":0,"end_time":1453994988504,"source.type":"test","start_time":1453994988504,"riflags":0,"rtt":"0.000","protocol":17}
+{"iflags":"A","uflags":0,"isn":0,"ip_dst_addr":"10.0.2.3","ip_dst_port":53,"duration":"0.000","rpkt":0,"original_string":"2016-01-28 15:29:48.504|2016-01-28 15:29:48.504|   0.000|   0.000| 17|                               10.0.2.15|56303|                                10.0.2.3|   53|       A|       0|       0|       0|00000000|00000000|000|000|       1|      56|       0|       0|    0|idle","pkt":1,"ruflags":0,"roct":0,"ip_src_addr":"10.0.2.15","tag":0,"rtag":0,"ip_src_port":56303,"timestamp":1453994988504,"app":0,"oct":56,"end_reason":"idle","risn":0,"end_time":1453994988504,"source.type":"test","start_time":1453994988504,"riflags":0,"rtt":"0.000","protocol":17}
+{"iflags":"A","uflags":0,"isn":0,"ip_dst_addr":"10.0.2.15","ip_dst_port":56303,"duration":"0.000","rpkt":0,"original_string":"2016-01-28 15:29:48.506|2016-01-28 15:29:48.506|   0.000|   0.000| 17|                                10.0.2.3|   53|                               10.0.2.15|56303|       A|       0|       0|       0|00000000|00000000|000|000|       1|      84|       0|       0|    0|idle","pkt":1,"ruflags":0,"roct":0,"ip_src_addr":"10.0.2.3","tag":0,"rtag":0,"ip_src_port":53,"timestamp":1453994988506,"app":0,"oct":84,"end_reason":"idle","risn":0,"end_time":1453994988506,"source.type":"test","start_time":1453994988506,"riflags":0,"rtt":"0.000","protocol":17}
+{"iflags":"S","uflags":0,"isn":"58c52fca","ip_dst_addr":"216.21.170.221","ip_dst_port":80,"duration":"0.000","rpkt":0,"original_string":"2016-01-28 15:29:48.508|2016-01-28 15:29:48.508|   0.000|   0.000|  6|                               10.0.2.15|39468|                          216.21.170.221|   80|       S|       0|       0|       0|58c52fca|00000000|000|000|       1|      60|       0|       0|    0|idle","pkt":1,"ruflags":0,"roct":0,"ip_src_addr":"10.0.2.15","tag":0,"rtag":0,"ip_src_port":39468,"timestamp":1453994988508,"app":0,"oct":60,"end_reason":"idle","risn":0,"end_time":1453994988508,"source.type":"test","start_time":1453994988508,"riflags":0,"rtt":"0.000","protocol":6}
+{"iflags":"A","uflags":0,"isn":"58c52fcb","ip_dst_addr":"216.21.170.221","ip_dst_port":80,"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|                               10.0.2.15|39468|                          216.21.170.221|   80|       A|       0|       0|       0|58c52fcb|00000000|000|000|       1|      40|       0|       0|    0|idle ","pkt":1,"ruflags":0,"roct":0,"ip_src_addr":"10.0.2.15","tag":0,"rtag":0,"ip_src_port":39468,"timestamp":1453994988512,"app":0,"oct":40,"end_reason":"idle ","risn":0,"end_time":1453994988512,"source.type":"test","start_time":1453994988512,"riflags":0,"rtt":"0.000","protocol":6}
+{"iflags":"AP","uflags":0,"isn":"58c52fcb","ip_dst_addr":"216.21.170.221","ip_dst_port":80,"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|                               10.0.2.15|39468|                          216.21.170.221|   80|      AP|       0|       0|       0|58c52fcb|00000000|000|000|       1|     148|       0|       0|    0|idle ","pkt":1,"ruflags":0,"roct":0,"ip_src_addr":"10.0.2.15","tag":0,"rtag":0,"ip_src_port":39468,"timestamp":1453994988512,"app":0,"oct":148,"end_reason":"idle ","risn":0,"end_time":1453994988512,"source.type":"test","start_time":1453994988512,"riflags":0,"rtt":"0.000","protocol":6}
+{"iflags":"A","uflags":0,"isn":"22efa002","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|       A|       0|       0|       0|22efa002|00000000|000|000|       1|      40|       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":40,"end_reason":"idle ","risn":0,"end_time":1453994988512,"source.type":"test","start_time":1453994988512,"riflags":0,"rtt":"0.000","protocol":6}
+{"iflags":"AP","uflags":0,"isn":"22efa002","ip_dst_addr":"10.0.2.15","ip_dst_port":39468,"duration":"0.000","rpkt":0,"original_string":"2016-01-28 15:29:48.562|2016-01-28 15:29:48.562|   0.000|   0.000|  6|                          216.21.170.221|   80|                               10.0.2.15|39468|      AP|       0|       0|       0|22efa002|00000000|000|000|       1|     604|       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":1453994988562,"app":0,"oct":604,"end_reason":"idle","risn":0,"end_time":1453994988562,"source.type":"test","start_time":1453994988562,"riflags":0,"rtt":"0.000","protocol":6}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/df8d682e/metron-platform/metron-integration-test/src/main/sample/data/websphere/parsed/WebsphereParsed
----------------------------------------------------------------------
diff --git a/metron-platform/metron-integration-test/src/main/sample/data/websphere/parsed/WebsphereParsed b/metron-platform/metron-integration-test/src/main/sample/data/websphere/parsed/WebsphereParsed
new file mode 100644
index 0000000..86e3895
--- /dev/null
+++ b/metron-platform/metron-integration-test/src/main/sample/data/websphere/parsed/WebsphereParsed
@@ -0,0 +1,5 @@
+{"severity":"notice","hostname":"ABCXML1413","event_type":"auth","original_string":"<133>Apr 15 17:47:28 ABCXML1413 [rojOut][0x81000033][auth][notice] user(rick007): [120.43.200.6]: User logged into 'cohlOut'.","event_code":"0x81000033","security_domain":"rojOut","event_subtype":"login","priority":133,"ip_src_addr":"120.43.200.6","timestamp":1460742448000,"username":"rick007","source.type":"websphere"}
+{"severity":"info","hostname":"PHIXML3RWD","event_type":"auth","original_string":"<134>Apr 15 18:02:27 PHIXML3RWD [0x81000019][auth][info] [14.122.2.201]: User 'hjpotter' logged out from 'default'.","event_code":"0x81000019","security_domain":"default","event_subtype":"logout","priority":134,"ip_src_addr":"14.122.2.201","timestamp":1460743347000,"username":"hjpotter","source.type":"websphere"}
+{"severity":"error","hostname":"ROBXML3QRS","process":"rbm","event_type":"auth","original_string":"<131>Apr 15 17:36:35 ROBXML3QRS [0x80800018][auth][error] rbm(RBM-Settings): trans(3502888135)[request] gtid(3502888135): RBM: Resource access denied.","event_code":"0x80800018","message":"trans(3502888135)[request] gtid(3502888135): RBM: Resource access denied.","priority":131,"timestamp":1460741795000,"source.type":"websphere"}
+{"severity":"info","hostname":"SAGPXMLQA333","process":"trans","event_type":"audit","original_string":"<134>Apr 15 17:17:34 SAGPXMLQA333 [0x8240001c][audit][info] trans(191): (admin:default:system:*): ntp-service 'NTP Service' - Operational state down","event_code":"0x8240001c","message":"(admin:default:system:*): ntp-service 'NTP Service' - Operational state down","priority":134,"timestamp":1460740654000,"source.type":"websphere"}
+{"severity":"info","hostname":"DOMXML3PUZ","event_type":"auth","original_string":"<134>Apr 15 17:46:52 DOMXML3PUZ [0x8100448e][auth][info] CLI timeout occurred.","event_code":"0x8100448e","message":"CLI timeout occurred.","priority":134,"timestamp":1460742412000,"source.type":"websphere"}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/df8d682e/metron-platform/metron-integration-test/src/main/sample/data/websphere/raw/WebsphereOutput.txt
----------------------------------------------------------------------
diff --git a/metron-platform/metron-integration-test/src/main/sample/data/websphere/raw/WebsphereOutput.txt b/metron-platform/metron-integration-test/src/main/sample/data/websphere/raw/WebsphereOutput.txt
new file mode 100644
index 0000000..9003548
--- /dev/null
+++ b/metron-platform/metron-integration-test/src/main/sample/data/websphere/raw/WebsphereOutput.txt
@@ -0,0 +1,5 @@
+<133>Apr 15 17:47:28 ABCXML1413 [rojOut][0x81000033][auth][notice] user(rick007): [120.43.200.6]: User logged into 'cohlOut'.
+<134>Apr 15 18:02:27 PHIXML3RWD [0x81000019][auth][info] [14.122.2.201]: User 'hjpotter' logged out from 'default'.
+<131>Apr 15 17:36:35 ROBXML3QRS [0x80800018][auth][error] rbm(RBM-Settings): trans(3502888135)[request] gtid(3502888135): RBM: Resource access denied.
+<134>Apr 15 17:17:34 SAGPXMLQA333 [0x8240001c][audit][info] trans(191): (admin:default:system:*): ntp-service 'NTP Service' - Operational state down
+<134>Apr 15 17:46:52 DOMXML3PUZ [0x8100448e][auth][info] CLI timeout occurred.
\ No newline at end of file


[12/15] incubator-metron git commit: METRON-142 Simplify Parser configuration (merrimanr via cestella) closes apache/incubator-metron#120

Posted by ce...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/df8d682e/metron-platform/metron-integration-test/src/main/resources/sample/data/SampleInput/BroExampleOutput
----------------------------------------------------------------------
diff --git a/metron-platform/metron-integration-test/src/main/resources/sample/data/SampleInput/BroExampleOutput b/metron-platform/metron-integration-test/src/main/resources/sample/data/SampleInput/BroExampleOutput
deleted file mode 100644
index 6ded8a8..0000000
--- a/metron-platform/metron-integration-test/src/main/resources/sample/data/SampleInput/BroExampleOutput
+++ /dev/null
@@ -1,23411 +0,0 @@
-{"http":{"ts":1402307733473,"uid":"CTo78A11g7CYbbOHvj","id.orig_h":"192.249.113.37","id.orig_p":58808,"id.resp_h":"72.163.4.161","id.resp_p":80,"trans_depth":1,"method":"GET","host":"www.cisco.com","uri":"/","user_agent":"curl/7.22.0 (x86_64-pc-linux-gnu) libcurl/7.22.0 OpenSSL/1.0.1 zlib/1.2.3.4 libidn/1.23 librtmp/2.3","request_body_len":0,"response_body_len":25523,"status_code":200,"status_msg":"OK","tags":[],"resp_fuids":["FJDyMC15lxUn5ngPfd"],"resp_mime_types":["text/html"]}}
-{"dns":{"ts":1402308259609,"uid":"CuJT272SKaJSuqO0Ia","id.orig_h":"10.122.196.204","id.orig_p":33976,"id.resp_h":"144.254.71.184","id.resp_p":53,"proto":"udp","trans_id":62418,"query":"www.cisco.com","qclass":1,"qclass_name":"C_INTERNET","qtype":28,"qtype_name":"AAAA","rcode":0,"rcode_name":"NOERROR","AA":true,"TC":false,"RD":true,"RA":true,"Z":0,"answers":["www.cisco.com.akadns.net","origin-www.cisco.com","2001:420:1201:2::a"],"TTLs":[3600.0,289.0,14.0],"rejected":false}}
-{"http":{"ts":1402307733473,"uid":"KIRAN","id.orig_h":"10.122.196.204","id.orig_p":58808,"id.resp_h":"72.163.4.161","id.resp_p":80,"trans_depth":1,"method":"GET","host":"www.cisco.com","uri":"/","user_agent":"curl/7.22.0 (x86_64-pc-linux-gnu) libcurl/7.22.0 OpenSSL/1.0.1 zlib/1.2.3.4 libidn/1.23 librtmp/2.3","request_body_len":0,"response_body_len":25523,"status_code":200,"status_msg":"OK","tags":[],"resp_fuids":["FJDyMC15lxUn5ngPfd"],"resp_mime_types":["text/html"]}}
-{"http":{"ts":1402307733473,"uid":"KIRAN12312312","id.orig_h":"192.249.113.37","id.orig_p":58808,"id.resp_h":"72.163.4.161","id.resp_p":80,"trans_depth":1,"method":"GET","host":"www.cisco.com","uri":"/","user_agent":"curl/7.22.0 (x86_64-pc-linux-gnu) libcurl/7.22.0 OpenSSL/1.0.1 zlib/1.2.3.4 libidn/1.23 librtmp/2.3","request_body_len":0,"response_body_len":25523,"status_code":200,"status_msg":"OK","tags":[],"resp_fuids":["FJDyMC15lxUn5ngPfd"],"resp_mime_types":["text/html"]}}
-{"http":{"ts":1402307733473,"uid":"KIRAN12312312","id.orig_h":"192.249.113.37","id.orig_p":58808,"id.resp_h":"72.163.4.161","id.resp_p":80,"trans_depth":1,"method":"GET","host":"www.cisco.com","uri":"/","user_agent":"curl/7.22.0 (x86_64-pc-linux-gnu) libcurl/7.22.0 OpenSSL/1.0.1 zlib/1.2.3.4 libidn/1.23 librtmp/2.3","request_body_len":0,"response_body_len":25523,"status_code":200,"status_msg":"OK","tags":[],"resp_fuids":["FJDyMC15lxUn5ngPfd"],"resp_mime_types":["text/html"]}}
-{"http":{"ts":1402307733473,"uid":"CTo78A11g7CYbbOHvj","id.orig_h":"10.122.196.204","id.orig_p":58808,"id.resp_h":"72.163.4.161","id.resp_p":80,"trans_depth":1,"email":"abullis@mail.csuchico.edu","method":"GET","host":"gabacentre.pw","uri":"/","user_agent":"curl/7.22.0 (x86_64-pc-linux-gnu) libcurl/7.22.0 OpenSSL/1.0.1 zlib/1.2.3.4 libidn/1.23 librtmp/2.3","request_body_len":0,"response_body_len":25523,"status_code":200,"status_msg":"OK","tags":[],"resp_fuids":["FJDyMC15lxUn5ngPfd"],"resp_mime_types":["text/html"]}}
-{"dns":{"ts":1402308259609,"uid":"CYbbOHvj","id.orig_h":"93.188.160.43","id.orig_p":33976,"id.resp_h":"144.254.71.184","id.resp_p":53,"proto":"udp","trans_id":62418,"query":"www.cisco.com","qclass":1,"qclass_name":"C_INTERNET","qtype":28,"qtype_name":"AAAA","rcode":0,"rcode_name":"NOERROR","AA":true,"TC":false,"RD":true,"RA":true,"Z":0,"answers":["gabacentre.pw","www.cisco.com.akadns.net","origin-www.cisco.com","2001:420:1201:2::a"],"TTLs":[3600.0,289.0,14.0],"rejected":false}}
-{"http":{"ts":1402307733473,"uid":"CTo78A11g7CYbbOHvj","id.orig_h":"192.249.113.37","id.orig_p":58808,"id.resp_h":"72.163.4.161","id.resp_p":80,"trans_depth":1,"method":"GET","host":"www.cisco.com","uri":"/","user_agent":"curl/7.22.0 (x86_64-pc-linux-gnu) libcurl/7.22.0 OpenSSL/1.0.1 zlib/1.2.3.4 libidn/1.23 librtmp/2.3","request_body_len":0,"response_body_len":25523,"status_code":200,"status_msg":"OK","tags":[],"resp_fuids":["FJDyMC15lxUn5ngPfd"],"resp_mime_types":["text/html"]}}
-{"dns":{"ts":1402308259609,"uid":"CuJT272SKaJSuqO0Ia","id.orig_h":"10.122.196.204","id.orig_p":33976,"id.resp_h":"144.254.71.184","id.resp_p":53,"proto":"udp","trans_id":62418,"query":"www.cisco.com","qclass":1,"qclass_name":"C_INTERNET","qtype":28,"qtype_name":"AAAA","rcode":0,"rcode_name":"NOERROR","AA":true,"TC":false,"RD":true,"RA":true,"Z":0,"answers":["www.cisco.com.akadns.net","origin-www.cisco.com","2001:420:1201:2::a"],"TTLs":[3600.0,289.0,14.0],"rejected":false}}
-{"http":{"ts":1402307733473,"uid":"KIRAN","id.orig_h":"10.122.196.204","id.orig_p":58808,"id.resp_h":"72.163.4.161","id.resp_p":80,"trans_depth":1,"method":"GET","host":"www.cisco.com","uri":"/","user_agent":"curl/7.22.0 (x86_64-pc-linux-gnu) libcurl/7.22.0 OpenSSL/1.0.1 zlib/1.2.3.4 libidn/1.23 librtmp/2.3","request_body_len":0,"response_body_len":25523,"status_code":200,"status_msg":"OK","tags":[],"resp_fuids":["FJDyMC15lxUn5ngPfd"],"resp_mime_types":["text/html"]}}
-{"http":{"ts":1402307733473,"uid":"KIRAN12312312","id.orig_h":"192.249.113.37","id.orig_p":58808,"id.resp_h":"72.163.4.161","id.resp_p":80,"trans_depth":1,"method":"GET","host":"www.cisco.com","uri":"/","user_agent":"curl/7.22.0 (x86_64-pc-linux-gnu) libcurl/7.22.0 OpenSSL/1.0.1 zlib/1.2.3.4 libidn/1.23 librtmp/2.3","request_body_len":0,"response_body_len":25523,"status_code":200,"status_msg":"OK","tags":[],"resp_fuids":["FJDyMC15lxUn5ngPfd"],"resp_mime_types":["text/html"]}}
-{"http":{"ts":1402307733473,"uid":"KIRAN12312312","id.orig_h":"192.249.113.37","id.orig_p":58808,"id.resp_h":"72.163.4.161","id.resp_p":80,"trans_depth":1,"method":"GET","host":"www.cisco.com","uri":"/","user_agent":"curl/7.22.0 (x86_64-pc-linux-gnu) libcurl/7.22.0 OpenSSL/1.0.1 zlib/1.2.3.4 libidn/1.23 librtmp/2.3","request_body_len":0,"response_body_len":25523,"status_code":200,"status_msg":"OK","tags":[],"resp_fuids":["FJDyMC15lxUn5ngPfd"],"resp_mime_types":["text/html"]}}
-{"http":{"ts":1402307733473,"uid":"CTo78A11g7CYbbOHvj","id.orig_h":"10.122.196.204","id.orig_p":58808,"id.resp_h":"72.163.4.161","id.resp_p":80,"trans_depth":1,"email":"abullis@mail.csuchico.edu","method":"GET","host":"gabacentre.pw","uri":"/","user_agent":"curl/7.22.0 (x86_64-pc-linux-gnu) libcurl/7.22.0 OpenSSL/1.0.1 zlib/1.2.3.4 libidn/1.23 librtmp/2.3","request_body_len":0,"response_body_len":25523,"status_code":200,"status_msg":"OK","tags":[],"resp_fuids":["FJDyMC15lxUn5ngPfd"],"resp_mime_types":["text/html"]}}
-{"dns":{"ts":1402308259609,"uid":"CYbbOHvj","id.orig_h":"93.188.160.43","id.orig_p":33976,"id.resp_h":"144.254.71.184","id.resp_p":53,"proto":"udp","trans_id":62418,"query":"www.cisco.com","qclass":1,"qclass_name":"C_INTERNET","qtype":28,"qtype_name":"AAAA","rcode":0,"rcode_name":"NOERROR","AA":true,"TC":false,"RD":true,"RA":true,"Z":0,"answers":["gabacentre.pw","www.cisco.com.akadns.net","origin-www.cisco.com","2001:420:1201:2::a"],"TTLs":[3600.0,289.0,14.0],"rejected":false}}
-{"http":{"ts":1402307733473,"uid":"CTo78A11g7CYbbOHvj","id.orig_h":"192.249.113.37","id.orig_p":58808,"id.resp_h":"72.163.4.161","id.resp_p":80,"trans_depth":1,"method":"GET","host":"www.cisco.com","uri":"/","user_agent":"curl/7.22.0 (x86_64-pc-linux-gnu) libcurl/7.22.0 OpenSSL/1.0.1 zlib/1.2.3.4 libidn/1.23 librtmp/2.3","request_body_len":0,"response_body_len":25523,"status_code":200,"status_msg":"OK","tags":[],"resp_fuids":["FJDyMC15lxUn5ngPfd"],"resp_mime_types":["text/html"]}}
-{"dns":{"ts":1402308259609,"uid":"CuJT272SKaJSuqO0Ia","id.orig_h":"10.122.196.204","id.orig_p":33976,"id.resp_h":"144.254.71.184","id.resp_p":53,"proto":"udp","trans_id":62418,"query":"www.cisco.com","qclass":1,"qclass_name":"C_INTERNET","qtype":28,"qtype_name":"AAAA","rcode":0,"rcode_name":"NOERROR","AA":true,"TC":false,"RD":true,"RA":true,"Z":0,"answers":["www.cisco.com.akadns.net","origin-www.cisco.com","2001:420:1201:2::a"],"TTLs":[3600.0,289.0,14.0],"rejected":false}}
-{"http":{"ts":1402307733473,"uid":"KIRAN","id.orig_h":"10.122.196.204","id.orig_p":58808,"id.resp_h":"72.163.4.161","id.resp_p":80,"trans_depth":1,"method":"GET","host":"www.cisco.com","uri":"/","user_agent":"curl/7.22.0 (x86_64-pc-linux-gnu) libcurl/7.22.0 OpenSSL/1.0.1 zlib/1.2.3.4 libidn/1.23 librtmp/2.3","request_body_len":0,"response_body_len":25523,"status_code":200,"status_msg":"OK","tags":[],"resp_fuids":["FJDyMC15lxUn5ngPfd"],"resp_mime_types":["text/html"]}}
-{"http":{"ts":1402307733473,"uid":"KIRAN12312312","id.orig_h":"192.249.113.37","id.orig_p":58808,"id.resp_h":"72.163.4.161","id.resp_p":80,"trans_depth":1,"method":"GET","host":"www.cisco.com","uri":"/","user_agent":"curl/7.22.0 (x86_64-pc-linux-gnu) libcurl/7.22.0 OpenSSL/1.0.1 zlib/1.2.3.4 libidn/1.23 librtmp/2.3","request_body_len":0,"response_body_len":25523,"status_code":200,"status_msg":"OK","tags":[],"resp_fuids":["FJDyMC15lxUn5ngPfd"],"resp_mime_types":["text/html"]}}
-{"http":{"ts":1402307733473,"uid":"KIRAN12312312","id.orig_h":"192.249.113.37","id.orig_p":58808,"id.resp_h":"72.163.4.161","id.resp_p":80,"trans_depth":1,"method":"GET","host":"www.cisco.com","uri":"/","user_agent":"curl/7.22.0 (x86_64-pc-linux-gnu) libcurl/7.22.0 OpenSSL/1.0.1 zlib/1.2.3.4 libidn/1.23 librtmp/2.3","request_body_len":0,"response_body_len":25523,"status_code":200,"status_msg":"OK","tags":[],"resp_fuids":["FJDyMC15lxUn5ngPfd"],"resp_mime_types":["text/html"]}}
-{"http":{"ts":1402307733473,"uid":"CTo78A11g7CYbbOHvj","id.orig_h":"10.122.196.204","id.orig_p":58808,"id.resp_h":"72.163.4.161","id.resp_p":80,"trans_depth":1,"email":"abullis@mail.csuchico.edu","method":"GET","host":"gabacentre.pw","uri":"/","user_agent":"curl/7.22.0 (x86_64-pc-linux-gnu) libcurl/7.22.0 OpenSSL/1.0.1 zlib/1.2.3.4 libidn/1.23 librtmp/2.3","request_body_len":0,"response_body_len":25523,"status_code":200,"status_msg":"OK","tags":[],"resp_fuids":["FJDyMC15lxUn5ngPfd"],"resp_mime_types":["text/html"]}}
-{"dns":{"ts":1402308259609,"uid":"CYbbOHvj","id.orig_h":"93.188.160.43","id.orig_p":33976,"id.resp_h":"144.254.71.184","id.resp_p":53,"proto":"udp","trans_id":62418,"query":"www.cisco.com","qclass":1,"qclass_name":"C_INTERNET","qtype":28,"qtype_name":"AAAA","rcode":0,"rcode_name":"NOERROR","AA":true,"TC":false,"RD":true,"RA":true,"Z":0,"answers":["gabacentre.pw","www.cisco.com.akadns.net","origin-www.cisco.com","2001:420:1201:2::a"],"TTLs":[3600.0,289.0,14.0],"rejected":false}}
-{"http":{"ts":1402307733473,"uid":"CTo78A11g7CYbbOHvj","id.orig_h":"192.249.113.37","id.orig_p":58808,"id.resp_h":"72.163.4.161","id.resp_p":80,"trans_depth":1,"method":"GET","host":"www.cisco.com","uri":"/","user_agent":"curl/7.22.0 (x86_64-pc-linux-gnu) libcurl/7.22.0 OpenSSL/1.0.1 zlib/1.2.3.4 libidn/1.23 librtmp/2.3","request_body_len":0,"response_body_len":25523,"status_code":200,"status_msg":"OK","tags":[],"resp_fuids":["FJDyMC15lxUn5ngPfd"],"resp_mime_types":["text/html"]}}
-{"dns":{"ts":1402308259609,"uid":"CuJT272SKaJSuqO0Ia","id.orig_h":"10.122.196.204","id.orig_p":33976,"id.resp_h":"144.254.71.184","id.resp_p":53,"proto":"udp","trans_id":62418,"query":"www.cisco.com","qclass":1,"qclass_name":"C_INTERNET","qtype":28,"qtype_name":"AAAA","rcode":0,"rcode_name":"NOERROR","AA":true,"TC":false,"RD":true,"RA":true,"Z":0,"answers":["www.cisco.com.akadns.net","origin-www.cisco.com","2001:420:1201:2::a"],"TTLs":[3600.0,289.0,14.0],"rejected":false}}
-{"http":{"ts":1402307733473,"uid":"KIRAN","id.orig_h":"10.122.196.204","id.orig_p":58808,"id.resp_h":"72.163.4.161","id.resp_p":80,"trans_depth":1,"method":"GET","host":"www.cisco.com","uri":"/","user_agent":"curl/7.22.0 (x86_64-pc-linux-gnu) libcurl/7.22.0 OpenSSL/1.0.1 zlib/1.2.3.4 libidn/1.23 librtmp/2.3","request_body_len":0,"response_body_len":25523,"status_code":200,"status_msg":"OK","tags":[],"resp_fuids":["FJDyMC15lxUn5ngPfd"],"resp_mime_types":["text/html"]}}
-{"http":{"ts":1402307733473,"uid":"KIRAN12312312","id.orig_h":"192.249.113.37","id.orig_p":58808,"id.resp_h":"72.163.4.161","id.resp_p":80,"trans_depth":1,"method":"GET","host":"www.cisco.com","uri":"/","user_agent":"curl/7.22.0 (x86_64-pc-linux-gnu) libcurl/7.22.0 OpenSSL/1.0.1 zlib/1.2.3.4 libidn/1.23 librtmp/2.3","request_body_len":0,"response_body_len":25523,"status_code":200,"status_msg":"OK","tags":[],"resp_fuids":["FJDyMC15lxUn5ngPfd"],"resp_mime_types":["text/html"]}}
-{"http":{"ts":1402307733473,"uid":"KIRAN12312312","id.orig_h":"192.249.113.37","id.orig_p":58808,"id.resp_h":"72.163.4.161","id.resp_p":80,"trans_depth":1,"method":"GET","host":"www.cisco.com","uri":"/","user_agent":"curl/7.22.0 (x86_64-pc-linux-gnu) libcurl/7.22.0 OpenSSL/1.0.1 zlib/1.2.3.4 libidn/1.23 librtmp/2.3","request_body_len":0,"response_body_len":25523,"status_code":200,"status_msg":"OK","tags":[],"resp_fuids":["FJDyMC15lxUn5ngPfd"],"resp_mime_types":["text/html"]}}
-{"http":{"ts":1402307733473,"uid":"CTo78A11g7CYbbOHvj","id.orig_h":"10.122.196.204","id.orig_p":58808,"id.resp_h":"72.163.4.161","id.resp_p":80,"trans_depth":1,"email":"abullis@mail.csuchico.edu","method":"GET","host":"gabacentre.pw","uri":"/","user_agent":"curl/7.22.0 (x86_64-pc-linux-gnu) libcurl/7.22.0 OpenSSL/1.0.1 zlib/1.2.3.4 libidn/1.23 librtmp/2.3","request_body_len":0,"response_body_len":25523,"status_code":200,"status_msg":"OK","tags":[],"resp_fuids":["FJDyMC15lxUn5ngPfd"],"resp_mime_types":["text/html"]}}
-{"dns":{"ts":1402308259609,"uid":"CYbbOHvj","id.orig_h":"93.188.160.43","id.orig_p":33976,"id.resp_h":"144.254.71.184","id.resp_p":53,"proto":"udp","trans_id":62418,"query":"www.cisco.com","qclass":1,"qclass_name":"C_INTERNET","qtype":28,"qtype_name":"AAAA","rcode":0,"rcode_name":"NOERROR","AA":true,"TC":false,"RD":true,"RA":true,"Z":0,"answers":["gabacentre.pw","www.cisco.com.akadns.net","origin-www.cisco.com","2001:420:1201:2::a"],"TTLs":[3600.0,289.0,14.0],"rejected":false}}
-{"http":{"ts":1402307733473,"uid":"CTo78A11g7CYbbOHvj","id.orig_h":"192.249.113.37","id.orig_p":58808,"id.resp_h":"72.163.4.161","id.resp_p":80,"trans_depth":1,"method":"GET","host":"www.cisco.com","uri":"/","user_agent":"curl/7.22.0 (x86_64-pc-linux-gnu) libcurl/7.22.0 OpenSSL/1.0.1 zlib/1.2.3.4 libidn/1.23 librtmp/2.3","request_body_len":0,"response_body_len":25523,"status_code":200,"status_msg":"OK","tags":[],"resp_fuids":["FJDyMC15lxUn5ngPfd"],"resp_mime_types":["text/html"]}}
-{"dns":{"ts":1402308259609,"uid":"CuJT272SKaJSuqO0Ia","id.orig_h":"10.122.196.204","id.orig_p":33976,"id.resp_h":"144.254.71.184","id.resp_p":53,"proto":"udp","trans_id":62418,"query":"www.cisco.com","qclass":1,"qclass_name":"C_INTERNET","qtype":28,"qtype_name":"AAAA","rcode":0,"rcode_name":"NOERROR","AA":true,"TC":false,"RD":true,"RA":true,"Z":0,"answers":["www.cisco.com.akadns.net","origin-www.cisco.com","2001:420:1201:2::a"],"TTLs":[3600.0,289.0,14.0],"rejected":false}}
-{"http":{"ts":1402307733473,"uid":"KIRAN","id.orig_h":"10.122.196.204","id.orig_p":58808,"id.resp_h":"72.163.4.161","id.resp_p":80,"trans_depth":1,"method":"GET","host":"www.cisco.com","uri":"/","user_agent":"curl/7.22.0 (x86_64-pc-linux-gnu) libcurl/7.22.0 OpenSSL/1.0.1 zlib/1.2.3.4 libidn/1.23 librtmp/2.3","request_body_len":0,"response_body_len":25523,"status_code":200,"status_msg":"OK","tags":[],"resp_fuids":["FJDyMC15lxUn5ngPfd"],"resp_mime_types":["text/html"]}}
-{"http":{"ts":1402307733473,"uid":"KIRAN12312312","id.orig_h":"192.249.113.37","id.orig_p":58808,"id.resp_h":"72.163.4.161","id.resp_p":80,"trans_depth":1,"method":"GET","host":"www.cisco.com","uri":"/","user_agent":"curl/7.22.0 (x86_64-pc-linux-gnu) libcurl/7.22.0 OpenSSL/1.0.1 zlib/1.2.3.4 libidn/1.23 librtmp/2.3","request_body_len":0,"response_body_len":25523,"status_code":200,"status_msg":"OK","tags":[],"resp_fuids":["FJDyMC15lxUn5ngPfd"],"resp_mime_types":["text/html"]}}
-{"http":{"ts":1402307733473,"uid":"KIRAN12312312","id.orig_h":"192.249.113.37","id.orig_p":58808,"id.resp_h":"72.163.4.161","id.resp_p":80,"trans_depth":1,"method":"GET","host":"www.cisco.com","uri":"/","user_agent":"curl/7.22.0 (x86_64-pc-linux-gnu) libcurl/7.22.0 OpenSSL/1.0.1 zlib/1.2.3.4 libidn/1.23 librtmp/2.3","request_body_len":0,"response_body_len":25523,"status_code":200,"status_msg":"OK","tags":[],"resp_fuids":["FJDyMC15lxUn5ngPfd"],"resp_mime_types":["text/html"]}}
-{"http":{"ts":1402307733473,"uid":"CTo78A11g7CYbbOHvj","id.orig_h":"10.122.196.204","id.orig_p":58808,"id.resp_h":"72.163.4.161","id.resp_p":80,"trans_depth":1,"email":"abullis@mail.csuchico.edu","method":"GET","host":"gabacentre.pw","uri":"/","user_agent":"curl/7.22.0 (x86_64-pc-linux-gnu) libcurl/7.22.0 OpenSSL/1.0.1 zlib/1.2.3.4 libidn/1.23 librtmp/2.3","request_body_len":0,"response_body_len":25523,"status_code":200,"status_msg":"OK","tags":[],"resp_fuids":["FJDyMC15lxUn5ngPfd"],"resp_mime_types":["text/html"]}}
-{"dns":{"ts":1402308259609,"uid":"CYbbOHvj","id.orig_h":"93.188.160.43","id.orig_p":33976,"id.resp_h":"144.254.71.184","id.resp_p":53,"proto":"udp","trans_id":62418,"query":"www.cisco.com","qclass":1,"qclass_name":"C_INTERNET","qtype":28,"qtype_name":"AAAA","rcode":0,"rcode_name":"NOERROR","AA":true,"TC":false,"RD":true,"RA":true,"Z":0,"answers":["gabacentre.pw","www.cisco.com.akadns.net","origin-www.cisco.com","2001:420:1201:2::a"],"TTLs":[3600.0,289.0,14.0],"rejected":false}}
-{"http":{"ts":1402307733473,"uid":"CTo78A11g7CYbbOHvj","id.orig_h":"192.249.113.37","id.orig_p":58808,"id.resp_h":"72.163.4.161","id.resp_p":80,"trans_depth":1,"method":"GET","host":"www.cisco.com","uri":"/","user_agent":"curl/7.22.0 (x86_64-pc-linux-gnu) libcurl/7.22.0 OpenSSL/1.0.1 zlib/1.2.3.4 libidn/1.23 librtmp/2.3","request_body_len":0,"response_body_len":25523,"status_code":200,"status_msg":"OK","tags":[],"resp_fuids":["FJDyMC15lxUn5ngPfd"],"resp_mime_types":["text/html"]}}
-{"dns":{"ts":1402308259609,"uid":"CuJT272SKaJSuqO0Ia","id.orig_h":"10.122.196.204","id.orig_p":33976,"id.resp_h":"144.254.71.184","id.resp_p":53,"proto":"udp","trans_id":62418,"query":"www.cisco.com","qclass":1,"qclass_name":"C_INTERNET","qtype":28,"qtype_name":"AAAA","rcode":0,"rcode_name":"NOERROR","AA":true,"TC":false,"RD":true,"RA":true,"Z":0,"answers":["www.cisco.com.akadns.net","origin-www.cisco.com","2001:420:1201:2::a"],"TTLs":[3600.0,289.0,14.0],"rejected":false}}
-{"http":{"ts":1402307733473,"uid":"KIRAN","id.orig_h":"10.122.196.204","id.orig_p":58808,"id.resp_h":"72.163.4.161","id.resp_p":80,"trans_depth":1,"method":"GET","host":"www.cisco.com","uri":"/","user_agent":"curl/7.22.0 (x86_64-pc-linux-gnu) libcurl/7.22.0 OpenSSL/1.0.1 zlib/1.2.3.4 libidn/1.23 librtmp/2.3","request_body_len":0,"response_body_len":25523,"status_code":200,"status_msg":"OK","tags":[],"resp_fuids":["FJDyMC15lxUn5ngPfd"],"resp_mime_types":["text/html"]}}
-{"http":{"ts":1402307733473,"uid":"KIRAN12312312","id.orig_h":"192.249.113.37","id.orig_p":58808,"id.resp_h":"72.163.4.161","id.resp_p":80,"trans_depth":1,"method":"GET","host":"www.cisco.com","uri":"/","user_agent":"curl/7.22.0 (x86_64-pc-linux-gnu) libcurl/7.22.0 OpenSSL/1.0.1 zlib/1.2.3.4 libidn/1.23 librtmp/2.3","request_body_len":0,"response_body_len":25523,"status_code":200,"status_msg":"OK","tags":[],"resp_fuids":["FJDyMC15lxUn5ngPfd"],"resp_mime_types":["text/html"]}}
-{"http":{"ts":1402307733473,"uid":"KIRAN12312312","id.orig_h":"192.249.113.37","id.orig_p":58808,"id.resp_h":"72.163.4.161","id.resp_p":80,"trans_depth":1,"method":"GET","host":"www.cisco.com","uri":"/","user_agent":"curl/7.22.0 (x86_64-pc-linux-gnu) libcurl/7.22.0 OpenSSL/1.0.1 zlib/1.2.3.4 libidn/1.23 librtmp/2.3","request_body_len":0,"response_body_len":25523,"status_code":200,"status_msg":"OK","tags":[],"resp_fuids":["FJDyMC15lxUn5ngPfd"],"resp_mime_types":["text/html"]}}
-{"http":{"ts":1402307733473,"uid":"CTo78A11g7CYbbOHvj","id.orig_h":"10.122.196.204","id.orig_p":58808,"id.resp_h":"72.163.4.161","id.resp_p":80,"trans_depth":1,"email":"abullis@mail.csuchico.edu","method":"GET","host":"gabacentre.pw","uri":"/","user_agent":"curl/7.22.0 (x86_64-pc-linux-gnu) libcurl/7.22.0 OpenSSL/1.0.1 zlib/1.2.3.4 libidn/1.23 librtmp/2.3","request_body_len":0,"response_body_len":25523,"status_code":200,"status_msg":"OK","tags":[],"resp_fuids":["FJDyMC15lxUn5ngPfd"],"resp_mime_types":["text/html"]}}
-{"dns":{"ts":1402308259609,"uid":"CYbbOHvj","id.orig_h":"93.188.160.43","id.orig_p":33976,"id.resp_h":"144.254.71.184","id.resp_p":53,"proto":"udp","trans_id":62418,"query":"www.cisco.com","qclass":1,"qclass_name":"C_INTERNET","qtype":28,"qtype_name":"AAAA","rcode":0,"rcode_name":"NOERROR","AA":true,"TC":false,"RD":true,"RA":true,"Z":0,"answers":["gabacentre.pw","www.cisco.com.akadns.net","origin-www.cisco.com","2001:420:1201:2::a"],"TTLs":[3600.0,289.0,14.0],"rejected":false}}
-{"http":{"ts":1402307733473,"uid":"CTo78A11g7CYbbOHvj","id.orig_h":"192.249.113.37","id.orig_p":58808,"id.resp_h":"72.163.4.161","id.resp_p":80,"trans_depth":1,"method":"GET","host":"www.cisco.com","uri":"/","user_agent":"curl/7.22.0 (x86_64-pc-linux-gnu) libcurl/7.22.0 OpenSSL/1.0.1 zlib/1.2.3.4 libidn/1.23 librtmp/2.3","request_body_len":0,"response_body_len":25523,"status_code":200,"status_msg":"OK","tags":[],"resp_fuids":["FJDyMC15lxUn5ngPfd"],"resp_mime_types":["text/html"]}}
-{"dns":{"ts":1402308259609,"uid":"CuJT272SKaJSuqO0Ia","id.orig_h":"10.122.196.204","id.orig_p":33976,"id.resp_h":"144.254.71.184","id.resp_p":53,"proto":"udp","trans_id":62418,"query":"www.cisco.com","qclass":1,"qclass_name":"C_INTERNET","qtype":28,"qtype_name":"AAAA","rcode":0,"rcode_name":"NOERROR","AA":true,"TC":false,"RD":true,"RA":true,"Z":0,"answers":["www.cisco.com.akadns.net","origin-www.cisco.com","2001:420:1201:2::a"],"TTLs":[3600.0,289.0,14.0],"rejected":false}}
-{"http":{"ts":1402307733473,"uid":"KIRAN","id.orig_h":"10.122.196.204","id.orig_p":58808,"id.resp_h":"72.163.4.161","id.resp_p":80,"trans_depth":1,"method":"GET","host":"www.cisco.com","uri":"/","user_agent":"curl/7.22.0 (x86_64-pc-linux-gnu) libcurl/7.22.0 OpenSSL/1.0.1 zlib/1.2.3.4 libidn/1.23 librtmp/2.3","request_body_len":0,"response_body_len":25523,"status_code":200,"status_msg":"OK","tags":[],"resp_fuids":["FJDyMC15lxUn5ngPfd"],"resp_mime_types":["text/html"]}}
-{"http":{"ts":1402307733473,"uid":"KIRAN12312312","id.orig_h":"192.249.113.37","id.orig_p":58808,"id.resp_h":"72.163.4.161","id.resp_p":80,"trans_depth":1,"method":"GET","host":"www.cisco.com","uri":"/","user_agent":"curl/7.22.0 (x86_64-pc-linux-gnu) libcurl/7.22.0 OpenSSL/1.0.1 zlib/1.2.3.4 libidn/1.23 librtmp/2.3","request_body_len":0,"response_body_len":25523,"status_code":200,"status_msg":"OK","tags":[],"resp_fuids":["FJDyMC15lxUn5ngPfd"],"resp_mime_types":["text/html"]}}
-{"http":{"ts":1402307733473,"uid":"KIRAN12312312","id.orig_h":"192.249.113.37","id.orig_p":58808,"id.resp_h":"72.163.4.161","id.resp_p":80,"trans_depth":1,"method":"GET","host":"www.cisco.com","uri":"/","user_agent":"curl/7.22.0 (x86_64-pc-linux-gnu) libcurl/7.22.0 OpenSSL/1.0.1 zlib/1.2.3.4 libidn/1.23 librtmp/2.3","request_body_len":0,"response_body_len":25523,"status_code":200,"status_msg":"OK","tags":[],"resp_fuids":["FJDyMC15lxUn5ngPfd"],"resp_mime_types":["text/html"]}}
-{"http":{"ts":1402307733473,"uid":"CTo78A11g7CYbbOHvj","id.orig_h":"10.122.196.204","id.orig_p":58808,"id.resp_h":"72.163.4.161","id.resp_p":80,"trans_depth":1,"email":"abullis@mail.csuchico.edu","method":"GET","host":"gabacentre.pw","uri":"/","user_agent":"curl/7.22.0 (x86_64-pc-linux-gnu) libcurl/7.22.0 OpenSSL/1.0.1 zlib/1.2.3.4 libidn/1.23 librtmp/2.3","request_body_len":0,"response_body_len":25523,"status_code":200,"status_msg":"OK","tags":[],"resp_fuids":["FJDyMC15lxUn5ngPfd"],"resp_mime_types":["text/html"]}}
-{"dns":{"ts":1402308259609,"uid":"CYbbOHvj","id.orig_h":"93.188.160.43","id.orig_p":33976,"id.resp_h":"144.254.71.184","id.resp_p":53,"proto":"udp","trans_id":62418,"query":"www.cisco.com","qclass":1,"qclass_name":"C_INTERNET","qtype":28,"qtype_name":"AAAA","rcode":0,"rcode_name":"NOERROR","AA":true,"TC":false,"RD":true,"RA":true,"Z":0,"answers":["gabacentre.pw","www.cisco.com.akadns.net","origin-www.cisco.com","2001:420:1201:2::a"],"TTLs":[3600.0,289.0,14.0],"rejected":false}}
-{"http":{"ts":1402307733473,"uid":"CTo78A11g7CYbbOHvj","id.orig_h":"192.249.113.37","id.orig_p":58808,"id.resp_h":"72.163.4.161","id.resp_p":80,"trans_depth":1,"method":"GET","host":"www.cisco.com","uri":"/","user_agent":"curl/7.22.0 (x86_64-pc-linux-gnu) libcurl/7.22.0 OpenSSL/1.0.1 zlib/1.2.3.4 libidn/1.23 librtmp/2.3","request_body_len":0,"response_body_len":25523,"status_code":200,"status_msg":"OK","tags":[],"resp_fuids":["FJDyMC15lxUn5ngPfd"],"resp_mime_types":["text/html"]}}
-{"dns":{"ts":1402308259609,"uid":"CuJT272SKaJSuqO0Ia","id.orig_h":"10.122.196.204","id.orig_p":33976,"id.resp_h":"144.254.71.184","id.resp_p":53,"proto":"udp","trans_id":62418,"query":"www.cisco.com","qclass":1,"qclass_name":"C_INTERNET","qtype":28,"qtype_name":"AAAA","rcode":0,"rcode_name":"NOERROR","AA":true,"TC":false,"RD":true,"RA":true,"Z":0,"answers":["www.cisco.com.akadns.net","origin-www.cisco.com","2001:420:1201:2::a"],"TTLs":[3600.0,289.0,14.0],"rejected":false}}
-{"http":{"ts":1402307733473,"uid":"KIRAN","id.orig_h":"10.122.196.204","id.orig_p":58808,"id.resp_h":"72.163.4.161","id.resp_p":80,"trans_depth":1,"method":"GET","host":"www.cisco.com","uri":"/","user_agent":"curl/7.22.0 (x86_64-pc-linux-gnu) libcurl/7.22.0 OpenSSL/1.0.1 zlib/1.2.3.4 libidn/1.23 librtmp/2.3","request_body_len":0,"response_body_len":25523,"status_code":200,"status_msg":"OK","tags":[],"resp_fuids":["FJDyMC15lxUn5ngPfd"],"resp_mime_types":["text/html"]}}
-{"http":{"ts":1402307733473,"uid":"KIRAN12312312","id.orig_h":"192.249.113.37","id.orig_p":58808,"id.resp_h":"72.163.4.161","id.resp_p":80,"trans_depth":1,"method":"GET","host":"www.cisco.com","uri":"/","user_agent":"curl/7.22.0 (x86_64-pc-linux-gnu) libcurl/7.22.0 OpenSSL/1.0.1 zlib/1.2.3.4 libidn/1.23 librtmp/2.3","request_body_len":0,"response_body_len":25523,"status_code":200,"status_msg":"OK","tags":[],"resp_fuids":["FJDyMC15lxUn5ngPfd"],"resp_mime_types":["text/html"]}}
-{"http":{"ts":1402307733473,"uid":"KIRAN12312312","id.orig_h":"192.249.113.37","id.orig_p":58808,"id.resp_h":"72.163.4.161","id.resp_p":80,"trans_depth":1,"method":"GET","host":"www.cisco.com","uri":"/","user_agent":"curl/7.22.0 (x86_64-pc-linux-gnu) libcurl/7.22.0 OpenSSL/1.0.1 zlib/1.2.3.4 libidn/1.23 librtmp/2.3","request_body_len":0,"response_body_len":25523,"status_code":200,"status_msg":"OK","tags":[],"resp_fuids":["FJDyMC15lxUn5ngPfd"],"resp_mime_types":["text/html"]}}
-{"http":{"ts":1402307733473,"uid":"CTo78A11g7CYbbOHvj","id.orig_h":"10.122.196.204","id.orig_p":58808,"id.resp_h":"72.163.4.161","id.resp_p":80,"trans_depth":1,"email":"abullis@mail.csuchico.edu","method":"GET","host":"gabacentre.pw","uri":"/","user_agent":"curl/7.22.0 (x86_64-pc-linux-gnu) libcurl/7.22.0 OpenSSL/1.0.1 zlib/1.2.3.4 libidn/1.23 librtmp/2.3","request_body_len":0,"response_body_len":25523,"status_code":200,"status_msg":"OK","tags":[],"resp_fuids":["FJDyMC15lxUn5ngPfd"],"resp_mime_types":["text/html"]}}
-{"dns":{"ts":1402308259609,"uid":"CYbbOHvj","id.orig_h":"93.188.160.43","id.orig_p":33976,"id.resp_h":"144.254.71.184","id.resp_p":53,"proto":"udp","trans_id":62418,"query":"www.cisco.com","qclass":1,"qclass_name":"C_INTERNET","qtype":28,"qtype_name":"AAAA","rcode":0,"rcode_name":"NOERROR","AA":true,"TC":false,"RD":true,"RA":true,"Z":0,"answers":["gabacentre.pw","www.cisco.com.akadns.net","origin-www.cisco.com","2001:420:1201:2::a"],"TTLs":[3600.0,289.0,14.0],"rejected":false}}
-{"http":{"ts":1402307733473,"uid":"CTo78A11g7CYbbOHvj","id.orig_h":"192.249.113.37","id.orig_p":58808,"id.resp_h":"72.163.4.161","id.resp_p":80,"trans_depth":1,"method":"GET","host":"www.cisco.com","uri":"/","user_agent":"curl/7.22.0 (x86_64-pc-linux-gnu) libcurl/7.22.0 OpenSSL/1.0.1 zlib/1.2.3.4 libidn/1.23 librtmp/2.3","request_body_len":0,"response_body_len":25523,"status_code":200,"status_msg":"OK","tags":[],"resp_fuids":["FJDyMC15lxUn5ngPfd"],"resp_mime_types":["text/html"]}}
-{"dns":{"ts":1402308259609,"uid":"CuJT272SKaJSuqO0Ia","id.orig_h":"10.122.196.204","id.orig_p":33976,"id.resp_h":"144.254.71.184","id.resp_p":53,"proto":"udp","trans_id":62418,"query":"www.cisco.com","qclass":1,"qclass_name":"C_INTERNET","qtype":28,"qtype_name":"AAAA","rcode":0,"rcode_name":"NOERROR","AA":true,"TC":false,"RD":true,"RA":true,"Z":0,"answers":["www.cisco.com.akadns.net","origin-www.cisco.com","2001:420:1201:2::a"],"TTLs":[3600.0,289.0,14.0],"rejected":false}}
-{"http":{"ts":1402307733473,"uid":"KIRAN","id.orig_h":"10.122.196.204","id.orig_p":58808,"id.resp_h":"72.163.4.161","id.resp_p":80,"trans_depth":1,"method":"GET","host":"www.cisco.com","uri":"/","user_agent":"curl/7.22.0 (x86_64-pc-linux-gnu) libcurl/7.22.0 OpenSSL/1.0.1 zlib/1.2.3.4 libidn/1.23 librtmp/2.3","request_body_len":0,"response_body_len":25523,"status_code":200,"status_msg":"OK","tags":[],"resp_fuids":["FJDyMC15lxUn5ngPfd"],"resp_mime_types":["text/html"]}}
-{"http":{"ts":1402307733473,"uid":"KIRAN12312312","id.orig_h":"192.249.113.37","id.orig_p":58808,"id.resp_h":"72.163.4.161","id.resp_p":80,"trans_depth":1,"method":"GET","host":"www.cisco.com","uri":"/","user_agent":"curl/7.22.0 (x86_64-pc-linux-gnu) libcurl/7.22.0 OpenSSL/1.0.1 zlib/1.2.3.4 libidn/1.23 librtmp/2.3","request_body_len":0,"response_body_len":25523,"status_code":200,"status_msg":"OK","tags":[],"resp_fuids":["FJDyMC15lxUn5ngPfd"],"resp_mime_types":["text/html"]}}
-{"http":{"ts":1402307733473,"uid":"KIRAN12312312","id.orig_h":"192.249.113.37","id.orig_p":58808,"id.resp_h":"72.163.4.161","id.resp_p":80,"trans_depth":1,"method":"GET","host":"www.cisco.com","uri":"/","user_agent":"curl/7.22.0 (x86_64-pc-linux-gnu) libcurl/7.22.0 OpenSSL/1.0.1 zlib/1.2.3.4 libidn/1.23 librtmp/2.3","request_body_len":0,"response_body_len":25523,"status_code":200,"status_msg":"OK","tags":[],"resp_fuids":["FJDyMC15lxUn5ngPfd"],"resp_mime_types":["text/html"]}}
-{"http":{"ts":1402307733473,"uid":"CTo78A11g7CYbbOHvj","id.orig_h":"10.122.196.204","id.orig_p":58808,"id.resp_h":"72.163.4.161","id.resp_p":80,"trans_depth":1,"email":"abullis@mail.csuchico.edu","method":"GET","host":"gabacentre.pw","uri":"/","user_agent":"curl/7.22.0 (x86_64-pc-linux-gnu) libcurl/7.22.0 OpenSSL/1.0.1 zlib/1.2.3.4 libidn/1.23 librtmp/2.3","request_body_len":0,"response_body_len":25523,"status_code":200,"status_msg":"OK","tags":[],"resp_fuids":["FJDyMC15lxUn5ngPfd"],"resp_mime_types":["text/html"]}}
-{"dns":{"ts":1402308259609,"uid":"CYbbOHvj","id.orig_h":"93.188.160.43","id.orig_p":33976,"id.resp_h":"144.254.71.184","id.resp_p":53,"proto":"udp","trans_id":62418,"query":"www.cisco.com","qclass":1,"qclass_name":"C_INTERNET","qtype":28,"qtype_name":"AAAA","rcode":0,"rcode_name":"NOERROR","AA":true,"TC":false,"RD":true,"RA":true,"Z":0,"answers":["gabacentre.pw","www.cisco.com.akadns.net","origin-www.cisco.com","2001:420:1201:2::a"],"TTLs":[3600.0,289.0,14.0],"rejected":false}}
-{"http":{"ts":1402307733473,"uid":"CTo78A11g7CYbbOHvj","id.orig_h":"192.249.113.37","id.orig_p":58808,"id.resp_h":"72.163.4.161","id.resp_p":80,"trans_depth":1,"method":"GET","host":"www.cisco.com","uri":"/","user_agent":"curl/7.22.0 (x86_64-pc-linux-gnu) libcurl/7.22.0 OpenSSL/1.0.1 zlib/1.2.3.4 libidn/1.23 librtmp/2.3","request_body_len":0,"response_body_len":25523,"status_code":200,"status_msg":"OK","tags":[],"resp_fuids":["FJDyMC15lxUn5ngPfd"],"resp_mime_types":["text/html"]}}
-{"dns":{"ts":1402308259609,"uid":"CuJT272SKaJSuqO0Ia","id.orig_h":"10.122.196.204","id.orig_p":33976,"id.resp_h":"144.254.71.184","id.resp_p":53,"proto":"udp","trans_id":62418,"query":"www.cisco.com","qclass":1,"qclass_name":"C_INTERNET","qtype":28,"qtype_name":"AAAA","rcode":0,"rcode_name":"NOERROR","AA":true,"TC":false,"RD":true,"RA":true,"Z":0,"answers":["www.cisco.com.akadns.net","origin-www.cisco.com","2001:420:1201:2::a"],"TTLs":[3600.0,289.0,14.0],"rejected":false}}
-{"http":{"ts":1402307733473,"uid":"KIRAN","id.orig_h":"10.122.196.204","id.orig_p":58808,"id.resp_h":"72.163.4.161","id.resp_p":80,"trans_depth":1,"method":"GET","host":"www.cisco.com","uri":"/","user_agent":"curl/7.22.0 (x86_64-pc-linux-gnu) libcurl/7.22.0 OpenSSL/1.0.1 zlib/1.2.3.4 libidn/1.23 librtmp/2.3","request_body_len":0,"response_body_len":25523,"status_code":200,"status_msg":"OK","tags":[],"resp_fuids":["FJDyMC15lxUn5ngPfd"],"resp_mime_types":["text/html"]}}
-{"http":{"ts":1402307733473,"uid":"KIRAN12312312","id.orig_h":"192.249.113.37","id.orig_p":58808,"id.resp_h":"72.163.4.161","id.resp_p":80,"trans_depth":1,"method":"GET","host":"www.cisco.com","uri":"/","user_agent":"curl/7.22.0 (x86_64-pc-linux-gnu) libcurl/7.22.0 OpenSSL/1.0.1 zlib/1.2.3.4 libidn/1.23 librtmp/2.3","request_body_len":0,"response_body_len":25523,"status_code":200,"status_msg":"OK","tags":[],"resp_fuids":["FJDyMC15lxUn5ngPfd"],"resp_mime_types":["text/html"]}}
-{"http":{"ts":1402307733473,"uid":"KIRAN12312312","id.orig_h":"192.249.113.37","id.orig_p":58808,"id.resp_h":"72.163.4.161","id.resp_p":80,"trans_depth":1,"method":"GET","host":"www.cisco.com","uri":"/","user_agent":"curl/7.22.0 (x86_64-pc-linux-gnu) libcurl/7.22.0 OpenSSL/1.0.1 zlib/1.2.3.4 libidn/1.23 librtmp/2.3","request_body_len":0,"response_body_len":25523,"status_code":200,"status_msg":"OK","tags":[],"resp_fuids":["FJDyMC15lxUn5ngPfd"],"resp_mime_types":["text/html"]}}
-{"http":{"ts":1402307733473,"uid":"CTo78A11g7CYbbOHvj","id.orig_h":"10.122.196.204","id.orig_p":58808,"id.resp_h":"72.163.4.161","id.resp_p":80,"trans_depth":1,"email":"abullis@mail.csuchico.edu","method":"GET","host":"gabacentre.pw","uri":"/","user_agent":"curl/7.22.0 (x86_64-pc-linux-gnu) libcurl/7.22.0 OpenSSL/1.0.1 zlib/1.2.3.4 libidn/1.23 librtmp/2.3","request_body_len":0,"response_body_len":25523,"status_code":200,"status_msg":"OK","tags":[],"resp_fuids":["FJDyMC15lxUn5ngPfd"],"resp_mime_types":["text/html"]}}
-{"dns":{"ts":1402308259609,"uid":"CYbbOHvj","id.orig_h":"93.188.160.43","id.orig_p":33976,"id.resp_h":"144.254.71.184","id.resp_p":53,"proto":"udp","trans_id":62418,"query":"www.cisco.com","qclass":1,"qclass_name":"C_INTERNET","qtype":28,"qtype_name":"AAAA","rcode":0,"rcode_name":"NOERROR","AA":true,"TC":false,"RD":true,"RA":true,"Z":0,"answers":["gabacentre.pw","www.cisco.com.akadns.net","origin-www.cisco.com","2001:420:1201:2::a"],"TTLs":[3600.0,289.0,14.0],"rejected":false}}
-{"http":{"ts":1402307733473,"uid":"CTo78A11g7CYbbOHvj","id.orig_h":"192.249.113.37","id.orig_p":58808,"id.resp_h":"72.163.4.161","id.resp_p":80,"trans_depth":1,"method":"GET","host":"www.cisco.com","uri":"/","user_agent":"curl/7.22.0 (x86_64-pc-linux-gnu) libcurl/7.22.0 OpenSSL/1.0.1 zlib/1.2.3.4 libidn/1.23 librtmp/2.3","request_body_len":0,"response_body_len":25523,"status_code":200,"status_msg":"OK","tags":[],"resp_fuids":["FJDyMC15lxUn5ngPfd"],"resp_mime_types":["text/html"]}}
-{"dns":{"ts":1402308259609,"uid":"CuJT272SKaJSuqO0Ia","id.orig_h":"10.122.196.204","id.orig_p":33976,"id.resp_h":"144.254.71.184","id.resp_p":53,"proto":"udp","trans_id":62418,"query":"www.cisco.com","qclass":1,"qclass_name":"C_INTERNET","qtype":28,"qtype_name":"AAAA","rcode":0,"rcode_name":"NOERROR","AA":true,"TC":false,"RD":true,"RA":true,"Z":0,"answers":["www.cisco.com.akadns.net","origin-www.cisco.com","2001:420:1201:2::a"],"TTLs":[3600.0,289.0,14.0],"rejected":false}}
-{"http":{"ts":1402307733473,"uid":"KIRAN","id.orig_h":"10.122.196.204","id.orig_p":58808,"id.resp_h":"72.163.4.161","id.resp_p":80,"trans_depth":1,"method":"GET","host":"www.cisco.com","uri":"/","user_agent":"curl/7.22.0 (x86_64-pc-linux-gnu) libcurl/7.22.0 OpenSSL/1.0.1 zlib/1.2.3.4 libidn/1.23 librtmp/2.3","request_body_len":0,"response_body_len":25523,"status_code":200,"status_msg":"OK","tags":[],"resp_fuids":["FJDyMC15lxUn5ngPfd"],"resp_mime_types":["text/html"]}}
-{"http":{"ts":1402307733473,"uid":"KIRAN12312312","id.orig_h":"192.249.113.37","id.orig_p":58808,"id.resp_h":"72.163.4.161","id.resp_p":80,"trans_depth":1,"method":"GET","host":"www.cisco.com","uri":"/","user_agent":"curl/7.22.0 (x86_64-pc-linux-gnu) libcurl/7.22.0 OpenSSL/1.0.1 zlib/1.2.3.4 libidn/1.23 librtmp/2.3","request_body_len":0,"response_body_len":25523,"status_code":200,"status_msg":"OK","tags":[],"resp_fuids":["FJDyMC15lxUn5ngPfd"],"resp_mime_types":["text/html"]}}
-{"http":{"ts":1402307733473,"uid":"KIRAN12312312","id.orig_h":"192.249.113.37","id.orig_p":58808,"id.resp_h":"72.163.4.161","id.resp_p":80,"trans_depth":1,"method":"GET","host":"www.cisco.com","uri":"/","user_agent":"curl/7.22.0 (x86_64-pc-linux-gnu) libcurl/7.22.0 OpenSSL/1.0.1 zlib/1.2.3.4 libidn/1.23 librtmp/2.3","request_body_len":0,"response_body_len":25523,"status_code":200,"status_msg":"OK","tags":[],"resp_fuids":["FJDyMC15lxUn5ngPfd"],"resp_mime_types":["text/html"]}}
-{"http":{"ts":1402307733473,"uid":"CTo78A11g7CYbbOHvj","id.orig_h":"10.122.196.204","id.orig_p":58808,"id.resp_h":"72.163.4.161","id.resp_p":80,"trans_depth":1,"email":"abullis@mail.csuchico.edu","method":"GET","host":"gabacentre.pw","uri":"/","user_agent":"curl/7.22.0 (x86_64-pc-linux-gnu) libcurl/7.22.0 OpenSSL/1.0.1 zlib/1.2.3.4 libidn/1.23 librtmp/2.3","request_body_len":0,"response_body_len":25523,"status_code":200,"status_msg":"OK","tags":[],"resp_fuids":["FJDyMC15lxUn5ngPfd"],"resp_mime_types":["text/html"]}}
-{"dns":{"ts":1402308259609,"uid":"CYbbOHvj","id.orig_h":"93.188.160.43","id.orig_p":33976,"id.resp_h":"144.254.71.184","id.resp_p":53,"proto":"udp","trans_id":62418,"query":"www.cisco.com","qclass":1,"qclass_name":"C_INTERNET","qtype":28,"qtype_name":"AAAA","rcode":0,"rcode_name":"NOERROR","AA":true,"TC":false,"RD":true,"RA":true,"Z":0,"answers":["gabacentre.pw","www.cisco.com.akadns.net","origin-www.cisco.com","2001:420:1201:2::a"],"TTLs":[3600.0,289.0,14.0],"rejected":false}}
-{"http":{"ts":1402307733473,"uid":"CTo78A11g7CYbbOHvj","id.orig_h":"192.249.113.37","id.orig_p":58808,"id.resp_h":"72.163.4.161","id.resp_p":80,"trans_depth":1,"method":"GET","host":"www.cisco.com","uri":"/","user_agent":"curl/7.22.0 (x86_64-pc-linux-gnu) libcurl/7.22.0 OpenSSL/1.0.1 zlib/1.2.3.4 libidn/1.23 librtmp/2.3","request_body_len":0,"response_body_len":25523,"status_code":200,"status_msg":"OK","tags":[],"resp_fuids":["FJDyMC15lxUn5ngPfd"],"resp_mime_types":["text/html"]}}
-{"dns":{"ts":1402308259609,"uid":"CuJT272SKaJSuqO0Ia","id.orig_h":"10.122.196.204","id.orig_p":33976,"id.resp_h":"144.254.71.184","id.resp_p":53,"proto":"udp","trans_id":62418,"query":"www.cisco.com","qclass":1,"qclass_name":"C_INTERNET","qtype":28,"qtype_name":"AAAA","rcode":0,"rcode_name":"NOERROR","AA":true,"TC":false,"RD":true,"RA":true,"Z":0,"answers":["www.cisco.com.akadns.net","origin-www.cisco.com","2001:420:1201:2::a"],"TTLs":[3600.0,289.0,14.0],"rejected":false}}
-{"http":{"ts":1402307733473,"uid":"KIRAN","id.orig_h":"10.122.196.204","id.orig_p":58808,"id.resp_h":"72.163.4.161","id.resp_p":80,"trans_depth":1,"method":"GET","host":"www.cisco.com","uri":"/","user_agent":"curl/7.22.0 (x86_64-pc-linux-gnu) libcurl/7.22.0 OpenSSL/1.0.1 zlib/1.2.3.4 libidn/1.23 librtmp/2.3","request_body_len":0,"response_body_len":25523,"status_code":200,"status_msg":"OK","tags":[],"resp_fuids":["FJDyMC15lxUn5ngPfd"],"resp_mime_types":["text/html"]}}
-{"http":{"ts":1402307733473,"uid":"KIRAN12312312","id.orig_h":"192.249.113.37","id.orig_p":58808,"id.resp_h":"72.163.4.161","id.resp_p":80,"trans_depth":1,"method":"GET","host":"www.cisco.com","uri":"/","user_agent":"curl/7.22.0 (x86_64-pc-linux-gnu) libcurl/7.22.0 OpenSSL/1.0.1 zlib/1.2.3.4 libidn/1.23 librtmp/2.3","request_body_len":0,"response_body_len":25523,"status_code":200,"status_msg":"OK","tags":[],"resp_fuids":["FJDyMC15lxUn5ngPfd"],"resp_mime_types":["text/html"]}}
-{"http":{"ts":1402307733473,"uid":"KIRAN12312312","id.orig_h":"192.249.113.37","id.orig_p":58808,"id.resp_h":"72.163.4.161","id.resp_p":80,"trans_depth":1,"method":"GET","host":"www.cisco.com","uri":"/","user_agent":"curl/7.22.0 (x86_64-pc-linux-gnu) libcurl/7.22.0 OpenSSL/1.0.1 zlib/1.2.3.4 libidn/1.23 librtmp/2.3","request_body_len":0,"response_body_len":25523,"status_code":200,"status_msg":"OK","tags":[],"resp_fuids":["FJDyMC15lxUn5ngPfd"],"resp_mime_types":["text/html"]}}
-{"http":{"ts":1402307733473,"uid":"CTo78A11g7CYbbOHvj","id.orig_h":"10.122.196.204","id.orig_p":58808,"id.resp_h":"72.163.4.161","id.resp_p":80,"trans_depth":1,"email":"abullis@mail.csuchico.edu","method":"GET","host":"gabacentre.pw","uri":"/","user_agent":"curl/7.22.0 (x86_64-pc-linux-gnu) libcurl/7.22.0 OpenSSL/1.0.1 zlib/1.2.3.4 libidn/1.23 librtmp/2.3","request_body_len":0,"response_body_len":25523,"status_code":200,"status_msg":"OK","tags":[],"resp_fuids":["FJDyMC15lxUn5ngPfd"],"resp_mime_types":["text/html"]}}
-{"dns":{"ts":1402308259609,"uid":"CYbbOHvj","id.orig_h":"93.188.160.43","id.orig_p":33976,"id.resp_h":"144.254.71.184","id.resp_p":53,"proto":"udp","trans_id":62418,"query":"www.cisco.com","qclass":1,"qclass_name":"C_INTERNET","qtype":28,"qtype_name":"AAAA","rcode":0,"rcode_name":"NOERROR","AA":true,"TC":false,"RD":true,"RA":true,"Z":0,"answers":["gabacentre.pw","www.cisco.com.akadns.net","origin-www.cisco.com","2001:420:1201:2::a"],"TTLs":[3600.0,289.0,14.0],"rejected":false}}
-{"http":{"ts":1402307733473,"uid":"CTo78A11g7CYbbOHvj","id.orig_h":"192.249.113.37","id.orig_p":58808,"id.resp_h":"72.163.4.161","id.resp_p":80,"trans_depth":1,"method":"GET","host":"www.cisco.com","uri":"/","user_agent":"curl/7.22.0 (x86_64-pc-linux-gnu) libcurl/7.22.0 OpenSSL/1.0.1 zlib/1.2.3.4 libidn/1.23 librtmp/2.3","request_body_len":0,"response_body_len":25523,"status_code":200,"status_msg":"OK","tags":[],"resp_fuids":["FJDyMC15lxUn5ngPfd"],"resp_mime_types":["text/html"]}}
-{"dns":{"ts":1402308259609,"uid":"CuJT272SKaJSuqO0Ia","id.orig_h":"10.122.196.204","id.orig_p":33976,"id.resp_h":"144.254.71.184","id.resp_p":53,"proto":"udp","trans_id":62418,"query":"www.cisco.com","qclass":1,"qclass_name":"C_INTERNET","qtype":28,"qtype_name":"AAAA","rcode":0,"rcode_name":"NOERROR","AA":true,"TC":false,"RD":true,"RA":true,"Z":0,"answers":["www.cisco.com.akadns.net","origin-www.cisco.com","2001:420:1201:2::a"],"TTLs":[3600.0,289.0,14.0],"rejected":false}}
-{"http":{"ts":1402307733473,"uid":"KIRAN","id.orig_h":"10.122.196.204","id.orig_p":58808,"id.resp_h":"72.163.4.161","id.resp_p":80,"trans_depth":1,"method":"GET","host":"www.cisco.com","uri":"/","user_agent":"curl/7.22.0 (x86_64-pc-linux-gnu) libcurl/7.22.0 OpenSSL/1.0.1 zlib/1.2.3.4 libidn/1.23 librtmp/2.3","request_body_len":0,"response_body_len":25523,"status_code":200,"status_msg":"OK","tags":[],"resp_fuids":["FJDyMC15lxUn5ngPfd"],"resp_mime_types":["text/html"]}}
-{"http":{"ts":1402307733473,"uid":"KIRAN12312312","id.orig_h":"192.249.113.37","id.orig_p":58808,"id.resp_h":"72.163.4.161","id.resp_p":80,"trans_depth":1,"method":"GET","host":"www.cisco.com","uri":"/","user_agent":"curl/7.22.0 (x86_64-pc-linux-gnu) libcurl/7.22.0 OpenSSL/1.0.1 zlib/1.2.3.4 libidn/1.23 librtmp/2.3","request_body_len":0,"response_body_len":25523,"status_code":200,"status_msg":"OK","tags":[],"resp_fuids":["FJDyMC15lxUn5ngPfd"],"resp_mime_types":["text/html"]}}
-{"http":{"ts":1402307733473,"uid":"KIRAN12312312","id.orig_h":"192.249.113.37","id.orig_p":58808,"id.resp_h":"72.163.4.161","id.resp_p":80,"trans_depth":1,"method":"GET","host":"www.cisco.com","uri":"/","user_agent":"curl/7.22.0 (x86_64-pc-linux-gnu) libcurl/7.22.0 OpenSSL/1.0.1 zlib/1.2.3.4 libidn/1.23 librtmp/2.3","request_body_len":0,"response_body_len":25523,"status_code":200,"status_msg":"OK","tags":[],"resp_fuids":["FJDyMC15lxUn5ngPfd"],"resp_mime_types":["text/html"]}}
-{"http":{"ts":1402307733473,"uid":"CTo78A11g7CYbbOHvj","id.orig_h":"10.122.196.204","id.orig_p":58808,"id.resp_h":"72.163.4.161","id.resp_p":80,"trans_depth":1,"email":"abullis@mail.csuchico.edu","method":"GET","host":"gabacentre.pw","uri":"/","user_agent":"curl/7.22.0 (x86_64-pc-linux-gnu) libcurl/7.22.0 OpenSSL/1.0.1 zlib/1.2.3.4 libidn/1.23 librtmp/2.3","request_body_len":0,"response_body_len":25523,"status_code":200,"status_msg":"OK","tags":[],"resp_fuids":["FJDyMC15lxUn5ngPfd"],"resp_mime_types":["text/html"]}}
-{"dns":{"ts":1402308259609,"uid":"CYbbOHvj","id.orig_h":"93.188.160.43","id.orig_p":33976,"id.resp_h":"144.254.71.184","id.resp_p":53,"proto":"udp","trans_id":62418,"query":"www.cisco.com","qclass":1,"qclass_name":"C_INTERNET","qtype":28,"qtype_name":"AAAA","rcode":0,"rcode_name":"NOERROR","AA":true,"TC":false,"RD":true,"RA":true,"Z":0,"answers":["gabacentre.pw","www.cisco.com.akadns.net","origin-www.cisco.com","2001:420:1201:2::a"],"TTLs":[3600.0,289.0,14.0],"rejected":false}}
-{"http":{"ts":1402307733473,"uid":"CTo78A11g7CYbbOHvj","id.orig_h":"192.249.113.37","id.orig_p":58808,"id.resp_h":"72.163.4.161","id.resp_p":80,"trans_depth":1,"method":"GET","host":"www.cisco.com","uri":"/","user_agent":"curl/7.22.0 (x86_64-pc-linux-gnu) libcurl/7.22.0 OpenSSL/1.0.1 zlib/1.2.3.4 libidn/1.23 librtmp/2.3","request_body_len":0,"response_body_len":25523,"status_code":200,"status_msg":"OK","tags":[],"resp_fuids":["FJDyMC15lxUn5ngPfd"],"resp_mime_types":["text/html"]}}
-{"dns":{"ts":1402308259609,"uid":"CuJT272SKaJSuqO0Ia","id.orig_h":"10.122.196.204","id.orig_p":33976,"id.resp_h":"144.254.71.184","id.resp_p":53,"proto":"udp","trans_id":62418,"query":"www.cisco.com","qclass":1,"qclass_name":"C_INTERNET","qtype":28,"qtype_name":"AAAA","rcode":0,"rcode_name":"NOERROR","AA":true,"TC":false,"RD":true,"RA":true,"Z":0,"answers":["www.cisco.com.akadns.net","origin-www.cisco.com","2001:420:1201:2::a"],"TTLs":[3600.0,289.0,14.0],"rejected":false}}
-{"http":{"ts":1402307733473,"uid":"KIRAN","id.orig_h":"10.122.196.204","id.orig_p":58808,"id.resp_h":"72.163.4.161","id.resp_p":80,"trans_depth":1,"method":"GET","host":"www.cisco.com","uri":"/","user_agent":"curl/7.22.0 (x86_64-pc-linux-gnu) libcurl/7.22.0 OpenSSL/1.0.1 zlib/1.2.3.4 libidn/1.23 librtmp/2.3","request_body_len":0,"response_body_len":25523,"status_code":200,"status_msg":"OK","tags":[],"resp_fuids":["FJDyMC15lxUn5ngPfd"],"resp_mime_types":["text/html"]}}
-{"http":{"ts":1402307733473,"uid":"KIRAN12312312","id.orig_h":"192.249.113.37","id.orig_p":58808,"id.resp_h":"72.163.4.161","id.resp_p":80,"trans_depth":1,"method":"GET","host":"www.cisco.com","uri":"/","user_agent":"curl/7.22.0 (x86_64-pc-linux-gnu) libcurl/7.22.0 OpenSSL/1.0.1 zlib/1.2.3.4 libidn/1.23 librtmp/2.3","request_body_len":0,"response_body_len":25523,"status_code":200,"status_msg":"OK","tags":[],"resp_fuids":["FJDyMC15lxUn5ngPfd"],"resp_mime_types":["text/html"]}}
-{"http":{"ts":1402307733473,"uid":"KIRAN12312312","id.orig_h":"192.249.113.37","id.orig_p":58808,"id.resp_h":"72.163.4.161","id.resp_p":80,"trans_depth":1,"method":"GET","host":"www.cisco.com","uri":"/","user_agent":"curl/7.22.0 (x86_64-pc-linux-gnu) libcurl/7.22.0 OpenSSL/1.0.1 zlib/1.2.3.4 libidn/1.23 librtmp/2.3","request_body_len":0,"response_body_len":25523,"status_code":200,"status_msg":"OK","tags":[],"resp_fuids":["FJDyMC15lxUn5ngPfd"],"resp_mime_types":["text/html"]}}
-{"http":{"ts":1402307733473,"uid":"CTo78A11g7CYbbOHvj","id.orig_h":"10.122.196.204","id.orig_p":58808,"id.resp_h":"72.163.4.161","id.resp_p":80,"trans_depth":1,"email":"abullis@mail.csuchico.edu","method":"GET","host":"gabacentre.pw","uri":"/","user_agent":"curl/7.22.0 (x86_64-pc-linux-gnu) libcurl/7.22.0 OpenSSL/1.0.1 zlib/1.2.3.4 libidn/1.23 librtmp/2.3","request_body_len":0,"response_body_len":25523,"status_code":200,"status_msg":"OK","tags":[],"resp_fuids":["FJDyMC15lxUn5ngPfd"],"resp_mime_types":["text/html"]}}
-{"dns":{"ts":1402308259609,"uid":"CYbbOHvj","id.orig_h":"93.188.160.43","id.orig_p":33976,"id.resp_h":"144.254.71.184","id.resp_p":53,"proto":"udp","trans_id":62418,"query":"www.cisco.com","qclass":1,"qclass_name":"C_INTERNET","qtype":28,"qtype_name":"AAAA","rcode":0,"rcode_name":"NOERROR","AA":true,"TC":false,"RD":true,"RA":true,"Z":0,"answers":["gabacentre.pw","www.cisco.com.akadns.net","origin-www.cisco.com","2001:420:1201:2::a"],"TTLs":[3600.0,289.0,14.0],"rejected":false}}
-{"http":{"ts":1402307733473,"uid":"CTo78A11g7CYbbOHvj","id.orig_h":"192.249.113.37","id.orig_p":58808,"id.resp_h":"72.163.4.161","id.resp_p":80,"trans_depth":1,"method":"GET","host":"www.cisco.com","uri":"/","user_agent":"curl/7.22.0 (x86_64-pc-linux-gnu) libcurl/7.22.0 OpenSSL/1.0.1 zlib/1.2.3.4 libidn/1.23 librtmp/2.3","request_body_len":0,"response_body_len":25523,"status_code":200,"status_msg":"OK","tags":[],"resp_fuids":["FJDyMC15lxUn5ngPfd"],"resp_mime_types":["text/html"]}}
-{"dns":{"ts":1402308259609,"uid":"CuJT272SKaJSuqO0Ia","id.orig_h":"10.122.196.204","id.orig_p":33976,"id.resp_h":"144.254.71.184","id.resp_p":53,"proto":"udp","trans_id":62418,"query":"www.cisco.com","qclass":1,"qclass_name":"C_INTERNET","qtype":28,"qtype_name":"AAAA","rcode":0,"rcode_name":"NOERROR","AA":true,"TC":false,"RD":true,"RA":true,"Z":0,"answers":["www.cisco.com.akadns.net","origin-www.cisco.com","2001:420:1201:2::a"],"TTLs":[3600.0,289.0,14.0],"rejected":false}}
-{"http":{"ts":1402307733473,"uid":"KIRAN","id.orig_h":"10.122.196.204","id.orig_p":58808,"id.resp_h":"72.163.4.161","id.resp_p":80,"trans_depth":1,"method":"GET","host":"www.cisco.com","uri":"/","user_agent":"curl/7.22.0 (x86_64-pc-linux-gnu) libcurl/7.22.0 OpenSSL/1.0.1 zlib/1.2.3.4 libidn/1.23 librtmp/2.3","request_body_len":0,"response_body_len":25523,"status_code":200,"status_msg":"OK","tags":[],"resp_fuids":["FJDyMC15lxUn5ngPfd"],"resp_mime_types":["text/html"]}}
-{"http":{"ts":1402307733473,"uid":"KIRAN12312312","id.orig_h":"192.249.113.37","id.orig_p":58808,"id.resp_h":"72.163.4.161","id.resp_p":80,"trans_depth":1,"method":"GET","host":"www.cisco.com","uri":"/","user_agent":"curl/7.22.0 (x86_64-pc-linux-gnu) libcurl/7.22.0 OpenSSL/1.0.1 zlib/1.2.3.4 libidn/1.23 librtmp/2.3","request_body_len":0,"response_body_len":25523,"status_code":200,"status_msg":"OK","tags":[],"resp_fuids":["FJDyMC15lxUn5ngPfd"],"resp_mime_types":["text/html"]}}
-{"http":{"ts":1402307733473,"uid":"KIRAN12312312","id.orig_h":"192.249.113.37","id.orig_p":58808,"id.resp_h":"72.163.4.161","id.resp_p":80,"trans_depth":1,"method":"GET","host":"www.cisco.com","uri":"/","user_agent":"curl/7.22.0 (x86_64-pc-linux-gnu) libcurl/7.22.0 OpenSSL/1.0.1 zlib/1.2.3.4 libidn/1.23 librtmp/2.3","request_body_len":0,"response_body_len":25523,"status_code":200,"status_msg":"OK","tags":[],"resp_fuids":["FJDyMC15lxUn5ngPfd"],"resp_mime_types":["text/html"]}}
-{"http":{"ts":1402307733473,"uid":"CTo78A11g7CYbbOHvj","id.orig_h":"10.122.196.204","id.orig_p":58808,"id.resp_h":"72.163.4.161","id.resp_p":80,"trans_depth":1,"email":"abullis@mail.csuchico.edu","method":"GET","host":"gabacentre.pw","uri":"/","user_agent":"curl/7.22.0 (x86_64-pc-linux-gnu) libcurl/7.22.0 OpenSSL/1.0.1 zlib/1.2.3.4 libidn/1.23 librtmp/2.3","request_body_len":0,"response_body_len":25523,"status_code":200,"status_msg":"OK","tags":[],"resp_fuids":["FJDyMC15lxUn5ngPfd"],"resp_mime_types":["text/html"]}}
-{"dns":{"ts":1402308259609,"uid":"CYbbOHvj","id.orig_h":"93.188.160.43","id.orig_p":33976,"id.resp_h":"144.254.71.184","id.resp_p":53,"proto":"udp","trans_id":62418,"query":"www.cisco.com","qclass":1,"qclass_name":"C_INTERNET","qtype":28,"qtype_name":"AAAA","rcode":0,"rcode_name":"NOERROR","AA":true,"TC":false,"RD":true,"RA":true,"Z":0,"answers":["gabacentre.pw","www.cisco.com.akadns.net","origin-www.cisco.com","2001:420:1201:2::a"],"TTLs":[3600.0,289.0,14.0],"rejected":false}}
-{"http":{"ts":1402307733473,"uid":"CTo78A11g7CYbbOHvj","id.orig_h":"192.249.113.37","id.orig_p":58808,"id.resp_h":"72.163.4.161","id.resp_p":80,"trans_depth":1,"method":"GET","host":"www.cisco.com","uri":"/","user_agent":"curl/7.22.0 (x86_64-pc-linux-gnu) libcurl/7.22.0 OpenSSL/1.0.1 zlib/1.2.3.4 libidn/1.23 librtmp/2.3","request_body_len":0,"response_body_len":25523,"status_code":200,"status_msg":"OK","tags":[],"resp_fuids":["FJDyMC15lxUn5ngPfd"],"resp_mime_types":["text/html"]}}
-{"dns":{"ts":1402308259609,"uid":"CuJT272SKaJSuqO0Ia","id.orig_h":"10.122.196.204","id.orig_p":33976,"id.resp_h":"144.254.71.184","id.resp_p":53,"proto":"udp","trans_id":62418,"query":"www.cisco.com","qclass":1,"qclass_name":"C_INTERNET","qtype":28,"qtype_name":"AAAA","rcode":0,"rcode_name":"NOERROR","AA":true,"TC":false,"RD":true,"RA":true,"Z":0,"answers":["www.cisco.com.akadns.net","origin-www.cisco.com","2001:420:1201:2::a"],"TTLs":[3600.0,289.0,14.0],"rejected":false}}
-{"http":{"ts":1402307733473,"uid":"KIRAN","id.orig_h":"10.122.196.204","id.orig_p":58808,"id.resp_h":"72.163.4.161","id.resp_p":80,"trans_depth":1,"method":"GET","host":"www.cisco.com","uri":"/","user_agent":"curl/7.22.0 (x86_64-pc-linux-gnu) libcurl/7.22.0 OpenSSL/1.0.1 zlib/1.2.3.4 libidn/1.23 librtmp/2.3","request_body_len":0,"response_body_len":25523,"status_code":200,"status_msg":"OK","tags":[],"resp_fuids":["FJDyMC15lxUn5ngPfd"],"resp_mime_types":["text/html"]}}
-{"http":{"ts":1402307733473,"uid":"KIRAN12312312","id.orig_h":"192.249.113.37","id.orig_p":58808,"id.resp_h":"72.163.4.161","id.resp_p":80,"trans_depth":1,"method":"GET","host":"www.cisco.com","uri":"/","user_agent":"curl/7.22.0 (x86_64-pc-linux-gnu) libcurl/7.22.0 OpenSSL/1.0.1 zlib/1.2.3.4 libidn/1.23 librtmp/2.3","request_body_len":0,"response_body_len":25523,"status_code":200,"status_msg":"OK","tags":[],"resp_fuids":["FJDyMC15lxUn5ngPfd"],"resp_mime_types":["text/html"]}}
-{"http":{"ts":1402307733473,"uid":"KIRAN12312312","id.orig_h":"192.249.113.37","id.orig_p":58808,"id.resp_h":"72.163.4.161","id.resp_p":80,"trans_depth":1,"method":"GET","host":"www.cisco.com","uri":"/","user_agent":"curl/7.22.0 (x86_64-pc-linux-gnu) libcurl/7.22.0 OpenSSL/1.0.1 zlib/1.2.3.4 libidn/1.23 librtmp/2.3","request_body_len":0,"response_body_len":25523,"status_code":200,"status_msg":"OK","tags":[],"resp_fuids":["FJDyMC15lxUn5ngPfd"],"resp_mime_types":["text/html"]}}
-{"http":{"ts":1402307733473,"uid":"CTo78A11g7CYbbOHvj","id.orig_h":"10.122.196.204","id.orig_p":58808,"id.resp_h":"72.163.4.161","id.resp_p":80,"trans_depth":1,"email":"abullis@mail.csuchico.edu","method":"GET","host":"gabacentre.pw","uri":"/","user_agent":"curl/7.22.0 (x86_64-pc-linux-gnu) libcurl/7.22.0 OpenSSL/1.0.1 zlib/1.2.3.4 libidn/1.23 librtmp/2.3","request_body_len":0,"response_body_len":25523,"status_code":200,"status_msg":"OK","tags":[],"resp_fuids":["FJDyMC15lxUn5ngPfd"],"resp_mime_types":["text/html"]}}
-{"dns":{"ts":1402308259609,"uid":"CYbbOHvj","id.orig_h":"93.188.160.43","id.orig_p":33976,"id.resp_h":"144.254.71.184","id.resp_p":53,"proto":"udp","trans_id":62418,"query":"www.cisco.com","qclass":1,"qclass_name":"C_INTERNET","qtype":28,"qtype_name":"AAAA","rcode":0,"rcode_name":"NOERROR","AA":true,"TC":false,"RD":true,"RA":true,"Z":0,"answers":["gabacentre.pw","www.cisco.com.akadns.net","origin-www.cisco.com","2001:420:1201:2::a"],"TTLs":[3600.0,289.0,14.0],"rejected":false}}
-{"http":{"ts":1402307733473,"uid":"CTo78A11g7CYbbOHvj","id.orig_h":"192.249.113.37","id.orig_p":58808,"id.resp_h":"72.163.4.161","id.resp_p":80,"trans_depth":1,"method":"GET","host":"www.cisco.com","uri":"/","user_agent":"curl/7.22.0 (x86_64-pc-linux-gnu) libcurl/7.22.0 OpenSSL/1.0.1 zlib/1.2.3.4 libidn/1.23 librtmp/2.3","request_body_len":0,"response_body_len":25523,"status_code":200,"status_msg":"OK","tags":[],"resp_fuids":["FJDyMC15lxUn5ngPfd"],"resp_mime_types":["text/html"]}}
-{"dns":{"ts":1402308259609,"uid":"CuJT272SKaJSuqO0Ia","id.orig_h":"10.122.196.204","id.orig_p":33976,"id.resp_h":"144.254.71.184","id.resp_p":53,"proto":"udp","trans_id":62418,"query":"www.cisco.com","qclass":1,"qclass_name":"C_INTERNET","qtype":28,"qtype_name":"AAAA","rcode":0,"rcode_name":"NOERROR","AA":true,"TC":false,"RD":true,"RA":true,"Z":0,"answers":["www.cisco.com.akadns.net","origin-www.cisco.com","2001:420:1201:2::a"],"TTLs":[3600.0,289.0,14.0],"rejected":false}}
-{"http":{"ts":1402307733473,"uid":"KIRAN","id.orig_h":"10.122.196.204","id.orig_p":58808,"id.resp_h":"72.163.4.161","id.resp_p":80,"trans_depth":1,"method":"GET","host":"www.cisco.com","uri":"/","user_agent":"curl/7.22.0 (x86_64-pc-linux-gnu) libcurl/7.22.0 OpenSSL/1.0.1 zlib/1.2.3.4 libidn/1.23 librtmp/2.3","request_body_len":0,"response_body_len":25523,"status_code":200,"status_msg":"OK","tags":[],"resp_fuids":["FJDyMC15lxUn5ngPfd"],"resp_mime_types":["text/html"]}}
-{"http":{"ts":1402307733473,"uid":"KIRAN12312312","id.orig_h":"192.249.113.37","id.orig_p":58808,"id.resp_h":"72.163.4.161","id.resp_p":80,"trans_depth":1,"method":"GET","host":"www.cisco.com","uri":"/","user_agent":"curl/7.22.0 (x86_64-pc-linux-gnu) libcurl/7.22.0 OpenSSL/1.0.1 zlib/1.2.3.4 libidn/1.23 librtmp/2.3","request_body_len":0,"response_body_len":25523,"status_code":200,"status_msg":"OK","tags":[],"resp_fuids":["FJDyMC15lxUn5ngPfd"],"resp_mime_types":["text/html"]}}
-{"http":{"ts":1402307733473,"uid":"KIRAN12312312","id.orig_h":"192.249.113.37","id.orig_p":58808,"id.resp_h":"72.163.4.161","id.resp_p":80,"trans_depth":1,"method":"GET","host":"www.cisco.com","uri":"/","user_agent":"curl/7.22.0 (x86_64-pc-linux-gnu) libcurl/7.22.0 OpenSSL/1.0.1 zlib/1.2.3.4 libidn/1.23 librtmp/2.3","request_body_len":0,"response_body_len":25523,"status_code":200,"status_msg":"OK","tags":[],"resp_fuids":["FJDyMC15lxUn5ngPfd"],"resp_mime_types":["text/html"]}}
-{"http":{"ts":1402307733473,"uid":"CTo78A11g7CYbbOHvj","id.orig_h":"10.122.196.204","id.orig_p":58808,"id.resp_h":"72.163.4.161","id.resp_p":80,"trans_depth":1,"email":"abullis@mail.csuchico.edu","method":"GET","host":"gabacentre.pw","uri":"/","user_agent":"curl/7.22.0 (x86_64-pc-linux-gnu) libcurl/7.22.0 OpenSSL/1.0.1 zlib/1.2.3.4 libidn/1.23 librtmp/2.3","request_body_len":0,"response_body_len":25523,"status_code":200,"status_msg":"OK","tags":[],"resp_fuids":["FJDyMC15lxUn5ngPfd"],"resp_mime_types":["text/html"]}}
-{"dns":{"ts":1402308259609,"uid":"CYbbOHvj","id.orig_h":"93.188.160.43","id.orig_p":33976,"id.resp_h":"144.254.71.184","id.resp_p":53,"proto":"udp","trans_id":62418,"query":"www.cisco.com","qclass":1,"qclass_name":"C_INTERNET","qtype":28,"qtype_name":"AAAA","rcode":0,"rcode_name":"NOERROR","AA":true,"TC":false,"RD":true,"RA":true,"Z":0,"answers":["gabacentre.pw","www.cisco.com.akadns.net","origin-www.cisco.com","2001:420:1201:2::a"],"TTLs":[3600.0,289.0,14.0],"rejected":false}}
-{"http":{"ts":1402307733473,"uid":"CTo78A11g7CYbbOHvj","id.orig_h":"192.249.113.37","id.orig_p":58808,"id.resp_h":"72.163.4.161","id.resp_p":80,"trans_depth":1,"method":"GET","host":"www.cisco.com","uri":"/","user_agent":"curl/7.22.0 (x86_64-pc-linux-gnu) libcurl/7.22.0 OpenSSL/1.0.1 zlib/1.2.3.4 libidn/1.23 librtmp/2.3","request_body_len":0,"response_body_len":25523,"status_code":200,"status_msg":"OK","tags":[],"resp_fuids":["FJDyMC15lxUn5ngPfd"],"resp_mime_types":["text/html"]}}
-{"dns":{"ts":1402308259609,"uid":"CuJT272SKaJSuqO0Ia","id.orig_h":"10.122.196.204","id.orig_p":33976,"id.resp_h":"144.254.71.184","id.resp_p":53,"proto":"udp","trans_id":62418,"query":"www.cisco.com","qclass":1,"qclass_name":"C_INTERNET","qtype":28,"qtype_name":"AAAA","rcode":0,"rcode_name":"NOERROR","AA":true,"TC":false,"RD":true,"RA":true,"Z":0,"answers":["www.cisco.com.akadns.net","origin-www.cisco.com","2001:420:1201:2::a"],"TTLs":[3600.0,289.0,14.0],"rejected":false}}
-{"http":{"ts":1402307733473,"uid":"KIRAN","id.orig_h":"10.122.196.204","id.orig_p":58808,"id.resp_h":"72.163.4.161","id.resp_p":80,"trans_depth":1,"method":"GET","host":"www.cisco.com","uri":"/","user_agent":"curl/7.22.0 (x86_64-pc-linux-gnu) libcurl/7.22.0 OpenSSL/1.0.1 zlib/1.2.3.4 libidn/1.23 librtmp/2.3","request_body_len":0,"response_body_len":25523,"status_code":200,"status_msg":"OK","tags":[],"resp_fuids":["FJDyMC15lxUn5ngPfd"],"resp_mime_types":["text/html"]}}
-{"http":{"ts":1402307733473,"uid":"KIRAN12312312","id.orig_h":"192.249.113.37","id.orig_p":58808,"id.resp_h":"72.163.4.161","id.resp_p":80,"trans_depth":1,"method":"GET","host":"www.cisco.com","uri":"/","user_agent":"curl/7.22.0 (x86_64-pc-linux-gnu) libcurl/7.22.0 OpenSSL/1.0.1 zlib/1.2.3.4 libidn/1.23 librtmp/2.3","request_body_len":0,"response_body_len":25523,"status_code":200,"status_msg":"OK","tags":[],"resp_fuids":["FJDyMC15lxUn5ngPfd"],"resp_mime_types":["text/html"]}}
-{"http":{"ts":1402307733473,"uid":"KIRAN12312312","id.orig_h":"192.249.113.37","id.orig_p":58808,"id.resp_h":"72.163.4.161","id.resp_p":80,"trans_depth":1,"method":"GET","host":"www.cisco.com","uri":"/","user_agent":"curl/7.22.0 (x86_64-pc-linux-gnu) libcurl/7.22.0 OpenSSL/1.0.1 zlib/1.2.3.4 libidn/1.23 librtmp/2.3","request_body_len":0,"response_body_len":25523,"status_code":200,"status_msg":"OK","tags":[],"resp_fuids":["FJDyMC15lxUn5ngPfd"],"resp_mime_types":["text/html"]}}
-{"http":{"ts":1402307733473,"uid":"CTo78A11g7CYbbOHvj","id.orig_h":"10.122.196.204","id.orig_p":58808,"id.resp_h":"72.163.4.161","id.resp_p":80,"trans_depth":1,"email":"abullis@mail.csuchico.edu","method":"GET","host":"gabacentre.pw","uri":"/","user_agent":"curl/7.22.0 (x86_64-pc-linux-gnu) libcurl/7.22.0 OpenSSL/1.0.1 zlib/1.2.3.4 libidn/1.23 librtmp/2.3","request_body_len":0,"response_body_len":25523,"status_code":200,"status_msg":"OK","tags":[],"resp_fuids":["FJDyMC15lxUn5ngPfd"],"resp_mime_types":["text/html"]}}
-{"dns":{"ts":1402308259609,"uid":"CYbbOHvj","id.orig_h":"93.188.160.43","id.orig_p":33976,"id.resp_h":"144.254.71.184","id.resp_p":53,"proto":"udp","trans_id":62418,"query":"www.cisco.com","qclass":1,"qclass_name":"C_INTERNET","qtype":28,"qtype_name":"AAAA","rcode":0,"rcode_name":"NOERROR","AA":true,"TC":false,"RD":true,"RA":true,"Z":0,"answers":["gabacentre.pw","www.cisco.com.akadns.net","origin-www.cisco.com","2001:420:1201:2::a"],"TTLs":[3600.0,289.0,14.0],"rejected":false}}
-{"http":{"ts":1402307733473,"uid":"CTo78A11g7CYbbOHvj","id.orig_h":"192.249.113.37","id.orig_p":58808,"id.resp_h":"72.163.4.161","id.resp_p":80,"trans_depth":1,"method":"GET","host":"www.cisco.com","uri":"/","user_agent":"curl/7.22.0 (x86_64-pc-linux-gnu) libcurl/7.22.0 OpenSSL/1.0.1 zlib/1.2.3.4 libidn/1.23 librtmp/2.3","request_body_len":0,"response_body_len":25523,"status_code":200,"status_msg":"OK","tags":[],"resp_fuids":["FJDyMC15lxUn5ngPfd"],"resp_mime_types":["text/html"]}}
-{"dns":{"ts":1402308259609,"uid":"CuJT272SKaJSuqO0Ia","id.orig_h":"10.122.196.204","id.orig_p":33976,"id.resp_h":"144.254.71.184","id.resp_p":53,"proto":"udp","trans_id":62418,"query":"www.cisco.com","qclass":1,"qclass_name":"C_INTERNET","qtype":28,"qtype_name":"AAAA","rcode":0,"rcode_name":"NOERROR","AA":true,"TC":false,"RD":true,"RA":true,"Z":0,"answers":["www.cisco.com.akadns.net","origin-www.cisco.com","2001:420:1201:2::a"],"TTLs":[3600.0,289.0,14.0],"rejected":false}}
-{"http":{"ts":1402307733473,"uid":"KIRAN","id.orig_h":"10.122.196.204","id.orig_p":58808,"id.resp_h":"72.163.4.161","id.resp_p":80,"trans_depth":1,"method":"GET","host":"www.cisco.com","uri":"/","user_agent":"curl/7.22.0 (x86_64-pc-linux-gnu) libcurl/7.22.0 OpenSSL/1.0.1 zlib/1.2.3.4 libidn/1.23 librtmp/2.3","request_body_len":0,"response_body_len":25523,"status_code":200,"status_msg":"OK","tags":[],"resp_fuids":["FJDyMC15lxUn5ngPfd"],"resp_mime_types":["text/html"]}}
-{"http":{"ts":1402307733473,"uid":"KIRAN12312312","id.orig_h":"192.249.113.37","id.orig_p":58808,"id.resp_h":"72.163.4.161","id.resp_p":80,"trans_depth":1,"method":"GET","host":"www.cisco.com","uri":"/","user_agent":"curl/7.22.0 (x86_64-pc-linux-gnu) libcurl/7.22.0 OpenSSL/1.0.1 zlib/1.2.3.4 libidn/1.23 librtmp/2.3","request_body_len":0,"response_body_len":25523,"status_code":200,"status_msg":"OK","tags":[],"resp_fuids":["FJDyMC15lxUn5ngPfd"],"resp_mime_types":["text/html"]}}
-{"http":{"ts":1402307733473,"uid":"KIRAN12312312","id.orig_h":"192.249.113.37","id.orig_p":58808,"id.resp_h":"72.163.4.161","id.resp_p":80,"trans_depth":1,"method":"GET","host":"www.cisco.com","uri":"/","user_agent":"curl/7.22.0 (x86_64-pc-linux-gnu) libcurl/7.22.0 OpenSSL/1.0.1 zlib/1.2.3.4 libidn/1.23 librtmp/2.3","request_body_len":0,"response_body_len":25523,"status_code":200,"status_msg":"OK","tags":[],"resp_fuids":["FJDyMC15lxUn5ngPfd"],"resp_mime_types":["text/html"]}}
-{"http":{"ts":1402307733473,"uid":"CTo78A11g7CYbbOHvj","id.orig_h":"10.122.196.204","id.orig_p":58808,"id.resp_h":"72.163.4.161","id.resp_p":80,"trans_depth":1,"email":"abullis@mail.csuchico.edu","method":"GET","host":"gabacentre.pw","uri":"/","user_agent":"curl/7.22.0 (x86_64-pc-linux-gnu) libcurl/7.22.0 OpenSSL/1.0.1 zlib/1.2.3.4 libidn/1.23 librtmp/2.3","request_body_len":0,"response_body_len":25523,"status_code":200,"status_msg":"OK","tags":[],"resp_fuids":["FJDyMC15lxUn5ngPfd"],"resp_mime_types":["text/html"]}}
-{"dns":{"ts":1402308259609,"uid":"CYbbOHvj","id.orig_h":"93.188.160.43","id.orig_p":33976,"id.resp_h":"144.254.71.184","id.resp_p":53,"proto":"udp","trans_id":62418,"query":"www.cisco.com","qclass":1,"qclass_name":"C_INTERNET","qtype":28,"qtype_name":"AAAA","rcode":0,"rcode_name":"NOERROR","AA":true,"TC":false,"RD":true,"RA":true,"Z":0,"answers":["gabacentre.pw","www.cisco.com.akadns.net","origin-www.cisco.com","2001:420:1201:2::a"],"TTLs":[3600.0,289.0,14.0],"rejected":false}}
-{"http":{"ts":1402307733473,"uid":"CTo78A11g7CYbbOHvj","id.orig_h":"192.249.113.37","id.orig_p":58808,"id.resp_h":"72.163.4.161","id.resp_p":80,"trans_depth":1,"method":"GET","host":"www.cisco.com","uri":"/","user_agent":"curl/7.22.0 (x86_64-pc-linux-gnu) libcurl/7.22.0 OpenSSL/1.0.1 zlib/1.2.3.4 libidn/1.23 librtmp/2.3","request_body_len":0,"response_body_len":25523,"status_code":200,"status_msg":"OK","tags":[],"resp_fuids":["FJDyMC15lxUn5ngPfd"],"resp_mime_types":["text/html"]}}
-{"dns":{"ts":1402308259609,"uid":"CuJT272SKaJSuqO0Ia","id.orig_h":"10.122.196.204","id.orig_p":33976,"id.resp_h":"144.254.71.184","id.resp_p":53,"proto":"udp","trans_id":62418,"query":"www.cisco.com","qclass":1,"qclass_name":"C_INTERNET","qtype":28,"qtype_name":"AAAA","rcode":0,"rcode_name":"NOERROR","AA":true,"TC":false,"RD":true,"RA":true,"Z":0,"answers":["www.cisco.com.akadns.net","origin-www.cisco.com","2001:420:1201:2::a"],"TTLs":[3600.0,289.0,14.0],"rejected":false}}
-{"http":{"ts":1402307733473,"uid":"KIRAN","id.orig_h":"10.122.196.204","id.orig_p":58808,"id.resp_h":"72.163.4.161","id.resp_p":80,"trans_depth":1,"method":"GET","host":"www.cisco.com","uri":"/","user_agent":"curl/7.22.0 (x86_64-pc-linux-gnu) libcurl/7.22.0 OpenSSL/1.0.1 zlib/1.2.3.4 libidn/1.23 librtmp/2.3","request_body_len":0,"response_body_len":25523,"status_code":200,"status_msg":"OK","tags":[],"resp_fuids":["FJDyMC15lxUn5ngPfd"],"resp_mime_types":["text/html"]}}
-{"http":{"ts":1402307733473,"uid":"KIRAN12312312","id.orig_h":"192.249.113.37","id.orig_p":58808,"id.resp_h":"72.163.4.161","id.resp_p":80,"trans_depth":1,"method":"GET","host":"www.cisco.com","uri":"/","user_agent":"curl/7.22.0 (x86_64-pc-linux-gnu) libcurl/7.22.0 OpenSSL/1.0.1 zlib/1.2.3.4 libidn/1.23 librtmp/2.3","request_body_len":0,"response_body_len":25523,"status_code":200,"status_msg":"OK","tags":[],"resp_fuids":["FJDyMC15lxUn5ngPfd"],"resp_mime_types":["text/html"]}}
-{"http":{"ts":1402307733473,"uid":"KIRAN12312312","id.orig_h":"192.249.113.37","id.orig_p":58808,"id.resp_h":"72.163.4.161","id.resp_p":80,"trans_depth":1,"method":"GET","host":"www.cisco.com","uri":"/","user_agent":"curl/7.22.0 (x86_64-pc-linux-gnu) libcurl/7.22.0 OpenSSL/1.0.1 zlib/1.2.3.4 libidn/1.23 librtmp/2.3","request_body_len":0,"response_body_len":25523,"status_code":200,"status_msg":"OK","tags":[],"resp_fuids":["FJDyMC15lxUn5ngPfd"],"resp_mime_types":["text/html"]}}
-{"http":{"ts":1402307733473,"uid":"CTo78A11g7CYbbOHvj","id.orig_h":"10.122.196.204","id.orig_p":58808,"id.resp_h":"72.163.4.161","id.resp_p":80,"trans_depth":1,"email":"abullis@mail.csuchico.edu","method":"GET","host":"gabacentre.pw","uri":"/","user_agent":"curl/7.22.0 (x86_64-pc-linux-gnu) libcurl/7.22.0 OpenSSL/1.0.1 zlib/1.2.3.4 libidn/1.23 librtmp/2.3","request_body_len":0,"response_body_len":25523,"status_code":200,"status_msg":"OK","tags":[],"resp_fuids":["FJDyMC15lxUn5ngPfd"],"resp_mime_types":["text/html"]}}
-{"dns":{"ts":1402308259609,"uid":"CYbbOHvj","id.orig_h":"93.188.160.43","id.orig_p":33976,"id.resp_h":"144.254.71.184","id.resp_p":53,"proto":"udp","trans_id":62418,"query":"www.cisco.com","qclass":1,"qclass_name":"C_INTERNET","qtype":28,"qtype_name":"AAAA","rcode":0,"rcode_name":"NOERROR","AA":true,"TC":false,"RD":true,"RA":true,"Z":0,"answers":["gabacentre.pw","www.cisco.com.akadns.net","origin-www.cisco.com","2001:420:1201:2::a"],"TTLs":[3600.0,289.0,14.0],"rejected":false}}
-{"http":{"ts":1402307733473,"uid":"CTo78A11g7CYbbOHvj","id.orig_h":"192.249.113.37","id.orig_p":58808,"id.resp_h":"72.163.4.161","id.resp_p":80,"trans_depth":1,"method":"GET","host":"www.cisco.com","uri":"/","user_agent":"curl/7.22.0 (x86_64-pc-linux-gnu) libcurl/7.22.0 OpenSSL/1.0.1 zlib/1.2.3.4 libidn/1.23 librtmp/2.3","request_body_len":0,"response_body_len":25523,"status_code":200,"status_msg":"OK","tags":[],"resp_fuids":["FJDyMC15lxUn5ngPfd"],"resp_mime_types":["text/html"]}}
-{"dns":{"ts":1402308259609,"uid":"CuJT272SKaJSuqO0Ia","id.orig_h":"10.122.196.204","id.orig_p":33976,"id.resp_h":"144.254.71.184","id.resp_p":53,"proto":"udp","trans_id":62418,"query":"www.cisco.com","qclass":1,"qclass_name":"C_INTERNET","qtype":28,"qtype_name":"AAAA","rcode":0,"rcode_name":"NOERROR","AA":true,"TC":false,"RD":true,"RA":true,"Z":0,"answers":["www.cisco.com.akadns.net","origin-www.cisco.com","2001:420:1201:2::a"],"TTLs":[3600.0,289.0,14.0],"rejected":false}}
-{"http":{"ts":1402307733473,"uid":"KIRAN","id.orig_h":"10.122.196.204","id.orig_p":58808,"id.resp_h":"72.163.4.161","id.resp_p":80,"trans_depth":1,"method":"GET","host":"www.cisco.com","uri":"/","user_agent":"curl/7.22.0 (x86_64-pc-linux-gnu) libcurl/7.22.0 OpenSSL/1.0.1 zlib/1.2.3.4 libidn/1.23 librtmp/2.3","request_body_len":0,"response_body_len":25523,"status_code":200,"status_msg":"OK","tags":[],"resp_fuids":["FJDyMC15lxUn5ngPfd"],"resp_mime_types":["text/html"]}}
-{"http":{"ts":1402307733473,"uid":"KIRAN12312312","id.orig_h":"192.249.113.37","id.orig_p":58808,"id.resp_h":"72.163.4.161","id.resp_p":80,"trans_depth":1,"method":"GET","host":"www.cisco.com","uri":"/","user_agent":"curl/7.22.0 (x86_64-pc-linux-gnu) libcurl/7.22.0 OpenSSL/1.0.1 zlib/1.2.3.4 libidn/1.23 librtmp/2.3","request_body_len":0,"response_body_len":25523,"status_code":200,"status_msg":"OK","tags":[],"resp_fuids":["FJDyMC15lxUn5ngPfd"],"resp_mime_types":["text/html"]}}
-{"http":{"ts":1402307733473,"uid":"KIRAN12312312","id.orig_h":"192.249.113.37","id.orig_p":58808,"id.resp_h":"72.163.4.161","id.resp_p":80,"trans_depth":1,"method":"GET","host":"www.cisco.com","uri":"/","user_agent":"curl/7.22.0 (x86_64-pc-linux-gnu) libcurl/7.22.0 OpenSSL/1.0.1 zlib/1.2.3.4 libidn/1.23 librtmp/2.3","request_body_len":0,"response_body_len":25523,"status_code":200,"status_msg":"OK","tags":[],"resp_fuids":["FJDyMC15lxUn5ngPfd"],"resp_mime_types":["text/html"]}}
-{"http":{"ts":1402307733473,"uid":"CTo78A11g7CYbbOHvj","id.orig_h":"10.122.196.204","id.orig_p":58808,"id.resp_h":"72.163.4.161","id.resp_p":80,"trans_depth":1,"email":"abullis@mail.csuchico.edu","method":"GET","host":"gabacentre.pw","uri":"/","user_agent":"curl/7.22.0 (x86_64-pc-linux-gnu) libcurl/7.22.0 OpenSSL/1.0.1 zlib/1.2.3.4 libidn/1.23 librtmp/2.3","request_body_len":0,"response_body_len":25523,"status_code":200,"status_msg":"OK","tags":[],"resp_fuids":["FJDyMC15lxUn5ngPfd"],"resp_mime_types":["text/html"]}}
-{"dns":{"ts":1402308259609,"uid":"CYbbOHvj","id.orig_h":"93.188.160.43","id.orig_p":33976,"id.resp_h":"144.254.71.184","id.resp_p":53,"proto":"udp","trans_id":62418,"query":"www.cisco.com","qclass":1,"qclass_name":"C_INTERNET","qtype":28,"qtype_name":"AAAA","rcode":0,"rcode_name":"NOERROR","AA":true,"TC":false,"RD":true,"RA":true,"Z":0,"answers":["gabacentre.pw","www.cisco.com.akadns.net","origin-www.cisco.com","2001:420:1201:2::a"],"TTLs":[3600.0,289.0,14.0],"rejected":false}}
-{"http":{"ts":1402307733473,"uid":"CTo78A11g7CYbbOHvj","id.orig_h":"192.249.113.37","id.orig_p":58808,"id.resp_h":"72.163.4.161","id.resp_p":80,"trans_depth":1,"method":"GET","host":"www.cisco.com","uri":"/","user_agent":"curl/7.22.0 (x86_64-pc-linux-gnu) libcurl/7.22.0 OpenSSL/1.0.1 zlib/1.2.3.4 libidn/1.23 librtmp/2.3","request_body_len":0,"response_body_len":25523,"status_code":200,"status_msg":"OK","tags":[],"resp_fuids":["FJDyMC15lxUn5ngPfd"],"resp_mime_types":["text/html"]}}
-{"dns":{"ts":1402308259609,"uid":"CuJT272SKaJSuqO0Ia","id.orig_h":"10.122.196.204","id.orig_p":33976,"id.resp_h":"144.254.71.184","id.resp_p":53,"proto":"udp","trans_id":62418,"query":"www.cisco.com","qclass":1,"qclass_name":"C_INTERNET","qtype":28,"qtype_name":"AAAA","rcode":0,"rcode_name":"NOERROR","AA":true,"TC":false,"RD":true,"RA":true,"Z":0,"answers":["www.cisco.com.akadns.net","origin-www.cisco.com","2001:420:1201:2::a"],"TTLs":[3600.0,289.0,14.0],"rejected":false}}
-{"http":{"ts":1402307733473,"uid":"KIRAN","id.orig_h":"10.122.196.204","id.orig_p":58808,"id.resp_h":"72.163.4.161","id.resp_p":80,"trans_depth":1,"method":"GET","host":"www.cisco.com","uri":"/","user_agent":"curl/7.22.0 (x86_64-pc-linux-gnu) libcurl/7.22.0 OpenSSL/1.0.1 zlib/1.2.3.4 libidn/1.23 librtmp/2.3","request_body_len":0,"response_body_len":25523,"status_code":200,"status_msg":"OK","tags":[],"resp_fuids":["FJDyMC15lxUn5ngPfd"],"resp_mime_types":["text/html"]}}
-{"http":{"ts":1402307733473,"uid":"KIRAN12312312","id.orig_h":"192.249.113.37","id.orig_p":58808,"id.resp_h":"72.163.4.161","id.resp_p":80,"trans_depth":1,"method":"GET","host":"www.cisco.com","uri":"/","user_agent":"curl/7.22.0 (x86_64-pc-linux-gnu) libcurl/7.22.0 OpenSSL/1.0.1 zlib/1.2.3.4 libidn/1.23 librtmp/2.3","request_body_len":0,"response_body_len":25523,"status_code":200,"status_msg":"OK","tags":[],"resp_fuids":["FJDyMC15lxUn5ngPfd"],"resp_mime_types":["text/html"]}}
-{"http":{"ts":1402307733473,"uid":"KIRAN12312312","id.orig_h":"192.249.113.37","id.orig_p":58808,"id.resp_h":"72.163.4.161","id.resp_p":80,"trans_depth":1,"method":"GET","host":"www.cisco.com","uri":"/","user_agent":"curl/7.22.0 (x86_64-pc-linux-gnu) libcurl/7.22.0 OpenSSL/1.0.1 zlib/1.2.3.4 libidn/1.23 librtmp/2.3","request_body_len":0,"response_body_len":25523,"status_code":200,"status_msg":"OK","tags":[],"resp_fuids":["FJDyMC15lxUn5ngPfd"],"resp_mime_types":["text/html"]}}
-{"http":{"ts":1402307733473,"uid":"CTo78A11g7CYbbOHvj","id.orig_h":"10.122.196.204","id.orig_p":58808,"id.resp_h":"72.163.4.161","id.resp_p":80,"trans_depth":1,"email":"abullis@mail.csuchico.edu","method":"GET","host":"gabacentre.pw","uri":"/","user_agent":"curl/7.22.0 (x86_64-pc-linux-gnu) libcurl/7.22.0 OpenSSL/1.0.1 zlib/1.2.3.4 libidn/1.23 librtmp/2.3","request_body_len":0,"response_body_len":25523,"status_code":200,"status_msg":"OK","tags":[],"resp_fuids":["FJDyMC15lxUn5ngPfd"],"resp_mime_types":["text/html"]}}
-{"dns":{"ts":1402308259609,"uid":"CYbbOHvj","id.orig_h":"93.188.160.43","id.orig_p":33976,"id.resp_h":"144.254.71.184","id.resp_p":53,"proto":"udp","trans_id":62418,"query":"www.cisco.com","qclass":1,"qclass_name":"C_INTERNET","qtype":28,"qtype_name":"AAAA","rcode":0,"rcode_name":"NOERROR","AA":true,"TC":false,"RD":true,"RA":true,"Z":0,"answers":["gabacentre.pw","www.cisco.com.akadns.net","origin-www.cisco.com","2001:420:1201:2::a"],"TTLs":[3600.0,289.0,14.0],"rejected":false}}
-{"http":{"ts":1402307733473,"uid":"CTo78A11g7CYbbOHvj","id.orig_h":"192.249.113.37","id.orig_p":58808,"id.resp_h":"72.163.4.161","id.resp_p":80,"trans_depth":1,"method":"GET","host":"www.cisco.com","uri":"/","user_agent":"curl/7.22.0 (x86_64-pc-linux-gnu) libcurl/7.22.0 OpenSSL/1.0.1 zlib/1.2.3.4 libidn/1.23 librtmp/2.3","request_body_len":0,"response_body_len":25523,"status_code":200,"status_msg":"OK","tags":[],"resp_fuids":["FJDyMC15lxUn5ngPfd"],"resp_mime_types":["text/html"]}}
-{"dns":{"ts":1402308259609,"uid":"CuJT272SKaJSuqO0Ia","id.orig_h":"10.122.196.204","id.orig_p":33976,"id.resp_h":"144.254.71.184","id.resp_p":53,"proto":"udp","trans_id":62418,"query":"www.cisco.com","qclass":1,"qclass_name":"C_INTERNET","qtype":28,"qtype_name":"AAAA","rcode":0,"rcode_name":"NOERROR","AA":true,"TC":false,"RD":true,"RA":true,"Z":0,"answers":["www.cisco.com.akadns.net","origin-www.cisco.com","2001:420:1201:2::a"],"TTLs":[3600.0,289.0,14.0],"rejected":false}}
-{"http":{"ts":1402307733473,"uid":"KIRAN","id.orig_h":"10.122.196.204","id.orig_p":58808,"id.resp_h":"72.163.4.161","id.resp_p":80,"trans_depth":1,"method":"GET","host":"www.cisco.com","uri":"/","user_agent":"curl/7.22.0 (x86_64-pc-linux-gnu) libcurl/7.22.0 OpenSSL/1.0.1 zlib/1.2.3.4 libidn/1.23 librtmp/2.3","request_body_len":0,"response_body_len":25523,"status_code":200,"status_msg":"OK","tags":[],"resp_fuids":["FJDyMC15lxUn5ngPfd"],"resp_mime_types":["text/html"]}}
-{"http":{"ts":1402307733473,"uid":"KIRAN12312312","id.orig_h":"192.249.113.37","id.orig_p":58808,"id.resp_h":"72.163.4.161","id.resp_p":80,"trans_depth":1,"method":"GET","host":"www.cisco.com","uri":"/","user_agent":"curl/7.22.0 (x86_64-pc-linux-gnu) libcurl/7.22.0 OpenSSL/1.0.1 zlib/1.2.3.4 libidn/1.23 librtmp/2.3","request_body_len":0,"response_body_len":25523,"status_code":200,"status_msg":"OK","tags":[],"resp_fuids":["FJDyMC15lxUn5ngPfd"],"resp_mime_types":["text/html"]}}
-{"http":{"ts":1402307733473,"uid":"KIRAN12312312","id.orig_h":"192.249.113.37","id.orig_p":58808,"id.resp_h":"72.163.4.161","id.resp_p":80,"trans_depth":1,"method":"GET","host":"www.cisco.com","uri":"/","user_agent":"curl/7.22.0 (x86_64-pc-linux-gnu) libcurl/7.22.0 OpenSSL/1.0.1 zlib/1.2.3.4 libidn/1.23 librtmp/2.3","request_body_len":0,"response_body_len":25523,"status_code":200,"status_msg":"OK","tags":[],"resp_fuids":["FJDyMC15lxUn5ngPfd"],"resp_mime_types":["text/html"]}}
-{"http":{"ts":1402307733473,"uid":"CTo78A11g7CYbbOHvj","id.orig_h":"10.122.196.204","id.orig_p":58808,"id.resp_h":"72.163.4.161","id.resp_p":80,"trans_depth":1,"email":"abullis@mail.csuchico.edu","method":"GET","host":"gabacentre.pw","uri":"/","user_agent":"curl/7.22.0 (x86_64-pc-linux-gnu) libcurl/7.22.0 OpenSSL/1.0.1 zlib/1.2.3.4 libidn/1.23 librtmp/2.3","request_body_len":0,"response_body_len":25523,"status_code":200,"status_msg":"OK","tags":[],"resp_fuids":["FJDyMC15lxUn5ngPfd"],"resp_mime_types":["text/html"]}}
-{"dns":{"ts":1402308259609,"uid":"CYbbOHvj","id.orig_h":"93.188.160.43","id.orig_p":33976,"id.resp_h":"144.254.71.184","id.resp_p":53,"proto":"udp","trans_id":62418,"query":"www.cisco.com","qclass":1,"qclass_name":"C_INTERNET","qtype":28,"qtype_name":"AAAA","rcode":0,"rcode_name":"NOERROR","AA":true,"TC":false,"RD":true,"RA":true,"Z":0,"answers":["gabacentre.pw","www.cisco.com.akadns.net","origin-www.cisco.com","2001:420:1201:2::a"],"TTLs":[3600.0,289.0,14.0],"rejected":false}}
-{"http":{"ts":1402307733473,"uid":"CTo78A11g7CYbbOHvj","id.orig_h":"192.249.113.37","id.orig_p":58808,"id.resp_h":"72.163.4.161","id.resp_p":80,"trans_depth":1,"method":"GET","host":"www.cisco.com","uri":"/","user_agent":"curl/7.22.0 (x86_64-pc-linux-gnu) libcurl/7.22.0 OpenSSL/1.0.1 zlib/1.2.3.4 libidn/1.23 librtmp/2.3","request_body_len":0,"response_body_len":25523,"status_code":200,"status_msg":"OK","tags":[],"resp_fuids":["FJDyMC15lxUn5ngPfd"],"resp_mime_types":["text/html"]}}
-{"dns":{"ts":1402308259609,"uid":"CuJT272SKaJSuqO0Ia","id.orig_h":"10.122.196.204","id.orig_p":33976,"id.resp_h":"144.254.71.184","id.resp_p":53,"proto":"udp","trans_id":62418,"query":"www.cisco.com","qclass":1,"qclass_name":"C_INTERNET","qtype":28,"qtype_name":"AAAA","rcode":0,"rcode_name":"NOERROR","AA":true,"TC":false,"RD":true,"RA":true,"Z":0,"answers":["www.cisco.com.akadns.net","origin-www.cisco.com","2001:420:1201:2::a"],"TTLs":[3600.0,289.0,14.0],"rejected":false}}
-{"http":{"ts":1402307733473,"uid":"KIRAN","id.orig_h":"10.122.196.204","id.orig_p":58808,"id.resp_h":"72.163.4.161","id.resp_p":80,"trans_depth":1,"method":"GET","host":"www.cisco.com","uri":"/","user_agent":"curl/7.22.0 (x86_64-pc-linux-gnu) libcurl/7.22.0 OpenSSL/1.0.1 zlib/1.2.3.4 libidn/1.23 librtmp/2.3","request_body_len":0,"response_body_len":25523,"status_code":200,"status_msg":"OK","tags":[],"resp_fuids":["FJDyMC15lxUn5ngPfd"],"resp_mime_types":["text/html"]}}
-{"http":{"ts":1402307733473,"uid":"KIRAN12312312","id.orig_h":"192.249.113.37","id.orig_p":58808,"id.resp_h":"72.163.4.161","id.resp_p":80,"trans_depth":1,"method":"GET","host":"www.cisco.com","uri":"/","user_agent":"curl/7.22.0 (x86_64-pc-linux-gnu) libcurl/7.22.0 OpenSSL/1.0.1 zlib/1.2.3.4 libidn/1.23 librtmp/2.3","request_body_len":0,"response_body_len":25523,"status_code":200,"status_msg":"OK","tags":[],"resp_fuids":["FJDyMC15lxUn5ngPfd"],"resp_mime_types":["text/html"]}}
-{"http":{"ts":1402307733473,"uid":"KIRAN12312312","id.orig_h":"192.249.113.37","id.orig_p":58808,"id.resp_h":"72.163.4.161","id.resp_p":80,"trans_depth":1,"method":"GET","host":"www.cisco.com","uri":"/","user_agent":"curl/7.22.0 (x86_64-pc-linux-gnu) libcurl/7.22.0 OpenSSL/1.0.1 zlib/1.2.3.4 libidn/1.23 librtmp/2.3","request_body_len":0,"response_body_len":25523,"status_code":200,"status_msg":"OK","tags":[],"resp_fuids":["FJDyMC15lxUn5ngPfd"],"resp_mime_types":["text/html"]}}
-{"http":{"ts":1402307733473,"uid":"CTo78A11g7CYbbOHvj","id.orig_h":"10.122.196.204","id.orig_p":58808,"id.resp_h":"72.163.4.161","id.resp_p":80,"trans_depth":1,"email":"abullis@mail.csuchico.edu","method":"GET","host":"gabacentre.pw","uri":"/","user_agent":"curl/7.22.0 (x86_64-pc-linux-gnu) libcurl/7.22.0 OpenSSL/1.0.1 zlib/1.2.3.4 libidn/1.23 librtmp/2.3","request_body_len":0,"response_body_len":25523,"status_code":200,"status_msg":"OK","tags":[],"resp_fuids":["FJDyMC15lxUn5ngPfd"],"resp_mime_types":["text/html"]}}
-{"dns":{"ts":1402308259609,"uid":"CYbbOHvj","id.orig_h":"93.188.160.43","id.orig_p":33976,"id.resp_h":"144.254.71.184","id.resp_p":53,"proto":"udp","trans_id":62418,"query":"www.cisco.com","qclass":1,"qclass_name":"C_INTERNET","qtype":28,"qtype_name":"AAAA","rcode":0,"rcode_name":"NOERROR","AA":true,"TC":false,"RD":true,"RA":true,"Z":0,"answers":["gabacentre.pw","www.cisco.com.akadns.net","origin-www.cisco.com","2001:420:1201:2::a"],"TTLs":[3600.0,289.0,14.0],"rejected":false}}
-{"http":{"ts":1402307733473,"uid":"CTo78A11g7CYbbOHvj","id.orig_h":"192.249.113.37","id.orig_p":58808,"id.resp_h":"72.163.4.161","id.resp_p":80,"trans_depth":1,"method":"GET","host":"www.cisco.com","uri":"/","user_agent":"curl/7.22.0 (x86_64-pc-linux-gnu) libcurl/7.22.0 OpenSSL/1.0.1 zlib/1.2.3.4 libidn/1.23 librtmp/2.3","request_body_len":0,"response_body_len":25523,"status_code":200,"status_msg":"OK","tags":[],"resp_fuids":["FJDyMC15lxUn5ngPfd"],"resp_mime_types":["text/html"]}}
-{"dns":{"ts":1402308259609,"uid":"CuJT272SKaJSuqO0Ia","id.orig_h":"10.122.196.204","id.orig_p":33976,"id.resp_h":"144.254.71.184","id.resp_p":53,"proto":"udp","trans_id":62418,"query":"www.cisco.com","qclass":1,"qclass_name":"C_INTERNET","qtype":28,"qtype_name":"AAAA","rcode":0,"rcode_name":"NOERROR","AA":true,"TC":false,"RD":true,"RA":true,"Z":0,"answers":["www.cisco.com.akadns.net","origin-www.cisco.com","2001:420:1201:2::a"],"TTLs":[3600.0,289.0,14.0],"rejected":false}}
-{"http":{"ts":1402307733473,"uid":"KIRAN","id.orig_h":"10.122.196.204","id.orig_p":58808,"id.resp_h":"72.163.4.161","id.resp_p":80,"trans_depth":1,"method":"GET","host":"www.cisco.com","uri":"/","user_agent":"curl/7.22.0 (x86_64-pc-linux-gnu) libcurl/7.22.0 OpenSSL/1.0.1 zlib/1.2.3.4 libidn/1.23 librtmp/2.3","request_body_len":0,"response_body_len":25523,"status_code":200,"status_msg":"OK","tags":[],"resp_fuids":["FJDyMC15lxUn5ngPfd"],"resp_mime_types":["text/html"]}}
-{"http":{"ts":1402307733473,"uid":"KIRAN12312312","id.orig_h":"192.249.113.37","id.orig_p":58808,"id.resp_h":"72.163.4.161","id.resp_p":80,"trans_depth":1,"method":"GET","host":"www.cisco.com","uri":"/","user_agent":"curl/7.22.0 (x86_64-pc-linux-gnu) libcurl/7.22.0 OpenSSL/1.0.1 zlib/1.2.3.4 libidn/1.23 librtmp/2.3","request_body_len":0,"response_body_len":25523,"status_code":200,"status_msg":"OK","tags":[],"resp_fuids":["FJDyMC15lxUn5ngPfd"],"resp_mime_types":["text/html"]}}
-{"http":{"ts":1402307733473,"uid":"KIRAN12312312","id.orig_h":"192.249.113.37","id.orig_p":58808,"id.resp_h":"72.163.4.161","id.resp_p":80,"trans_depth":1,"method":"GET","host":"www.cisco.com","uri":"/","user_agent":"curl/7.22.0 (x86_64-pc-linux-gnu) libcurl/7.22.0 OpenSSL/1.0.1 zlib/1.2.3.4 libidn/1.23 librtmp/2.3","request_body_len":0,"response_body_len":25523,"status_code":200,"status_msg":"OK","tags":[],"resp_fuids":["FJDyMC15lxUn5ngPfd"],"resp_mime_types":["text/html"]}}
-{"http":{"ts":1402307733473,"uid":"CTo78A11g7CYbbOHvj","id.orig_h":"10.122.196.204","id.orig_p":58808,"id.resp_h":"72.163.4.161","id.resp_p":80,"trans_depth":1,"email":"abullis@mail.csuchico.edu","method":"GET","host":"gabacentre.pw","uri":"/","user_agent":"curl/7.22.0 (x86_64-pc-linux-gnu) libcurl/7.22.0 OpenSSL/1.0.1 zlib/1.2.3.4 libidn/1.23 librtmp/2.3","request_body_len":0,"response_body_len":25523,"status_code":200,"status_msg":"OK","tags":[],"resp_fuids":["FJDyMC15lxUn5ngPfd"],"resp_mime_types":["text/html"]}}
-{"dns":{"ts":1402308259609,"uid":"CYbbOHvj","id.orig_h":"93.188.160.43","id.orig_p":33976,"id.resp_h":"144.254.71.184","id.resp_p":53,"proto":"udp","trans_id":62418,"query":"www.cisco.com","qclass":1,"qclass_name":"C_INTERNET","qtype":28,"qtype_name":"AAAA","rcode":0,"rcode_name":"NOERROR","AA":true,"TC":false,"RD":true,"RA":true,"Z":0,"answers":["gabacentre.pw","www.cisco.com.akadns.net","origin-www.cisco.com","2001:420:1201:2::a"],"TTLs":[3600.0,289.0,14.0],"rejected":false}}
-{"http":{"ts":1402307733473,"uid":"CTo78A11g7CYbbOHvj","id.orig_h":"192.249.113.37","id.orig_p":58808,"id.resp_h":"72.163.4.161","id.resp_p":80,"trans_depth":1,"method":"GET","host":"www.cisco.com","uri":"/","user_agent":"curl/7.22.0 (x86_64-pc-linux-gnu) libcurl/7.22.0 OpenSSL/1.0.1 zlib/1.2.3.4 libidn/1.23 librtmp/2.3","request_body_len":0,"response_body_len":25523,"status_code":200,"status_msg":"OK","tags":[],"resp_fuids":["FJDyMC15lxUn5ngPfd"],"resp_mime_types":["text/html"]}}
-{"dns":{"ts":1402308259609,"uid":"CuJT272SKaJSuqO0Ia","id.orig_h":"10.122.196.204","id.orig_p":33976,"id.resp_h":"144.254.71.184","id.resp_p":53,"proto":"udp","trans_id":62418,"query":"www.cisco.com","qclass":1,"qclass_name":"C_INTERNET","qtype":28,"qtype_name":"AAAA","rcode":0,"rcode_name":"NOERROR","AA":true,"TC":false,"RD":true,"RA":true,"Z":0,"answers":["www.cisco.com.akadns.net","origin-www.cisco.com","2001:420:1201:2::a"],"TTLs":[3600.0,289.0,14.0],"rejected":false}}
-{"http":{"ts":1402307733473,"uid":"KIRAN","id.orig_h":"10.122.196.204","id.orig_p":58808,"id.resp_h":"72.163.4.161","id.resp_p":80,"trans_depth":1,"method":"GET","host":"www.cisco.com","uri":"/","user_agent":"curl/7.22.0 (x86_64-pc-linux-gnu) libcurl/7.22.0 OpenSSL/1.0.1 zlib/1.2.3.4 libidn/1.23 librtmp/2.3","request_body_len":0,"response_body_len":25523,"status_code":200,"status_msg":"OK","tags":[],"resp_fuids":["FJDyMC15lxUn5ngPfd"],"resp_mime_types":["text/html"]}}
-{"http":{"ts":1402307733473,"uid":"KIRAN12312312","id.orig_h":"192.249.113.37","id.orig_p":58808,"id.resp_h":"72.163.4.161","id.resp_p":80,"trans_depth":1,"method":"GET","host":"www.cisco.com","uri":"/","user_agent":"curl/7.22.0 (x86_64-pc-linux-gnu) libcurl/7.22.0 OpenSSL/1.0.1 zlib/1.2.3.4 libidn/1.23 librtmp/2.3","request_body_len":0,"response_body_len":25523,"status_code":200,"status_msg":"OK","tags":[],"resp_fuids":["FJDyMC15lxUn5ngPfd"],"resp_mime_types":["text/html"]}}
-{"http":{"ts":1402307733473,"uid":"KIRAN12312312","id.orig_h":"192.249.113.37","id.orig_p":58808,"id.resp_h":"72.163.4.161","id.resp_p":80,"trans_depth":1,"method":"GET","host":"www.cisco.com","uri":"/","user_agent":"curl/7.22.0 (x86_64-pc-linux-gnu) libcurl/7.22.0 OpenSSL/1.0.1 zlib/1.2.3.4 libidn/1.23 librtmp/2.3","request_body_len":0,"response_body_len":25523,"status_code":200,"status_msg":"OK","tags":[],"resp_fuids":["FJDyMC15lxUn5ngPfd"],"resp_mime_types":["text/html"]}}
-{"http":{"ts":1402307733473,"uid":"CTo78A11g7CYbbOHvj","id.orig_h":"10.122.196.204","id.orig_p":58808,"id.resp_h":"72.163.4.161","id.resp_p":80,"trans_depth":1,"email":"abullis@mail.csuchico.edu","method":"GET","host":"gabacentre.pw","uri":"/","user_agent":"curl/7.22.0 (x86_64-pc-linux-gnu) libcurl/7.22.0 OpenSSL/1.0.1 zlib/1.2.3.4 libidn/1.23 librtmp/2.3","request_body_len":0,"response_body_len":25523,"status_code":200,"status_msg":"OK","tags":[],"resp_fuids":["FJDyMC15lxUn5ngPfd"],"resp_mime_types":["text/html"]}}
-{"dns":{"ts":1402308259609,"uid":"CYbbOHvj","id.orig_h":"93.188.160.43","id.orig_p":33976,"id.resp_h":"144.254.71.184","id.resp_p":53,"proto":"udp","trans_id":62418,"query":"www.cisco.com","qclass":1,"qclass_name":"C_INTERNET","qtype":28,"qtype_name":"AAAA","rcode":0,"rcode_name":"NOERROR","AA":true,"TC":false,"RD":true,"RA":true,"Z":0,"answers":["gabacentre.pw","www.cisco.com.akadns.net","origin-www.cisco.com","2001:420:1201:2::a"],"TTLs":[3600.0,289.0,14.0],"rejected":false}}
-{"http":{"ts":1402307733473,"uid":"CTo78A11g7CYbbOHvj","id.orig_h":"192.249.113.37","id.orig_p":58808,"id.resp_h":"72.163.4.161","id.resp_p":80,"trans_depth"

<TRUNCATED>


[11/15] incubator-metron git commit: METRON-142 Simplify Parser configuration (merrimanr via cestella) closes apache/incubator-metron#120

Posted by ce...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/df8d682e/metron-platform/metron-integration-test/src/main/resources/sample/data/SampleInput/FireeyeExampleOutput
----------------------------------------------------------------------
diff --git a/metron-platform/metron-integration-test/src/main/resources/sample/data/SampleInput/FireeyeExampleOutput b/metron-platform/metron-integration-test/src/main/resources/sample/data/SampleInput/FireeyeExampleOutput
deleted file mode 100644
index 0210010..0000000
--- a/metron-platform/metron-integration-test/src/main/resources/sample/data/SampleInput/FireeyeExampleOutput
+++ /dev/null
@@ -1,90 +0,0 @@
-<164>fenotify-3483808.2.alert: 1::~~User-Agent: WinHttpClient::~~Host: www.microads.me::~~Connection: Keep-Alive::~~::~~GET /files/microads/update/InjectScript.js HTTP/1.1::~~User-Agent: WinHttpClient::~~Host: www.microads.me::~~Connection: Keep-Alive::~~::~~GET /files/microads/update/InjectScript.js HTTP/1.1::~~User-Agent: WinHttpClient::~~Host: www.microads.me::~~Connection: Keep-Alive::~~::~~GET /files/microads/update/InjectScript.js HTTP/1.1::~~User-Agent: WinHttpClient::~~Host: www.microads.me::~~Connection: Keep-Alive::~~::~~GET /files/microads/update/InjectScript.js HTTP/1.1::~~User-Agent: WinHttpClient::~~Host: www.microads.me::~~Connection: Keep-Alive::~~::~~GET /files/microads/update/InjectScript.js HTTP/1.1::~~User-Agent: WinHttpClient::~~Host: www.microads.me::~~Connection: Keep-Alive::~~::~~GET /files/microads/update/InjectScript.js HTTP/1.1::~~User-Agent: WinHttpClient::~~Host: www.microads.me::~~Connection: Keep-Alive::~~::~~GET /files/microads/update/InjectScript.js 
 HTTP
-<164>fenotify-793972.2.alert: ontrol: no-cache::~~::~~ dmac=00:1d:a2:af:32:a1 cs1Label=sname cs1=Exploit.Kit.Magnitude 
-<164>fenotify-797180.2.alert: 0.8::~~User-Agent: Mozilla/5.0 (Windows NT 6.1; WOW64) AppleWebKit/537.36 (KHTML, like Gecko) Chrome/39.0.2171.95 Safari/537.36::~~Accept-Encoding: gzip, deflate, sdch::~~Accept-Language: en-US,en;q\=0.8::~~::~~ dmac=00:1d:a2:af:32:a1 cs1Label=sname cs1=Trojan.Redirector 
-<164>fenotify-3483808.3.alert: /1.1::~~User-Agent: WinHttpClient::~~Host: www.microads.me::~~Connection: Keep-Alive::~~::~~GET /files/microads/update/InjectScript.js HTTP/1.1::~~User-Agent: WinHttpClient::~~Host: www.microads.me::~~Connection: Keep-Alive::~~::~~GET /files/microads/update/InjectScript.js HTTP/1.1::~~User-Agent: WinHttpClient::~~Host: www.microads.me::~~Connection: Keep-Alive::~~::~~GET /files/microads/update/InjectScript.js HTTP/1.1::~~User-Agent: WinHttpClient::~~Host: www.microads.me::~~Connection: Keep-Alive::~~::~~GET /files/microads/update/InjectScript.js HTTP/1.1::~~User-Agent: WinHttpClient::~~Host: www.microads.me::~~Connection: Keep-Alive::~~::~~GET /files/microads/update/InjectScript.js HTTP/1.1::~~User-Agent: WinHttpClient::~~Host: www.microad cs2Label=anomaly cs2=misc-anomaly cs1Label=sname cs1=Malware.Binary 
-<164>fenotify-791429.2.alert: t: rapidvideohere.pw::~~Connection: Keep-Alive::~~::~~ dmac=88:43:e1:95:13:29 cs1Label=sname cs1=Exploit.Kit.Magnitude 
-<164>fenotify-851777.alert: CEF:0|FireEye|CMS|7.2.1.244420|DM|domain-match|1|rt=Feb 09 2015 00:27:43 UTC dvc=10.201.78.190 cn3Label=cncPort cn3=53 cn2Label=sid cn2=80494706 shost=dev001srv02.example.com proto=udp cs5Label=cncHost cs5=mfdclk001.org dvchost=DEVFEYE1 spt=61395 dvc=10.100.25.16 smac=00:00:0c:07:ac:00 cn1Label=vlan cn1=0 externalId=851777 cs4Label=link cs4=https://DEVCMS01.example.com/event_stream/events_for_bot?ev_id\=851777 dmac=00:1d:a2:af:32:a1 cs1Label=sname cs1=Trojan.Generic.DNS 
-<164>fenotify-851901.alert: CEF:0|FireEye|CMS|7.2.1.244420|DM|domain-match|1|rt=Feb 09 2015 08:56:45 UTC dvc=10.201.78.6 cn3Label=cncPort cn3=53 cn2Label=sid cn2=80494706 shost=dev001srv02.example.com proto=udp cs5Label=cncHost cs5=mfdclk001.org dvchost=DEVFEYE1 spt=59131 dvc=10.100.25.16 smac=00:00:0c:07:ac:c8 cn1Label=vlan cn1=0 externalId=851901 cs4Label=link cs4=https://DEVCMS01.example.com/event_stream/events_for_bot?ev_id\=851901 dmac=88:43:e1:95:13:29 cs1Label=sname cs1=Trojan.Generic.DNS 
-<164>fenotify-851980.alert: CEF:0|FireEye|CMS|7.2.1.244420|DM|domain-match|1|rt=Feb 09 2015 12:23:51 UTC dvc=10.201.78.20 cn3Label=cncPort cn3=53 cn2Label=sid cn2=80494706 shost=dev001srv02.example.com proto=udp cs5Label=cncHost cs5=mfdclk001.org dvchost=DEVFEYE1 spt=53295 dvc=10.100.25.16 smac=00:00:0c:07:ac:00 cn1Label=vlan cn1=0 externalId=851980 cs4Label=link cs4=https://DEVCMS01.example.com/event_stream/events_for_bot?ev_id\=851980 dmac=00:1d:a2:af:32:a1 cs1Label=sname cs1=Trojan.Generic.DNS 
-<164>fenotify-851795.alert: CEF:0|FireEye|CMS|7.2.1.244420|DM|domain-match|1|rt=Feb 09 2015 02:19:05 UTC dvc=10.201.78.37 cn3Label=cncPort cn3=53 cn2Label=sid cn2=80494706 shost=dev001srv02.example.com proto=udp cs5Label=cncHost cs5=mfdclk001.org dvchost=DEVFEYE1 spt=54975 dvc=10.100.25.16 smac=00:00:0c:07:ac:00 cn1Label=vlan cn1=0 externalId=851795 cs4Label=link cs4=https://DEVCMS01.example.com/event_stream/events_for_bot?ev_id\=851795 dmac=00:1d:a2:af:32:a1 cs1Label=sname cs1=Trojan.Generic.DNS 
-<164>fenotify-851805.alert: CEF:0|FireEye|CMS|7.2.1.244420|DM|domain-match|1|rt=Feb 09 2015 03:23:14 UTC dvc=10.201.78.113 cn3Label=cncPort cn3=53 cn2Label=sid cn2=80494706 shost=dev001srv02.example.com proto=udp cs5Label=cncHost cs5=mfdclk001.org dvchost=DEVFEYE1 spt=50807 dvc=10.100.25.16 smac=00:00:0c:07:ac:00 cn1Label=vlan cn1=0 externalId=851805 cs4Label=link cs4=https://DEVCMS01.example.com/event_stream/events_for_bot?ev_id\=851805 dmac=00:1d:a2:af:32:a1 cs1Label=sname cs1=Trojan.Generic.DNS 
-<164>fenotify-851844.alert: CEF:0|FireEye|CMS|7.2.1.244420|DM|domain-match|1|rt=Feb 09 2015 06:19:41 UTC dvc=10.201.78.59 cn3Label=cncPort cn3=53 cn2Label=sid cn2=80494706 shost=dev001srv02.example.com proto=udp cs5Label=cncHost cs5=mfdclk001.org dvchost=DEVFEYE1 spt=50767 dvc=10.100.25.16 smac=00:00:0c:07:ac:00 cn1Label=vlan cn1=0 externalId=851844 cs4Label=link cs4=https://DEVCMS01.example.com/event_stream/events_for_bot?ev_id\=851844 dmac=00:1d:a2:af:32:a1 cs1Label=sname cs1=Trojan.Generic.DNS 
-<164>fenotify-851782.alert: CEF:0|FireEye|CMS|7.2.1.244420|DM|domain-match|1|rt=Feb 09 2015 01:18:22 UTC dvc=10.201.78.59 cn3Label=cncPort cn3=53 cn2Label=sid cn2=80494706 shost=dev001srv02.example.com proto=udp cs5Label=cncHost cs5=mfdclk001.org dvchost=DEVFEYE1 spt=50940 dvc=10.100.25.16 smac=00:00:0c:07:ac:00 cn1Label=vlan cn1=0 externalId=851782 cs4Label=link cs4=https://DEVCMS01.example.com/event_stream/events_for_bot?ev_id\=851782 dmac=00:1d:a2:af:32:a1 cs1Label=sname cs1=Trojan.Generic.DNS 
-<164>fenotify-851940.alert: CEF:0|FireEye|CMS|7.2.1.244420|DM|domain-match|1|rt=Feb 09 2015 10:57:19 UTC dvc=10.201.78.85 cn3Label=cncPort cn3=53 cn2Label=sid cn2=80494706 shost=dev001srv02.example.com proto=udp cs5Label=cncHost cs5=mfdclk001.org dvchost=DEVFEYE1 spt=50646 dvc=10.100.25.16 smac=00:00:0c:07:ac:c8 cn1Label=vlan cn1=0 externalId=851940 cs4Label=link cs4=https://DEVCMS01.example.com/event_stream/events_for_bot?ev_id\=851940 dmac=88:43:e1:95:13:29 cs1Label=sname cs1=Trojan.Generic.DNS 
-<164>fenotify-851881.alert: CEF:0|FireEye|CMS|7.2.1.244420|DM|domain-match|1|rt=Feb 09 2015 08:13:15 UTC dvc=10.201.78.84 cn3Label=cncPort cn3=53 cn2Label=sid cn2=80494706 shost=dev001srv02.example.com proto=udp cs5Label=cncHost cs5=mfdclk001.org dvchost=DEVFEYE1 spt=61237 dvc=10.100.25.16 smac=00:00:0c:07:ac:c8 cn1Label=vlan cn1=0 externalId=851881 cs4Label=link cs4=https://DEVCMS01.example.com/event_stream/events_for_bot?ev_id\=851881 dmac=88:43:e1:95:13:29 cs1Label=sname cs1=Trojan.Generic.DNS 
-<164>fenotify-851839.alert: CEF:0|FireEye|CMS|7.2.1.244420|DM|domain-match|1|rt=Feb 09 2015 05:33:19 UTC dvc=10.201.78.10 cn3Label=cncPort cn3=53 cn2Label=sid cn2=80494706 proto=udp cs5Label=cncHost cs5=mfdclk001.org dvchost=DEVFEYE1 spt=49186 dvc=10.100.25.5 smac=00:00:0c:07:ac:63 cn1Label=vlan cn1=143 externalId=851839 cs4Label=link cs4=https://DEVCMS01.example.com/event_stream/events_for_bot?ev_id\=851839 dmac=00:1b:17:00:09:01 cs1Label=sname cs1=Trojan.Generic.DNS 
-<164>fenotify-851983.alert: CEF:0|FireEye|CMS|7.2.1.244420|DM|domain-match|1|rt=Feb 09 2015 12:28:26 UTC dvc=10.201.78.57 cn3Label=cncPort cn3=53 cn2Label=sid cn2=80494706 shost=dev001srv02.example.com proto=udp cs5Label=cncHost cs5=mfdclk001.org dvchost=DEVFEYE1 spt=54527 dvc=10.100.25.16 smac=00:00:0c:07:ac:00 cn1Label=vlan cn1=0 externalId=851983 cs4Label=link cs4=https://DEVCMS01.example.com/event_stream/events_for_bot?ev_id\=851983 dmac=00:1d:a2:af:32:a1 cs1Label=sname cs1=Trojan.Generic.DNS 
-<164>fenotify-851987.alert: CEF:0|FireEye|CMS|7.2.1.244420|DM|domain-match|1|rt=Feb 09 2015 12:33:41 UTC dvc=10.201.78.113 cn3Label=cncPort cn3=53 cn2Label=sid cn2=80494706 shost=dev001srv02.example.com proto=udp cs5Label=cncHost cs5=mfdclk001.org dvchost=DEVFEYE1 spt=51218 dvc=10.100.25.16 smac=00:00:0c:07:ac:00 cn1Label=vlan cn1=0 externalId=851987 cs4Label=link cs4=https://DEVCMS01.example.com/event_stream/events_for_bot?ev_id\=851987 dmac=00:1d:a2:af:32:a1 cs1Label=sname cs1=Trojan.Generic.DNS 
-<164>fenotify-852010.alert: CEF:0|FireEye|CMS|7.2.1.244420|DM|domain-match|1|rt=Feb 09 2015 13:15:08 UTC dvc=10.201.78.12 cn3Label=cncPort cn3=53 cn2Label=sid cn2=80494706 shost=dev001srv02.example.com proto=udp cs5Label=cncHost cs5=mfdclk001.org dvchost=DEVFEYE1 spt=55203 dvc=10.100.25.16 smac=00:00:0c:07:ac:c8 cn1Label=vlan cn1=0 externalId=852010 cs4Label=link cs4=https://DEVCMS01.example.com/event_stream/events_for_bot?ev_id\=852010 dmac=88:43:e1:95:13:29 cs1Label=sname cs1=Trojan.Generic.DNS 
-<164>fenotify-852053.alert: CEF:0|FireEye|CMS|7.2.1.244420|DM|domain-match|1|rt=Feb 09 2015 14:16:45 UTC dvc=10.201.78.84 cn3Label=cncPort cn3=53 cn2Label=sid cn2=80494706 shost=dev001srv02.example.com proto=udp cs5Label=cncHost cs5=mfdclk001.org dvchost=DEVFEYE1 spt=62235 dvc=10.100.25.16 smac=00:00:0c:07:ac:c8 cn1Label=vlan cn1=0 externalId=852053 cs4Label=link cs4=https://DEVCMS01.example.com/event_stream/events_for_bot?ev_id\=852053 dmac=88:43:e1:95:13:29 cs1Label=sname cs1=Trojan.Generic.DNS 
-<164>fenotify-852455.alert: CEF:0|FireEye|CMS|7.2.1.244420|DM|domain-match|1|rt=Feb 09 2015 17:28:38 UTC dvc=10.201.78.34 cn3Label=cncPort cn3=53 cn2Label=sid cn2=80494706 shost=dev001srv02.example.com proto=udp cs5Label=cncHost cs5=mfdclk001.org dvchost=DEVFEYE1 spt=65175 dvc=10.100.25.16 smac=00:00:0c:07:ac:c8 cn1Label=vlan cn1=0 externalId=852455 cs4Label=link cs4=https://DEVCMS01.example.com/event_stream/events_for_bot?ev_id\=852455 dmac=88:43:e1:95:13:29 cs1Label=sname cs1=Trojan.Generic.DNS 
-<164>fenotify-851887.alert: CEF:0|FireEye|CMS|7.2.1.244420|DM|domain-match|1|rt=Feb 09 2015 08:24:54 UTC dvc=10.201.78.44 cn3Label=cncPort cn3=53 cn2Label=sid cn2=80494706 shost=dev001srv02.example.com proto=udp cs5Label=cncHost cs5=mfdclk001.org dvchost=DEVFEYE1 spt=56334 dvc=10.100.25.16 smac=00:00:0c:07:ac:c8 cn1Label=vlan cn1=0 externalId=851887 cs4Label=link cs4=https://DEVCMS01.example.com/event_stream/events_for_bot?ev_id\=851887 dmac=88:43:e1:95:13:29 cs1Label=sname cs1=Trojan.Generic.DNS 
-<164>fenotify-851822.alert: CEF:0|FireEye|CMS|7.2.1.244420|DM|domain-match|1|rt=Feb 09 2015 04:41:49 UTC dvc=10.201.78.54 cn3Label=cncPort cn3=53 cn2Label=sid cn2=80494706 shost=dev001srv02.example.com proto=udp cs5Label=cncHost cs5=mfdclk001.org dvchost=DEVFEYE1 spt=49732 dvc=10.100.25.16 smac=00:00:0c:07:ac:c8 cn1Label=vlan cn1=0 externalId=851822 cs4Label=link cs4=https://DEVCMS01.example.com/event_stream/events_for_bot?ev_id\=851822 dmac=88:43:e1:95:13:29 cs1Label=sname cs1=Trojan.Generic.DNS 
-<164>fenotify-851832.alert: CEF:0|FireEye|CMS|7.2.1.244420|DM|domain-match|1|rt=Feb 09 2015 05:19:15 UTC dvc=10.201.78.160 cn3Label=cncPort cn3=53 cn2Label=sid cn2=80494706 shost=dev001srv02.example.com proto=udp cs5Label=cncHost cs5=mfdclk001.org dvchost=DEVFEYE1 spt=62962 dvc=10.100.25.16 smac=00:00:0c:07:ac:00 cn1Label=vlan cn1=0 externalId=851832 cs4Label=link cs4=https://DEVCMS01.example.com/event_stream/events_for_bot?ev_id\=851832 dmac=00:1d:a2:af:32:a1 cs1Label=sname cs1=Trojan.Generic.DNS 
-<164>fenotify-851780.alert: CEF:0|FireEye|CMS|7.2.1.244420|DM|domain-match|1|rt=Feb 09 2015 00:56:46 UTC dvc=10.201.78.12 cn3Label=cncPort cn3=53 cn2Label=sid cn2=80494706 shost=dev001srv02.example.com proto=udp cs5Label=cncHost cs5=mfdclk001.org dvchost=DEVFEYE1 spt=54301 dvc=10.100.25.16 smac=00:00:0c:07:ac:c8 cn1Label=vlan cn1=0 externalId=851780 cs4Label=link cs4=https://DEVCMS01.example.com/event_stream/events_for_bot?ev_id\=851780 dmac=88:43:e1:95:13:29 cs1Label=sname cs1=Trojan.Generic.DNS 
-<164>fenotify-851792.alert: CEF:0|FireEye|CMS|7.2.1.244420|DM|domain-match|1|rt=Feb 09 2015 02:15:06 UTC dvc=10.201.78.194 cn3Label=cncPort cn3=53 cn2Label=sid cn2=80494706 shost=dev001srv02.example.com proto=udp cs5Label=cncHost cs5=mfdclk001.org dvchost=DEVFEYE1 spt=64831 dvc=10.100.25.16 smac=00:00:0c:07:ac:c8 cn1Label=vlan cn1=0 externalId=851792 cs4Label=link cs4=https://DEVCMS01.example.com/event_stream/events_for_bot?ev_id\=851792 dmac=88:43:e1:95:13:29 cs1Label=sname cs1=Trojan.Generic.DNS 
-<164>fenotify-851806.alert: CEF:0|FireEye|CMS|7.2.1.244420|DM|domain-match|1|rt=Feb 09 2015 03:24:05 UTC dvc=10.201.78.57 cn3Label=cncPort cn3=53 cn2Label=sid cn2=80494706 shost=dev001srv02.example.com proto=udp cs5Label=cncHost cs5=mfdclk001.org dvchost=DEVFEYE1 spt=53417 dvc=10.100.25.16 smac=00:00:0c:07:ac:00 cn1Label=vlan cn1=0 externalId=851806 cs4Label=link cs4=https://DEVCMS01.example.com/event_stream/events_for_bot?ev_id\=851806 dmac=00:1d:a2:af:32:a1 cs1Label=sname cs1=Trojan.Generic.DNS 
-<164>fenotify-851840.alert: CEF:0|FireEye|CMS|7.2.1.244420|DM|domain-match|1|rt=Feb 09 2015 06:00:58 UTC dvc=10.201.78.40 cn3Label=cncPort cn3=53 cn2Label=sid cn2=80494706 shost=dev001srv02.example.com proto=udp cs5Label=cncHost cs5=mfdclk001.org dvchost=DEVFEYE1 spt=50709 dvc=10.100.25.16 smac=00:00:0c:07:ac:c8 cn1Label=vlan cn1=0 externalId=851840 cs4Label=link cs4=https://DEVCMS01.example.com/event_stream/events_for_bot?ev_id\=851840 dmac=88:43:e1:95:13:29 cs1Label=sname cs1=Trojan.Generic.DNS 
-<164>fenotify-851929.alert: CEF:0|FireEye|CMS|7.2.1.244420|DM|domain-match|1|rt=Feb 09 2015 10:37:14 UTC dvc=10.201.78.87 cn3Label=cncPort cn3=53 cn2Label=sid cn2=80494706 shost=dev001srv02.example.com proto=udp cs5Label=cncHost cs5=mfdclk001.org dvchost=DEVFEYE1 spt=62909 dvc=10.100.25.16 smac=00:00:0c:07:ac:c8 cn1Label=vlan cn1=0 externalId=851929 cs4Label=link cs4=https://DEVCMS01.example.com/event_stream/events_for_bot?ev_id\=851929 dmac=88:43:e1:95:13:29 cs1Label=sname cs1=Trojan.Generic.DNS 
-<164>fenotify-851918.alert: CEF:0|FireEye|CMS|7.2.1.244420|DM|domain-match|1|rt=Feb 09 2015 10:17:41 UTC dvc=10.201.78.34 cn3Label=cncPort cn3=53 cn2Label=sid cn2=80494706 proto=udp cs5Label=cncHost cs5=mfdclk001.org dvchost=DEVFEYE1 spt=63483 dvc=10.100.25.5 smac=00:00:0c:07:ac:63 cn1Label=vlan cn1=143 externalId=851918 cs4Label=link cs4=https://DEVCMS01.example.com/event_stream/events_for_bot?ev_id\=851918 dmac=00:1b:17:00:09:01 cs1Label=sname cs1=Trojan.Generic.DNS 
-<164>fenotify-851842.alert: CEF:0|FireEye|CMS|7.2.1.244420|DM|domain-match|1|rt=Feb 09 2015 06:03:05 UTC dvc=10.201.78.68 cn3Label=cncPort cn3=53 cn2Label=sid cn2=80494706 proto=udp cs5Label=cncHost cs5=mfdclk001.org dvchost=DEVFEYE1 spt=59908 dvc=10.100.25.16 smac=00:00:0c:07:ac:5a cn1Label=vlan cn1=0 externalId=851842 cs4Label=link cs4=https://DEVCMS01.example.com/event_stream/events_for_bot?ev_id\=851842 dmac=00:09:0f:33:4f:48 cs1Label=sname cs1=Trojan.Generic.DNS 
-<164>fenotify-851948.alert: CEF:0|FireEye|CMS|7.2.1.244420|DM|domain-match|1|rt=Feb 09 2015 11:13:18 UTC dvc=10.201.78.86 cn3Label=cncPort cn3=53 cn2Label=sid cn2=80494706 shost=dev001srv02.example.com proto=udp cs5Label=cncHost cs5=mfdclk001.org dvchost=DEVFEYE1 spt=51327 dvc=10.100.25.16 smac=00:00:0c:07:ac:c8 cn1Label=vlan cn1=0 externalId=851948 cs4Label=link cs4=https://DEVCMS01.example.com/event_stream/events_for_bot?ev_id\=851948 dmac=88:43:e1:95:13:29 cs1Label=sname cs1=Trojan.Generic.DNS 
-<164>fenotify-852008.alert: CEF:0|FireEye|CMS|7.2.1.244420|DM|domain-match|1|rt=Feb 09 2015 13:13:25 UTC dvc=10.201.78.20 cn3Label=cncPort cn3=53 cn2Label=sid cn2=80494706 shost=dev001srv02.example.com proto=udp cs5Label=cncHost cs5=mfdclk001.org dvchost=DEVFEYE1 spt=63619 dvc=10.100.25.16 smac=00:00:0c:07:ac:c8 cn1Label=vlan cn1=0 externalId=852008 cs4Label=link cs4=https://DEVCMS01.example.com/event_stream/events_for_bot?ev_id\=852008 dmac=88:43:e1:95:13:29 cs1Label=sname cs1=Trojan.Generic.DNS 
-<164>fenotify-852072.alert: CEF:0|FireEye|CMS|7.2.1.244420|DM|domain-match|1|rt=Feb 09 2015 14:30:09 UTC dvc=10.201.78.37 cn3Label=cncPort cn3=53 cn2Label=sid cn2=80494706 shost=dev001srv02.example.com proto=udp cs5Label=cncHost cs5=mfdclk001.org dvchost=DEVFEYE1 spt=53467 dvc=10.100.25.16 smac=00:00:0c:07:ac:00 cn1Label=vlan cn1=0 externalId=852072 cs4Label=link cs4=https://DEVCMS01.example.com/event_stream/events_for_bot?ev_id\=852072 dmac=00:1d:a2:af:32:a1 cs1Label=sname cs1=Trojan.Generic.DNS 
-<164>fenotify-852077.alert: CEF:0|FireEye|CMS|7.2.1.244420|DM|domain-match|1|rt=Feb 09 2015 14:31:58 UTC dvc=10.201.78.11 cn3Label=cncPort cn3=53 cn2Label=sid cn2=80494706 proto=udp cs5Label=cncHost cs5=mfdclk001.org dvchost=DEVFEYE1 spt=58546 dvc=10.100.25.5 smac=00:00:0c:07:ac:63 cn1Label=vlan cn1=143 externalId=852077 cs4Label=link cs4=https://DEVCMS01.example.com/event_stream/events_for_bot?ev_id\=852077 dmac=00:1b:17:00:09:01 cs1Label=sname cs1=Trojan.Generic.DNS 
-<164>fenotify-852110.alert: CEF:0|FireEye|CMS|7.2.1.244420|DM|domain-match|1|rt=Feb 09 2015 14:56:32 UTC dvc=10.201.78.160 cn3Label=cncPort cn3=53 cn2Label=sid cn2=80494706 shost=dev001srv02.example.com proto=udp cs5Label=cncHost cs5=mfdclk001.org dvchost=DEVFEYE1 spt=61983 dvc=10.100.25.16 smac=00:00:0c:07:ac:00 cn1Label=vlan cn1=0 externalId=852110 cs4Label=link cs4=https://DEVCMS01.example.com/event_stream/events_for_bot?ev_id\=852110 dmac=00:1d:a2:af:32:a1 cs1Label=sname cs1=Trojan.Generic.DNS 
-<164>fenotify-852378.alert: CEF:0|FireEye|CMS|7.2.1.244420|DM|domain-match|1|rt=Feb 09 2015 17:03:31 UTC dvc=10.201.78.85 cn3Label=cncPort cn3=53 cn2Label=sid cn2=80494706 shost=dev001srv02.example.com proto=udp cs5Label=cncHost cs5=mfdclk001.org dvchost=DEVFEYE1 spt=49942 dvc=10.100.25.16 smac=00:00:0c:07:ac:c8 cn1Label=vlan cn1=0 externalId=852378 cs4Label=link cs4=https://DEVCMS01.example.com/event_stream/events_for_bot?ev_id\=852378 dmac=88:43:e1:95:13:29 cs1Label=sname cs1=Trojan.Generic.DNS 
-<164>fenotify-851787.alert: CEF:0|FireEye|CMS|7.2.1.244420|DM|domain-match|1|rt=Feb 09 2015 01:57:21 UTC dvc=10.201.78.44 cn3Label=cncPort cn3=53 cn2Label=sid cn2=80494706 shost=dev001srv02.example.com proto=udp cs5Label=cncHost cs5=mfdclk001.org dvchost=DEVFEYE1 spt=55199 dvc=10.100.25.16 smac=00:00:0c:07:ac:c8 cn1Label=vlan cn1=0 externalId=851787 cs4Label=link cs4=https://DEVCMS01.example.com/event_stream/events_for_bot?ev_id\=851787 dmac=88:43:e1:95:13:29 cs1Label=sname cs1=Trojan.Generic.DNS 
-<164>fenotify-851800.alert: CEF:0|FireEye|CMS|7.2.1.244420|DM|domain-match|1|rt=Feb 09 2015 02:54:32 UTC dvc=10.201.78.34 cn3Label=cncPort cn3=53 cn2Label=sid cn2=80494706 shost=dev001srv02.example.com proto=udp cs5Label=cncHost cs5=mfdclk001.org dvchost=DEVFEYE1 spt=50605 dvc=10.100.25.16 smac=00:00:0c:07:ac:c8 cn1Label=vlan cn1=0 externalId=851800 cs4Label=link cs4=https://DEVCMS01.example.com/event_stream/events_for_bot?ev_id\=851800 dmac=88:43:e1:95:13:29 cs1Label=sname cs1=Trojan.Generic.DNS 
-<164>fenotify-851941.alert: CEF:0|FireEye|CMS|7.2.1.244420|DM|domain-match|1|rt=Feb 09 2015 10:58:30 UTC dvc=10.201.78.54 cn3Label=cncPort cn3=53 cn2Label=sid cn2=80494706 shost=dev001srv02.example.com proto=udp cs5Label=cncHost cs5=mfdclk001.org dvchost=DEVFEYE1 spt=51721 dvc=10.100.25.16 smac=00:00:0c:07:ac:c8 cn1Label=vlan cn1=0 externalId=851941 cs4Label=link cs4=https://DEVCMS01.example.com/event_stream/events_for_bot?ev_id\=851941 dmac=88:43:e1:95:13:29 cs1Label=sname cs1=Trojan.Generic.DNS 
-<164>fenotify-851850.alert: CEF:0|FireEye|CMS|7.2.1.244420|DM|domain-match|1|rt=Feb 09 2015 06:29:59 UTC dvc=10.201.78.113 cn3Label=cncPort cn3=53 cn2Label=sid cn2=80494706 shost=dev001srv02.example.com proto=udp cs5Label=cncHost cs5=mfdclk001.org dvchost=DEVFEYE1 spt=50606 dvc=10.100.25.16 smac=00:00:0c:07:ac:00 cn1Label=vlan cn1=0 externalId=851850 cs4Label=link cs4=https://DEVCMS01.example.com/event_stream/events_for_bot?ev_id\=851850 dmac=00:1d:a2:af:32:a1 cs1Label=sname cs1=Trojan.Generic.DNS 
-<164>fenotify-851885.alert: CEF:0|FireEye|CMS|7.2.1.244420|DM|domain-match|1|rt=Feb 09 2015 08:22:40 UTC dvc=10.201.78.37 cn3Label=cncPort cn3=53 cn2Label=sid cn2=80494706 shost=dev001srv02.example.com proto=udp cs5Label=cncHost cs5=mfdclk001.org dvchost=DEVFEYE1 spt=53481 dvc=10.100.25.16 smac=00:00:0c:07:ac:00 cn1Label=vlan cn1=0 externalId=851885 cs4Label=link cs4=https://DEVCMS01.example.com/event_stream/events_for_bot?ev_id\=851885 dmac=00:1d:a2:af:32:a1 cs1Label=sname cs1=Trojan.Generic.DNS 
-<164>fenotify-851801.alert: CEF:0|FireEye|CMS|7.2.1.244420|DM|domain-match|1|rt=Feb 09 2015 02:55:09 UTC dvc=10.201.78.6 cn3Label=cncPort cn3=53 cn2Label=sid cn2=80494706 shost=dev001srv02.example.com proto=udp cs5Label=cncHost cs5=mfdclk001.org dvchost=DEVFEYE1 spt=59875 dvc=10.100.25.16 smac=00:00:0c:07:ac:c8 cn1Label=vlan cn1=0 externalId=851801 cs4Label=link cs4=https://DEVCMS01.example.com/event_stream/events_for_bot?ev_id\=851801 dmac=88:43:e1:95:13:29 cs1Label=sname cs1=Trojan.Generic.DNS 
-<164>fenotify-851884.alert: CEF:0|FireEye|CMS|7.2.1.244420|DM|domain-match|1|rt=Feb 09 2015 08:20:10 UTC dvc=10.201.78.194 cn3Label=cncPort cn3=53 cn2Label=sid cn2=80494706 shost=dev001srv02.example.com proto=udp cs5Label=cncHost cs5=mfdclk001.org dvchost=DEVFEYE1 spt=50039 dvc=10.100.25.16 smac=00:00:0c:07:ac:c8 cn1Label=vlan cn1=0 externalId=851884 cs4Label=link cs4=https://DEVCMS01.example.com/event_stream/events_for_bot?ev_id\=851884 dmac=88:43:e1:95:13:29 cs1Label=sname cs1=Trojan.Generic.DNS 
-<164>fenotify-851815.alert: CEF:0|FireEye|CMS|7.2.1.244420|DM|domain-match|1|rt=Feb 09 2015 04:06:05 UTC dvc=10.201.78.11 cn3Label=cncPort cn3=53 cn2Label=sid cn2=80494706 shost=dev001srv02.example.com proto=udp cs5Label=cncHost cs5=mfdclk001.org dvchost=DEVFEYE1 spt=53889 dvc=10.100.25.16 smac=00:00:0c:07:ac:c8 cn1Label=vlan cn1=0 externalId=851815 cs4Label=link cs4=https://DEVCMS01.example.com/event_stream/events_for_bot?ev_id\=851815 dmac=88:43:e1:95:13:29 cs1Label=sname cs1=Trojan.Generic.DNS 
-<164>fenotify-851825.alert: CEF:0|FireEye|CMS|7.2.1.244420|DM|domain-match|1|rt=Feb 09 2015 04:49:07 UTC dvc=10.201.78.85 cn3Label=cncPort cn3=53 cn2Label=sid cn2=80494706 shost=dev001srv02.example.com proto=udp cs5Label=cncHost cs5=mfdclk001.org dvchost=DEVFEYE1 spt=51906 dvc=10.100.25.16 smac=00:00:0c:07:ac:c8 cn1Label=vlan cn1=0 externalId=851825 cs4Label=link cs4=https://DEVCMS01.example.com/event_stream/events_for_bot?ev_id\=851825 dmac=88:43:e1:95:13:29 cs1Label=sname cs1=Trojan.Generic.DNS 
-<164>fenotify-851966.alert: CEF:0|FireEye|CMS|7.2.1.244420|DM|domain-match|1|rt=Feb 09 2015 11:50:43 UTC dvc=10.201.78.10 cn3Label=cncPort cn3=53 cn2Label=sid cn2=80494706 proto=udp cs5Label=cncHost cs5=mfdclk001.org dvchost=DEVFEYE1 spt=50758 dvc=10.100.25.5 smac=00:00:0c:07:ac:63 cn1Label=vlan cn1=143 externalId=851966 cs4Label=link cs4=https://DEVCMS01.example.com/event_stream/events_for_bot?ev_id\=851966 dmac=00:1b:17:00:09:01 cs1Label=sname cs1=Trojan.Generic.DNS 
-<164>fenotify-852112.alert: CEF:0|FireEye|CMS|7.2.1.244420|DM|domain-match|1|rt=Feb 09 2015 14:58:20 UTC dvc=10.201.78.6 cn3Label=cncPort cn3=53 cn2Label=sid cn2=80494706 shost=dev001srv02.example.com proto=udp cs5Label=cncHost cs5=mfdclk001.org dvchost=DEVFEYE1 spt=60631 dvc=10.100.25.16 smac=00:00:0c:07:ac:c8 cn1Label=vlan cn1=0 externalId=852112 cs4Label=link cs4=https://DEVCMS01.example.com/event_stream/events_for_bot?ev_id\=852112 dmac=88:43:e1:95:13:29 cs1Label=sname cs1=Trojan.Generic.DNS 
-<164>fenotify-852126.alert: CEF:0|FireEye|CMS|7.2.1.244420|DM|domain-match|1|rt=Feb 09 2015 15:03:43 UTC dvc=10.201.78.60 cn3Label=cncPort cn3=53 cn2Label=sid cn2=80494706 shost=dev001srv02.example.com proto=udp cs5Label=cncHost cs5=mfdclk001.org dvchost=DEVFEYE1 spt=65017 dvc=10.100.25.16 smac=00:00:0c:07:ac:00 cn1Label=vlan cn1=0 externalId=852126 cs4Label=link cs4=https://DEVCMS01.example.com/event_stream/events_for_bot?ev_id\=852126 dmac=00:1d:a2:af:32:a1 cs1Label=sname cs1=Trojan.Generic.DNS 
-<164>fenotify-852407.alert: CEF:0|FireEye|CMS|7.2.1.244420|DM|domain-match|1|rt=Feb 09 2015 17:15:10 UTC dvc=10.201.78.54 cn3Label=cncPort cn3=53 cn2Label=sid cn2=80494706 shost=dev001srv02.example.com proto=udp cs5Label=cncHost cs5=mfdclk001.org dvchost=DEVFEYE1 spt=49620 dvc=10.100.25.16 smac=00:00:0c:07:ac:c8 cn1Label=vlan cn1=0 externalId=852407 cs4Label=link cs4=https://DEVCMS01.example.com/event_stream/events_for_bot?ev_id\=852407 dmac=88:43:e1:95:13:29 cs1Label=sname cs1=Trojan.Generic.DNS 
-<164>fenotify-852417.alert: CEF:0|FireEye|CMS|7.2.1.244420|DM|domain-match|1|rt=Feb 09 2015 17:17:11 UTC dvc=10.201.78.86 cn3Label=cncPort cn3=53 cn2Label=sid cn2=80494706 shost=dev001srv02.example.com proto=udp cs5Label=cncHost cs5=mfdclk001.org dvchost=DEVFEYE1 spt=51333 dvc=10.100.25.16 smac=00:00:0c:07:ac:c8 cn1Label=vlan cn1=0 externalId=852417 cs4Label=link cs4=https://DEVCMS01.example.com/event_stream/events_for_bot?ev_id\=852417 dmac=88:43:e1:95:13:29 cs1Label=sname cs1=Trojan.Generic.DNS 
-<164>fenotify-852431.alert: CEF:0|FireEye|CMS|7.2.1.244420|DM|domain-match|1|rt=Feb 09 2015 17:20:08 UTC dvc=10.201.78.11 cn3Label=cncPort cn3=53 cn2Label=sid cn2=80494706 shost=dev001srv02.example.com proto=udp cs5Label=cncHost cs5=mfdclk001.org dvchost=DEVFEYE1 spt=53525 dvc=10.100.25.16 smac=00:00:0c:07:ac:c8 cn1Label=vlan cn1=0 externalId=852431 cs4Label=link cs4=https://DEVCMS01.example.com/event_stream/events_for_bot?ev_id\=852431 dmac=88:43:e1:95:13:29 cs1Label=sname cs1=Trojan.Generic.DNS 
-<164>fenotify-852438.alert: CEF:0|FireEye|CMS|7.2.1.244420|DM|domain-match|1|rt=Feb 09 2015 17:21:21 UTC dvc=10.201.78.84 cn3Label=cncPort cn3=53 cn2Label=sid cn2=80494706 shost=dev001srv02.example.com proto=udp cs5Label=cncHost cs5=mfdclk001.org dvchost=DEVFEYE1 spt=62464 dvc=10.100.25.16 smac=00:00:0c:07:ac:c8 cn1Label=vlan cn1=0 externalId=852438 cs4Label=link cs4=https://DEVCMS01.example.com/event_stream/events_for_bot?ev_id\=852438 dmac=88:43:e1:95:13:29 cs1Label=sname cs1=Trojan.Generic.DNS 
-<164>fenotify-3483822.1.alert: CEF:0|FireEye|CMS|7.2.1.244420|MO|malware-object|4|rt=Feb 09 2015 07:24:06 UTC dvc=10.201.78.216 cn3Label=cncPort cn3=80 dst=191.235.179.140 fileHash=6126d97e5bd4e6d93e3e3579cc5b3ce0 filePath=/analysis/191.235.179.140_80-10.220.55.216_56118--833719413_9204551_T.pcoff cs5Label=cncHost cs5=api.shamenchik.info cs3Label=osinfo cs3=Microsoft WindowsXP 32-bit 5.1 sp3 14.0528 proto=tcp dvchost=DEVFEYE1 dvc=10.100.25.16 cn1Label=vlan cn1=0 externalId=3483822 cs4Label=link cs4=https://DEVCMS01.example.com/event_stream/events_for_bot?ma_id\=3483822 cs6Label=channel cs6=POST /api/sdm HTTP/1.1::~~Content-Type: application/x-json::~~Accept: */*::~~User-Agent: Mozilla/4.0 (compatible; MSIE 7.0; Windows NT 5.1; Trident/4.0; .NET CLR 2.0.50727; .NET CLR 3.0.04506.648; .NET CLR 3.5.21022; .NET4.0C; .NET4.0E)::~~Host: api.shamenchik.info::~~Content-Length: 800::~~Connection: Keep-Alive::~~Cache-Control: no-cache::~~::~~g+3CouWsTcAym6cirpXcrPeCqh2q2xYh//aNKX15/lgvTM
-<164>fenotify-851890.alert: CEF:0|FireEye|CMS|7.2.1.244420|DM|domain-match|1|rt=Feb 09 2015 08:36:36 UTC dvc=10.201.78.160 cn3Label=cncPort cn3=53 cn2Label=sid cn2=80494706 shost=dev001srv02.example.com proto=udp cs5Label=cncHost cs5=mfdclk001.org dvchost=DEVFEYE1 spt=63018 dvc=10.100.25.16 smac=00:00:0c:07:ac:00 cn1Label=vlan cn1=0 externalId=851890 cs4Label=link cs4=https://DEVCMS01.example.com/event_stream/events_for_bot?ev_id\=851890 dmac=00:1d:a2:af:32:a1 cs1Label=sname cs1=Trojan.Generic.DNS 
-<164>fenotify-851861.alert: CEF:0|FireEye|CMS|7.2.1.244420|DM|domain-match|1|rt=Feb 09 2015 07:11:45 UTC dvc=10.201.78.20 cn3Label=cncPort cn3=53 cn2Label=sid cn2=80494706 shost=dev001srv02.example.com proto=udp cs5Label=cncHost cs5=mfdclk001.org dvchost=DEVFEYE1 spt=62660 dvc=10.100.25.16 smac=00:00:0c:07:ac:c8 cn1Label=vlan cn1=0 externalId=851861 cs4Label=link cs4=https://DEVCMS01.example.com/event_stream/events_for_bot?ev_id\=851861 dmac=88:43:e1:95:13:29 cs1Label=sname cs1=Trojan.Generic.DNS 
-<164>fenotify-851781.alert: CEF:0|FireEye|CMS|7.2.1.244420|DM|domain-match|1|rt=Feb 09 2015 01:10:09 UTC dvc=10.201.78.20 cn3Label=cncPort cn3=53 cn2Label=sid cn2=80494706 shost=dev001srv02.example.com proto=udp cs5Label=cncHost cs5=mfdclk001.org dvchost=DEVFEYE1 spt=63319 dvc=10.100.25.16 smac=00:00:0c:07:ac:c8 cn1Label=vlan cn1=0 externalId=851781 cs4Label=link cs4=https://DEVCMS01.example.com/event_stream/events_for_bot?ev_id\=851781 dmac=88:43:e1:95:13:29 cs1Label=sname cs1=Trojan.Generic.DNS 
-<164>fenotify-851837.alert: CEF:0|FireEye|CMS|7.2.1.244420|DM|domain-match|1|rt=Feb 09 2015 05:30:01 UTC dvc=10.201.78.60 cn3Label=cncPort cn3=53 cn2Label=sid cn2=80494706 shost=dev001srv02.example.com proto=udp cs5Label=cncHost cs5=mfdclk001.org dvchost=DEVFEYE1 spt=49533 dvc=10.100.25.16 smac=00:00:0c:07:ac:00 cn1Label=vlan cn1=0 externalId=851837 cs4Label=link cs4=https://DEVCMS01.example.com/event_stream/events_for_bot?ev_id\=851837 dmac=00:1d:a2:af:32:a1 cs1Label=sname cs1=Trojan.Generic.DNS 
-<164>fenotify-851846.alert: CEF:0|FireEye|CMS|7.2.1.244420|DM|domain-match|1|rt=Feb 09 2015 06:26:50 UTC dvc=10.201.78.57 cn3Label=cncPort cn3=53 cn2Label=sid cn2=80494706 shost=dev001srv02.example.com proto=udp cs5Label=cncHost cs5=mfdclk001.org dvchost=DEVFEYE1 spt=53933 dvc=10.100.25.16 smac=00:00:0c:07:ac:00 cn1Label=vlan cn1=0 externalId=851846 cs4Label=link cs4=https://DEVCMS01.example.com/event_stream/events_for_bot?ev_id\=851846 dmac=00:1d:a2:af:32:a1 cs1Label=sname cs1=Trojan.Generic.DNS 
-<164>fenotify-851920.alert: CEF:0|FireEye|CMS|7.2.1.244420|DM|domain-match|1|rt=Feb 09 2015 10:26:37 UTC dvc=10.201.78.51 cn3Label=cncPort cn3=53 cn2Label=sid cn2=80494706 shost=dev001srv02.example.com proto=udp cs5Label=cncHost cs5=mfdclk001.org dvchost=DEVFEYE1 spt=60410 dvc=10.100.25.16 smac=00:00:0c:07:ac:63 cn1Label=vlan cn1=0 externalId=851920 cs4Label=link cs4=https://DEVCMS01.example.com/event_stream/events_for_bot?ev_id\=851920 dmac=5c:5e:ab:eb:ab:0d cs1Label=sname cs1=Trojan.Generic.DNS 
-<164>fenotify-851818.alert: CEF:0|FireEye|CMS|7.2.1.244420|DM|domain-match|1|rt=Feb 09 2015 04:25:02 UTC dvc=10.201.78.51 cn3Label=cncPort cn3=53 cn2Label=sid cn2=80494706 shost=dev001srv02.example.com proto=udp cs5Label=cncHost cs5=mfdclk001.org dvchost=DEVFEYE1 spt=60319 dvc=10.100.25.16 smac=00:00:0c:07:ac:63 cn1Label=vlan cn1=0 externalId=851818 cs4Label=link cs4=https://DEVCMS01.example.com/event_stream/events_for_bot?ev_id\=851818 dmac=5c:5e:ab:eb:ab:0d cs1Label=sname cs1=Trojan.Generic.DNS 
-<164>fenotify-851866.alert: CEF:0|FireEye|CMS|7.2.1.244420|DM|domain-match|1|rt=Feb 09 2015 07:13:28 UTC dvc=10.201.78.12 cn3Label=cncPort cn3=53 cn2Label=sid cn2=80494706 shost=dev001srv02.example.com proto=udp cs5Label=cncHost cs5=mfdclk001.org dvchost=DEVFEYE1 spt=54836 dvc=10.100.25.16 smac=00:00:0c:07:ac:c8 cn1Label=vlan cn1=0 externalId=851866 cs4Label=link cs4=https://DEVCMS01.example.com/event_stream/events_for_bot?ev_id\=851866 dmac=88:43:e1:95:13:29 cs1Label=sname cs1=Trojan.Generic.DNS 
-<164>fenotify-851773.alert: CEF:0|FireEye|CMS|7.2.1.244420|DM|domain-match|1|rt=Feb 09 2015 00:01:29 UTC dvc=10.201.78.68 cn3Label=cncPort cn3=53 cn2Label=sid cn2=80494706 proto=udp cs5Label=cncHost cs5=mfdclk001.org dvchost=DEVFEYE1 spt=60239 dvc=10.100.25.16 smac=00:00:0c:07:ac:5a cn1Label=vlan cn1=0 externalId=851773 cs4Label=link cs4=https://DEVCMS01.example.com/event_stream/events_for_bot?ev_id\=851773 dmac=00:09:0f:33:4f:48 cs1Label=sname cs1=Trojan.Generic.DNS 
-<164>fenotify-851935.alert: CEF:0|FireEye|CMS|7.2.1.244420|DM|domain-match|1|rt=Feb 09 2015 10:48:18 UTC dvc=10.201.78.11 cn3Label=cncPort cn3=53 cn2Label=sid cn2=80494706 shost=dev001srv02.example.com proto=udp cs5Label=cncHost cs5=mfdclk001.org dvchost=DEVFEYE1 spt=54362 dvc=10.100.25.16 smac=00:00:0c:07:ac:c8 cn1Label=vlan cn1=0 externalId=851935 cs4Label=link cs4=https://DEVCMS01.example.com/event_stream/events_for_bot?ev_id\=851935 dmac=88:43:e1:95:13:29 cs1Label=sname cs1=Trojan.Generic.DNS 
-<164>fenotify-851970.alert: CEF:0|FireEye|CMS|7.2.1.244420|DM|domain-match|1|rt=Feb 09 2015 12:04:50 UTC dvc=10.201.78.40 cn3Label=cncPort cn3=53 cn2Label=sid cn2=80494706 shost=dev001srv02.example.com proto=udp cs5Label=cncHost cs5=mfdclk001.org dvchost=DEVFEYE1 spt=50327 dvc=10.100.25.16 smac=00:00:0c:07:ac:c8 cn1Label=vlan cn1=0 externalId=851970 cs4Label=link cs4=https://DEVCMS01.example.com/event_stream/events_for_bot?ev_id\=851970 dmac=88:43:e1:95:13:29 cs1Label=sname cs1=Trojan.Generic.DNS 
-<164>fenotify-851975.alert: CEF:0|FireEye|CMS|7.2.1.244420|DM|domain-match|1|rt=Feb 09 2015 12:21:18 UTC dvc=10.201.78.59 cn3Label=cncPort cn3=53 cn2Label=sid cn2=80494706 shost=dev001srv02.example.com proto=udp cs5Label=cncHost cs5=mfdclk001.org dvchost=DEVFEYE1 spt=51420 dvc=10.100.25.16 smac=00:00:0c:07:ac:00 cn1Label=vlan cn1=0 externalId=851975 cs4Label=link cs4=https://DEVCMS01.example.com/event_stream/events_for_bot?ev_id\=851975 dmac=00:1d:a2:af:32:a1 cs1Label=sname cs1=Trojan.Generic.DNS 
-<164>fenotify-852454.alert: CEF:0|FireEye|CMS|7.2.1.244420|DM|domain-match|1|rt=Feb 09 2015 17:28:34 UTC dvc=10.201.78.44 cn3Label=cncPort cn3=53 cn2Label=sid cn2=80494706 shost=dev001srv02.example.com proto=udp cs5Label=cncHost cs5=mfdclk001.org dvchost=DEVFEYE1 spt=55348 dvc=10.100.25.16 smac=00:00:0c:07:ac:c8 cn1Label=vlan cn1=0 externalId=852454 cs4Label=link cs4=https://DEVCMS01.example.com/event_stream/events_for_bot?ev_id\=852454 dmac=88:43:e1:95:13:29 cs1Label=sname cs1=Trojan.Generic.DNS 
-<164>fenotify-3483798.2.alert: act;Trojan.Kuloz;Trojan.Kuluoz 
-<164>fenotify-834781.2.alert: Connection: Keep-Alive::~~::~~ dmac=00:1d:a2:af:32:a1 cs1Label=sname cs1=Malicious.URL 
-<164>fenotify-3483794.3.alert: 0d3cc7cc055f8d686a1b5d5c30db85c5423620e6bd231d592266782cf5e1647ae575e77b HTTP/1.1::~~Accept: */*::~~Proxy-Authorization: Basic ::~~User-Agent: Mozilla/5.0 (Windows NT 6.1; WOW64) AppleWebKit/537.36 (KHTML, like Gecko) Chrome/35.0.1916.153 Safari/537.36::~~Host: 5aqobwcp1xuqztwht.0eq0w6k.com::~~Connection: Keep-Alive::~~::~~ cs2Label=anomaly cs2=misc-anomaly cs1Label=sname cs1=FE_Evasion_Sandboxie;FE_Evasion_VMDetect 
-<164>fenotify-3483796.2.alert: jan.Kuloz;Trojan.Kuluoz 
-<164>fenotify-851894.alert: CEF:0|FireEye|CMS|7.2.1.244420|DM|domain-match|1|rt=Feb 09 2015 08:45:48 UTC dvc=10.201.78.60 cn3Label=cncPort cn3=53 cn2Label=sid cn2=80494706 shost=dev001srv02.example.com proto=udp cs5Label=cncHost cs5=mfdclk001.org dvchost=DEVFEYE1 spt=49433 dvc=10.100.25.16 smac=00:00:0c:07:ac:00 cn1Label=vlan cn1=0 externalId=851894 cs4Label=link cs4=https://DEVCMS01.example.com/event_stream/events_for_bot?ev_id\=851894 dmac=00:1d:a2:af:32:a1 cs1Label=sname cs1=Trojan.Generic.DNS 
-<164>fenotify-851899.alert: CEF:0|FireEye|CMS|7.2.1.244420|DM|domain-match|1|rt=Feb 09 2015 08:54:50 UTC dvc=10.201.78.34 cn3Label=cncPort cn3=53 cn2Label=sid cn2=80494706 shost=dev001srv02.example.com proto=udp cs5Label=cncHost cs5=mfdclk001.org dvchost=DEVFEYE1 spt=50711 dvc=10.100.25.16 smac=00:00:0c:07:ac:c8 cn1Label=vlan cn1=0 externalId=851899 cs4Label=link cs4=https://DEVCMS01.example.com/event_stream/events_for_bot?ev_id\=851899 dmac=88:43:e1:95:13:29 cs1Label=sname cs1=Trojan.Generic.DNS 
-<164>fenotify-851851.alert: CEF:0|FireEye|CMS|7.2.1.244420|DM|domain-match|1|rt=Feb 09 2015 06:31:05 UTC dvc=10.201.78.190 cn3Label=cncPort cn3=53 cn2Label=sid cn2=80494706 shost=dev001srv02.example.com proto=udp cs5Label=cncHost cs5=mfdclk001.org dvchost=DEVFEYE1 spt=61134 dvc=10.100.25.16 smac=00:00:0c:07:ac:00 cn1Label=vlan cn1=0 externalId=851851 cs4Label=link cs4=https://DEVCMS01.example.com/event_stream/events_for_bot?ev_id\=851851 dmac=00:1d:a2:af:32:a1 cs1Label=sname cs1=Trojan.Generic.DNS 
-<164>fenotify-851845.alert: CEF:0|FireEye|CMS|7.2.1.244420|DM|domain-match|1|rt=Feb 09 2015 06:20:46 UTC dvc=10.201.78.20 cn3Label=cncPort cn3=53 cn2Label=sid cn2=80494706 shost=dev001srv02.example.com proto=udp cs5Label=cncHost cs5=mfdclk001.org dvchost=DEVFEYE1 spt=55294 dvc=10.100.25.16 smac=00:00:0c:07:ac:00 cn1Label=vlan cn1=0 externalId=851845 cs4Label=link cs4=https://DEVCMS01.example.com/event_stream/events_for_bot?ev_id\=851845 dmac=00:1d:a2:af:32:a1 cs1Label=sname cs1=Trojan.Generic.DNS 
-<164>fenotify-851789.alert: CEF:0|FireEye|CMS|7.2.1.244420|DM|domain-match|1|rt=Feb 09 2015 02:03:48 UTC dvc=10.201.78.84 cn3Label=cncPort cn3=53 cn2Label=sid cn2=80494706 shost=dev001srv02.example.com proto=udp cs5Label=cncHost cs5=mfdclk001.org dvchost=DEVFEYE1 spt=62782 dvc=10.100.25.16 smac=00:00:0c:07:ac:c8 cn1Label=vlan cn1=0 externalId=851789 cs4Label=link cs4=https://DEVCMS01.example.com/event_stream/events_for_bot?ev_id\=851789 dmac=88:43:e1:95:13:29 cs1Label=sname cs1=Trojan.Generic.DNS 
-<164>fenotify-851820.alert: CEF:0|FireEye|CMS|7.2.1.244420|DM|domain-match|1|rt=Feb 09 2015 04:33:45 UTC dvc=10.201.78.87 cn3Label=cncPort cn3=53 cn2Label=sid cn2=80494706 shost=dev001srv02.example.com proto=udp cs5Label=cncHost cs5=mfdclk001.org dvchost=DEVFEYE1 spt=63559 dvc=10.100.25.16 smac=00:00:0c:07:ac:c8 cn1Label=vlan cn1=0 externalId=851820 cs4Label=link cs4=https://DEVCMS01.example.com/event_stream/events_for_bot?ev_id\=851820 dmac=88:43:e1:95:13:29 cs1Label=sname cs1=Trojan.Generic.DNS 
-<164>fenotify-851828.alert: CEF:0|FireEye|CMS|7.2.1.244420|DM|domain-match|1|rt=Feb 09 2015 05:09:07 UTC dvc=10.201.78.86 cn3Label=cncPort cn3=53 cn2Label=sid cn2=80494706 shost=dev001srv02.example.com proto=udp cs5Label=cncHost cs5=mfdclk001.org dvchost=DEVFEYE1 spt=52967 dvc=10.100.25.16 smac=00:00:0c:07:ac:c8 cn1Label=vlan cn1=0 externalId=851828 cs4Label=link cs4=https://DEVCMS01.example.com/event_stream/events_for_bot?ev_id\=851828 dmac=88:43:e1:95:13:29 cs1Label=sname cs1=Trojan.Generic.DNS 
-<164>fenotify-851816.alert: CEF:0|FireEye|CMS|7.2.1.244420|DM|domain-match|1|rt=Feb 09 2015 04:16:05 UTC dvc=10.201.78.34 cn3Label=cncPort cn3=53 cn2Label=sid cn2=80494706 proto=udp cs5Label=cncHost cs5=mfdclk001.org dvchost=DEVFEYE1 spt=61806 dvc=10.100.25.5 smac=00:00:0c:07:ac:63 cn1Label=vlan cn1=143 externalId=851816 cs4Label=link cs4=https://DEVCMS01.example.com/event_stream/events_for_bot?ev_id\=851816 dmac=00:1b:17:00:09:01 cs1Label=sname cs1=Trojan.Generic.DNS 
-<164>fenotify-851831.alert: CEF:0|FireEye|CMS|7.2.1.244420|DM|domain-match|1|rt=Feb 09 2015 05:14:31 UTC dvc=10.201.78.11 cn3Label=cncPort cn3=53 cn2Label=sid cn2=80494706 proto=udp cs5Label=cncHost cs5=mfdclk001.org dvchost=DEVFEYE1 spt=58655 dvc=10.100.25.5 smac=00:00:0c:07:ac:63 cn1Label=vlan cn1=143 externalId=851831 cs4Label=link cs4=https://DEVCMS01.example.com/event_stream/events_for_bot?ev_id\=851831 dmac=00:1b:17:00:09:01 cs1Label=sname cs1=Trojan.Generic.DNS 
-<164>fenotify-851950.alert: CEF:0|FireEye|CMS|7.2.1.244420|DM|domain-match|1|rt=Feb 09 2015 11:16:07 UTC dvc=10.201.78.11 cn3Label=cncPort cn3=53 cn2Label=sid cn2=80494706 proto=udp cs5Label=cncHost cs5=mfdclk001.org dvchost=DEVFEYE1 spt=58855 dvc=10.100.25.5 smac=00:00:0c:07:ac:63 cn1Label=vlan cn1=143 externalId=851950 cs4Label=link cs4=https://DEVCMS01.example.com/event_stream/events_for_bot?ev_id\=851950 dmac=00:1b:17:00:09:01 cs1Label=sname cs1=Trojan.Generic.DNS 
-<164>fenotify-851988.alert: CEF:0|FireEye|CMS|7.2.1.244420|DM|domain-match|1|rt=Feb 09 2015 12:35:26 UTC dvc=10.201.78.190 cn3Label=cncPort cn3=53 cn2Label=sid cn2=80494706 shost=dev001srv02.example.com proto=udp cs5Label=cncHost cs5=mfdclk001.org dvchost=DEVFEYE1 spt=61427 dvc=10.100.25.16 smac=00:00:0c:07:ac:00 cn1Label=vlan cn1=0 externalId=851988 cs4Label=link cs4=https://DEVCMS01.example.com/event_stream/events_for_bot?ev_id\=851988 dmac=00:1d:a2:af:32:a1 cs1Label=sname cs1=Trojan.Generic.DNS 
-<164>fenotify-852013.alert: CEF:0|FireEye|CMS|7.2.1.244420|DM|domain-match|1|rt=Feb 09 2015 13:18:29 UTC dvc=10.201.78.34 cn3Label=cncPort cn3=53 cn2Label=sid cn2=80494706 proto=udp cs5Label=cncHost cs5=mfdclk001.org dvchost=DEVFEYE1 spt=61630 dvc=10.100.25.5 smac=00:00:0c:07:ac:63 cn1Label=vlan cn1=143 externalId=852013 cs4Label=link cs4=https://DEVCMS01.example.com/event_stream/events_for_bot?ev_id\=852013 dmac=00:1b:17:00:09:01 cs1Label=sname cs1=Trojan.Generic.DNS 
-<164>fenotify-852070.alert: CEF:0|FireEye|CMS|7.2.1.244420|DM|domain-match|1|rt=Feb 09 2015 14:27:45 UTC dvc=10.201.78.44 cn3Label=cncPort cn3=53 cn2Label=sid cn2=80494706 shost=dev001srv02.example.com proto=udp cs5Label=cncHost cs5=mfdclk001.org dvchost=DEVFEYE1 spt=54769 dvc=10.100.25.16 smac=00:00:0c:07:ac:c8 cn1Label=vlan cn1=0 externalId=852070 cs4Label=link cs4=https://DEVCMS01.example.com/event_stream/events_for_bot?ev_id\=852070 dmac=88:43:e1:95:13:29 cs1Label=sname cs1=Trojan.Generic.DNS 
-<164>fenotify-852082.alert: CEF:0|FireEye|CMS|7.2.1.244420|DM|domain-match|1|rt=Feb 09 2015 14:35:15 UTC dvc=10.201.78.68 cn3Label=cncPort cn3=53 cn2Label=sid cn2=80494706 proto=udp cs5Label=cncHost cs5=mfdclk001.org dvchost=DEVFEYE1 spt=60922 dvc=10.100.25.16 smac=00:00:0c:07:ac:5a cn1Label=vlan cn1=0 externalId=852082 cs4Label=link cs4=https://DEVCMS01.example.com/event_stream/events_for_bot?ev_id\=852082 dmac=00:09:0f:33:4f:48 cs1Label=sname cs1=Trojan.Generic.DNS 
-<164>fenotify-852114.alert: CEF:0|FireEye|CMS|7.2.1.244420|DM|domain-match|1|rt=Feb 09 2015 14:59:08 UTC dvc=10.201.78.194 cn3Label=cncPort cn3=53 cn2Label=sid cn2=80494706 shost=dev001srv02.example.com proto=udp cs5Label=cncHost cs5=mfdclk001.org dvchost=DEVFEYE1 spt=50396 dvc=10.100.25.16 smac=00:00:0c:07:ac:c8 cn1Label=vlan cn1=0 externalId=852114 cs4Label=link cs4=https://DEVCMS01.example.com/event_stream/events_for_bot?ev_id\=852114 dmac=88:43:e1:95:13:29 cs1Label=sname cs1=Trojan.Generic.DNS 
-<164>fenotify-852295.alert: CEF:0|FireEye|CMS|7.2.1.244420|DM|domain-match|1|rt=Feb 09 2015 16:30:40 UTC dvc=10.201.78.51 cn3Label=cncPort cn3=53 cn2Label=sid cn2=80494706 shost=dev001srv02.example.com proto=udp cs5Label=cncHost cs5=mfdclk001.org dvchost=DEVFEYE1 spt=60266 dvc=10.100.25.16 smac=00:00:0c:07:ac:63 cn1Label=vlan cn1=0 externalId=852295 cs4Label=link cs4=https://DEVCMS01.example.com/event_stream/events_for_bot?ev_id\=852295 dmac=5c:5e:ab:eb:ab:0d cs1Label=sname cs1=Trojan.Generic.DNS 
-<164>fenotify-3483807.2.alert: z0Q6RNzwu2BoLSVUhiBihE4z0mlPDacuE1Waqs86Z9VVYg6iM2MlFH8GZgagnlOuzfB2JHdKPc/GwnzFk5DPfUPJAe8DH9Y6hwohv0t6XFVWx5UDSGARW8w3GAop9R+9iaSCuomuLU26/gaqL4gfjZqjLHzoDx+vhOCiOP6RnGMio5v2kcKxitPL7pPVu5FJ6MwUG7QOLecwONRzQsFh/jXFT4gyR2iS/EbufAeRofQVZHsj9dhgHZKNLcsFPnw/8lWlvgku7s28l57lAGxtp99jkzYD58jPgBm1nGbJPubFTL47ZmBkPPNsc1XjRcNvPz5/nzb0eWctXZ2cKocAQnT2zHOgBxRM6my9QW/Lg0JWaQyqBO2EOyTfej6KgVlHoIf0E3bv6C5PgVrJunAIqqlO6EvKvILlDYk2yoklvP3Fry5p4Nrw2isE95Used9Zqsoxx0bWInNcfyQhoqqlmYKiZZb+aBwGvJEL634pmoTMEBMdn4s3gz2a7aLV+vOVULQbgR15PygsYQdOnymv7uWZtdKOp7ut21GwNu9ZxJGMrssW0gzvaZiZDs7FSordVPUiUqcfS6ciU1cl29fNTWnmRkq4vk+vBgvUQLxTTAleV9k5svtB237GvvolWE72ugJQXUun51WxAqOAZpV0c6tEbK5qd6Z55z8Rs/LpN8VM4/nbZmfB5XY+eCCLfULjisVoStUUeH67&report\=p509XA27GEFLLes0RJ8pJJdIkbJ+3YkVUv2qjhuxlRPlVrrEZckeXFIaD+4/a1xulR8kKMx9GrPD2uc/wC+NxgKg/ok/kttHH45shX4YjPLsS4QtXUHugcE5Rr1238CYegHwOKWzAp3g5Mpt7loabRTBtmzXXeLBV4cFKv3zWpxQ7+CBGpsDfsvkD2Qgst3FX05VQHBpnJfXgRqdRrLyUjezF1tlIgvvNCv6hQ+zffxKk0WcD
 oUe8
-<164>fenotify-3483794.2.alert: 53 Safari/537.36::~~Host: 5aqobwcp1xuqztwht.0eq0w6k.com::~~Connection: Keep-Alive::~~::~~GET /93ea73bcdaf32d5074e62be84ee83a84cacefa8dcf855c265457842d6b05f469863ca7110d3cc7cc055f8d686a1b5d5c30db85c5423620e6bd231d592266782cf5e1647ae575e77b HTTP/1.1::~~Accept: */*::~~Proxy-Authorization: Basic ::~~User-Agent: Mozilla/5.0 (Windows NT 6.1; WOW64) AppleWebKit/537.36 (KHTML, like Gecko) Chrome/35.0.1916.153 Safari/537.36::~~Host: 5aqobwcp1xuqztwht.0eq0w6k.com::~~Connection: Keep-Alive::~~::~~GET /93ea73bcdaf32d5074e62be84ee83a84cacefa8dcf855c265457842d6b05f469863ca7110d3cc7cc055f8d686a1b5d5c30db85c5423620e6bd231d592266782cf5e1647ae575e77b HTTP/1.1::~~Accept: */*::~~Proxy-Authorization: Basic ::~~User-Agent: Mozilla/5.0 (Windows NT 6.1; WOW64) AppleWebKit/537.36 (KHTML, like Gecko) Chrome/35.0.1916.153 Safari/537.36::~~Host: 5aqobwcp1xuqztwht.0eq0w6k.com::~~Connection: Keep-Alive::~~::~~GET /93ea73bcdaf32d5074e62be84ee83a84cacefa8dcf855c265457842d6b05f469863c
 a711
-<164>fenotify-3483799.2.alert: L, like Gecko) Chrome/35.0.1916.153 Safari/537.36::~~Host: pkeyqcot5gzamu.5t9dyvo2.com::~~Connection: Keep-Alive::~~::~~ cs2Label=anomaly cs2=misc-anomaly cs1Label=sname cs1=FE_PUP_Softpulse;FE_Evasion_VMDetect;FE_Evasion_DBGDetect_Files;FE_Evasion_Sandboxie 
-<164>fenotify-3483807.3.alert: n6o4JWRQX2V1jsLkx8LFQz3nXe7Bbiuuc1sMcdS/lEv7f9zpw09qs0LvVpRJe4tZjE4Gsghh7Xh5OAxE2A7HBLnWjloIazv6jvun+R1BpF1vuujyEdDgKWIv4BeMmQQJ6p66O/U0jHvWelTBMT+RTVFERsryrpWE+g7AHeRyzDIERgWxHxzA9y6cQ9JYp2/JOPdUzWnLWM24Be6fWmlJ37J90GuEvHh+WXWsaewcBg8xUAhlQBfEHP01PGcuX2yJin2rQ8/GhkiF210HCJUCIbxxz6rZuf6CaksKSXPIeXf1Iifha58Rtm cs2Label=anomaly cs2=misc-anomaly cs1Label=sname cs1=Malware.Binary 
\ No newline at end of file



[05/15] incubator-metron git commit: METRON-142 Simplify Parser configuration (merrimanr via cestella) closes apache/incubator-metron#120

Posted by ce...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/df8d682e/metron-platform/metron-integration-test/src/main/sample/data/SampleInput/LancopeExampleOutput
----------------------------------------------------------------------
diff --git a/metron-platform/metron-integration-test/src/main/sample/data/SampleInput/LancopeExampleOutput b/metron-platform/metron-integration-test/src/main/sample/data/SampleInput/LancopeExampleOutput
new file mode 100644
index 0000000..b1bccf9
--- /dev/null
+++ b/metron-platform/metron-integration-test/src/main/sample/data/SampleInput/LancopeExampleOutput
@@ -0,0 +1,40 @@
+{"message":"<131>Jul 17 15:27:27 smc-01 StealthWatch[12365]: 2014-06-24T14:37:58Z 192.168.200.9 199.237.198.232 Critical Bad Host The host has been observed doing something bad to another host. Source Host is http (80/tcp) client to target.host.name (199.237.198.232)","@version":"1","@timestamp":"2014-07-17T15:24:32.217Z","type":"syslog","host":"10.122.196.201"}
+{"message":"<131>Jul 17 15:35:00 smc-01 StealthWatch[12365]: 2014-07-17T15:34:30Z 10.201.3.83 0.0.0.0 Minor High Concern Index The host's concern index has either exceeded the CI threshold or rapidly increased. Observed 92.64M points. Policy maximum allows up to 20M points.","@version":"1","@timestamp":"2014-07-17T15:32:05.934Z","type":"syslog","host":"10.122.196.201"}
+{"message":"<131>Jul 17 15:35:00 smc-01 StealthWatch[12365]: 2014-07-17T15:34:30Z 10.201.3.145 0.0.0.0 Minor High Concern Index The host's concern index has either exceeded the CI threshold or rapidly increased. Observed 45.2M points. Policy maximum allows up to 20M points.","@version":"1","@timestamp":"2014-07-17T15:32:05.935Z","type":"syslog","host":"10.122.196.201"}
+{"message":"<131>Jul 17 15:35:00 smc-01 StealthWatch[12365]: 2014-07-17T15:34:30Z 10.201.3.50 0.0.0.0 Minor High Concern Index The host's concern index has either exceeded the CI threshold or rapidly increased. Observed 41.46M points. Policy maximum allows up to 20M points.","@version":"1","@timestamp":"2014-07-17T15:32:05.936Z","type":"syslog","host":"10.122.196.201"}
+{"message":"<131>Jul 17 15:42:01 smc-01 StealthWatch[12365]: 2014-07-17T15:42:00Z 10.10.101.24 0.0.0.0 Minor High Concern Index The host's concern index has either exceeded the CI threshold or rapidly increased. Observed 39.37M points. Policy maximum allows up to 20M points.","@version":"1","@timestamp":"2014-07-17T15:39:05.976Z","type":"syslog","host":"10.122.196.201"}
+{"message":"<131>Jul 17 15:56:01 smc-01 StealthWatch[12365]: 2014-07-17T15:55:00Z 0.0.100.0 0.0.0.0 Major ICMP Flood The source IP has sent an excessive number of ICMP packets in the last 5 minutes. Observed 262.4k pp5m. Policy maximum allows up to 100k pp5m.","@version":"1","@timestamp":"2014-07-17T15:53:05.995Z","type":"syslog","host":"10.122.196.201"}
+{"message":"<131>Jul 17 15:56:01 smc-01 StealthWatch[12365]: 2014-07-17T15:55:00Z 0.0.88.0 0.0.0.0 Major High Total Traffic The total traffic inbound + outbound exceeds the acceptable total traffic values. Observed 16.26G bytes. Expected 4.17G bytes, tolerance of 50 allows up to 15.06G bytes.","@version":"1","@timestamp":"2014-07-17T15:53:05.996Z","type":"syslog","host":"10.122.196.201"}
+{"message":"<131>Jul 17 15:57:01 smc-01 StealthWatch[12365]: 2014-07-17T15:56:30Z 10.201.3.50 0.0.0.0 Minor High Concern Index The host's concern index has either exceeded the CI threshold or rapidly increased. Observed 42.49M points. Policy maximum allows up to 20M points.","@version":"1","@timestamp":"2014-07-17T15:54:05.984Z","type":"syslog","host":"10.122.196.201"}
+{"message":"<131>Jul 17 15:59:01 smc-01 StealthWatch[12365]: 2014-07-17T15:58:30Z 10.40.10.254 0.0.0.0 Minor High Concern Index The host's concern index has either exceeded the CI threshold or rapidly increased. Observed 36.55M points. Policy maximum allows up to 20M points.","@version":"1","@timestamp":"2014-07-17T15:56:05.992Z","type":"syslog","host":"10.122.196.201"}
+{"message":"<131>Jul 17 15:59:01 smc-01 StealthWatch[12365]: 2014-07-17T15:58:30Z 10.30.10.254 0.0.0.0 Minor High Concern Index The host's concern index has either exceeded the CI threshold or rapidly increased. Observed 36.47M points. Policy maximum allows up to 20M points.","@version":"1","@timestamp":"2014-07-17T15:56:05.995Z","type":"syslog","host":"10.122.196.201"}
+{"message":"<131>Jul 17 15:59:01 smc-01 StealthWatch[12365]: 2014-07-17T15:58:30Z 10.20.10.254 0.0.0.0 Minor High Concern Index The host's concern index has either exceeded the CI threshold or rapidly increased. Observed 40.48M points. Policy maximum allows up to 20M points.","@version":"1","@timestamp":"2014-07-17T15:56:05.995Z","type":"syslog","host":"10.122.196.201"}
+{"message":"<131>Jul 17 15:59:01 smc-01 StealthWatch[12365]: 2014-07-17T15:58:30Z 10.201.3.83 0.0.0.0 Minor High Concern Index The host's concern index has either exceeded the CI threshold or rapidly increased. Observed 96.74M points. Policy maximum allows up to 20M points.","@version":"1","@timestamp":"2014-07-17T15:56:05.992Z","type":"syslog","host":"10.122.196.201"}
+{"message":"<131>Jul 17 15:59:01 smc-01 StealthWatch[12365]: 2014-07-17T15:58:30Z 10.100.10.254 0.0.0.0 Minor High Concern Index The host's concern index has either exceeded the CI threshold or rapidly increased. Observed 32.95M points. Policy maximum allows up to 20M points.","@version":"1","@timestamp":"2014-07-17T15:56:05.997Z","type":"syslog","host":"10.122.196.201"}
+{"message":"<131>Jul 17 15:59:01 smc-01 StealthWatch[12365]: 2014-07-17T15:58:30Z 10.90.10.254 0.0.0.0 Minor High Concern Index The host's concern index has either exceeded the CI threshold or rapidly increased. Observed 36.52M points. Policy maximum allows up to 20M points.","@version":"1","@timestamp":"2014-07-17T15:56:06.000Z","type":"syslog","host":"10.122.196.201"}
+{"message":"<131>Jul 17 15:59:01 smc-01 StealthWatch[12365]: 2014-07-17T15:58:30Z 10.80.10.254 0.0.0.0 Minor High Concern Index The host's concern index has either exceeded the CI threshold or rapidly increased. Observed 36.51M points. Policy maximum allows up to 20M points.","@version":"1","@timestamp":"2014-07-17T15:56:06.002Z","type":"syslog","host":"10.122.196.201"}
+{"message":"<131>Jul 17 15:59:01 smc-01 StealthWatch[12365]: 2014-07-17T15:58:30Z 10.70.10.254 0.0.0.0 Minor High Concern Index The host's concern index has either exceeded the CI threshold or rapidly increased. Observed 36.49M points. Policy maximum allows up to 20M points.","@version":"1","@timestamp":"2014-07-17T15:56:06.002Z","type":"syslog","host":"10.122.196.201"}
+{"message":"<131>Jul 17 15:59:01 smc-01 StealthWatch[12365]: 2014-07-17T15:58:30Z 10.110.10.254 0.0.0.0 Minor High Concern Index The host's concern index has either exceeded the CI threshold or rapidly increased. Observed 32.92M points. Policy maximum allows up to 20M points.","@version":"1","@timestamp":"2014-07-17T15:56:05.997Z","type":"syslog","host":"10.122.196.201"}
+{"message":"<131>Jul 17 15:59:01 smc-01 StealthWatch[12365]: 2014-07-17T15:58:30Z 10.60.10.254 0.0.0.0 Minor High Concern Index The host's concern index has either exceeded the CI threshold or rapidly increased. Observed 36.49M points. Policy maximum allows up to 20M points.","@version":"1","@timestamp":"2014-07-17T15:56:06.003Z","type":"syslog","host":"10.122.196.201"}
+{"message":"<131>Jul 17 15:59:01 smc-01 StealthWatch[12365]: 2014-07-17T15:58:30Z 10.50.10.254 0.0.0.0 Minor High Concern Index The host's concern index has either exceeded the CI threshold or rapidly increased. Observed 36.48M points. Policy maximum allows up to 20M points.","@version":"1","@timestamp":"2014-07-17T15:56:06.004Z","type":"syslog","host":"10.122.196.201"}
+{"message":"<131>Jul 17 16:06:01 smc-01 StealthWatch[12365]: 2014-07-17T16:05:00Z 10.10.101.46 0.0.0.0 Major New Flows Initiated The host has exceeded the acceptable total number of new flows initiated in a 5-minute period. ","@version":"1","@timestamp":"2014-07-17T16:03:06.046Z","type":"syslog","host":"10.122.196.201"}
+{"message":"<131>Jul 17 16:06:01 smc-01 StealthWatch[12365]: 2014-07-17T16:05:00Z 10.10.101.46 0.0.0.0 Major Max Flows Initiated The host has initiated more than an acceptable maximum number of flows. ","@version":"1","@timestamp":"2014-07-17T16:03:06.046Z","type":"syslog","host":"10.122.196.201"}
+{"message":"<131>Jul 17 16:20:00 smc-01 StealthWatch[12365]: 2014-07-17T16:19:30Z 10.110.10.254 0.0.0.0 Minor High Concern Index The host's concern index has either exceeded the CI threshold or rapidly increased. Observed 33.01M points. Policy maximum allows up to 20M points.","@version":"1","@timestamp":"2014-07-17T16:17:05.146Z","type":"syslog","host":"10.122.196.201"}
+{"message":"<131>Jul 17 16:20:00 smc-01 StealthWatch[12365]: 2014-07-17T16:19:30Z 10.100.10.254 0.0.0.0 Minor High Concern Index The host's concern index has either exceeded the CI threshold or rapidly increased. Observed 33.03M points. Policy maximum allows up to 20M points.","@version":"1","@timestamp":"2014-07-17T16:17:05.147Z","type":"syslog","host":"10.122.196.201"}
+{"message":"<131>Jul 17 16:20:00 smc-01 StealthWatch[12365]: 2014-07-17T16:19:30Z 10.90.10.254 0.0.0.0 Minor High Concern Index The host's concern index has either exceeded the CI threshold or rapidly increased. Observed 36.59M points. Policy maximum allows up to 20M points.","@version":"1","@timestamp":"2014-07-17T16:17:05.148Z","type":"syslog","host":"10.122.196.201"}
+{"message":"<131>Jul 17 16:20:00 smc-01 StealthWatch[12365]: 2014-07-17T16:19:30Z 10.80.10.254 0.0.0.0 Minor High Concern Index The host's concern index has either exceeded the CI threshold or rapidly increased. Observed 36.58M points. Policy maximum allows up to 20M points.","@version":"1","@timestamp":"2014-07-17T16:17:05.157Z","type":"syslog","host":"10.122.196.201"}
+{"message":"<131>Jul 17 16:20:00 smc-01 StealthWatch[12365]: 2014-07-17T16:19:30Z 10.70.10.254 0.0.0.0 Minor High Concern Index The host's concern index has either exceeded the CI threshold or rapidly increased. Observed 36.56M points. Policy maximum allows up to 20M points.","@version":"1","@timestamp":"2014-07-17T16:17:05.157Z","type":"syslog","host":"10.122.196.201"}
+{"message":"<131>Jul 17 16:20:00 smc-01 StealthWatch[12365]: 2014-07-17T16:19:30Z 10.60.10.254 0.0.0.0 Minor High Concern Index The host's concern index has either exceeded the CI threshold or rapidly increased. Observed 36.56M points. Policy maximum allows up to 20M points.","@version":"1","@timestamp":"2014-07-17T16:17:05.158Z","type":"syslog","host":"10.122.196.201"}
+{"message":"<131>Jul 17 16:20:00 smc-01 StealthWatch[12365]: 2014-07-17T16:19:30Z 10.50.10.254 0.0.0.0 Minor High Concern Index The host's concern index has either exceeded the CI threshold or rapidly increased. Observed 36.55M points. Policy maximum allows up to 20M points.","@version":"1","@timestamp":"2014-07-17T16:17:05.160Z","type":"syslog","host":"10.122.196.201"}
+{"message":"<131>Jul 17 16:20:00 smc-01 StealthWatch[12365]: 2014-07-17T16:19:30Z 10.30.10.254 0.0.0.0 Minor High Concern Index The host's concern index has either exceeded the CI threshold or rapidly increased. Observed 36.55M points. Policy maximum allows up to 20M points.","@version":"1","@timestamp":"2014-07-17T16:17:05.173Z","type":"syslog","host":"10.122.196.201"}
+{"message":"<131>Jul 17 16:20:00 smc-01 StealthWatch[12365]: 2014-07-17T16:19:30Z 10.201.3.83 0.0.0.0 Minor High Concern Index The host's concern index has either exceeded the CI threshold or rapidly increased. Observed 96.82M points. Policy maximum allows up to 20M points.","@version":"1","@timestamp":"2014-07-17T16:17:05.173Z","type":"syslog","host":"10.122.196.201"}
+{"message":"<131>Jul 17 16:20:00 smc-01 StealthWatch[12365]: 2014-07-17T16:19:30Z 10.20.10.254 0.0.0.0 Minor High Concern Index The host's concern index has either exceeded the CI threshold or rapidly increased. Observed 40.55M points. Policy maximum allows up to 20M points.","@version":"1","@timestamp":"2014-07-17T16:17:05.174Z","type":"syslog","host":"10.122.196.201"}
+{"message":"<131>Jul 17 16:20:00 smc-01 StealthWatch[12365]: 2014-07-17T16:20:00Z 10.110.10.254 10.120.80.254 Minor Worm Propagation The host has scanned and connected on a particular port across more than one subnet, and the host was previously scanned and connected to by a host for which the Worm Activity alarm has been raised. Worm propagated from Source Host using smb (445/tcp)","@version":"1","@timestamp":"2014-07-17T16:17:05.174Z","type":"syslog","host":"10.122.196.201"}
+{"message":"<131>Jul 17 16:20:00 smc-01 StealthWatch[12365]: 2014-07-17T16:20:00Z 10.100.10.254 10.110.100.254 Minor Worm Propagation The host has scanned and connected on a particular port across more than one subnet, and the host was previously scanned and connected to by a host for which the Worm Activity alarm has been raised. Worm propagated from Source Host using smb (445/tcp)","@version":"1","@timestamp":"2014-07-17T16:17:05.174Z","type":"syslog","host":"10.122.196.201"}
+{"message":"<131>Jul 17 16:20:00 smc-01 StealthWatch[12365]: 2014-07-17T16:20:00Z 10.90.10.254 10.100.100.254 Minor Worm Propagation The host has scanned and connected on a particular port across more than one subnet, and the host was previously scanned and connected to by a host for which the Worm Activity alarm has been raised. Worm propagated from Source Host using smb (445/tcp)","@version":"1","@timestamp":"2014-07-17T16:17:05.174Z","type":"syslog","host":"10.122.196.201"}
+{"message":"<131>Jul 17 16:20:00 smc-01 StealthWatch[12365]: 2014-07-17T16:20:00Z 10.80.10.254 10.90.100.254 Minor Worm Propagation The host has scanned and connected on a particular port across more than one subnet, and the host was previously scanned and connected to by a host for which the Worm Activity alarm has been raised. Worm propagated from Source Host using smb (445/tcp)","@version":"1","@timestamp":"2014-07-17T16:17:05.175Z","type":"syslog","host":"10.122.196.201"}
+{"message":"<131>Jul 17 16:20:00 smc-01 StealthWatch[12365]: 2014-07-17T16:20:00Z 10.70.10.254 10.80.100.254 Minor Worm Propagation The host has scanned and connected on a particular port across more than one subnet, and the host was previously scanned and connected to by a host for which the Worm Activity alarm has been raised. Worm propagated from Source Host using smb (445/tcp)","@version":"1","@timestamp":"2014-07-17T16:17:05.183Z","type":"syslog","host":"10.122.196.201"}
+{"message":"<131>Jul 17 16:20:00 smc-01 StealthWatch[12365]: 2014-07-17T16:20:00Z 10.60.10.254 10.70.100.254 Minor Worm Propagation The host has scanned and connected on a particular port across more than one subnet, and the host was previously scanned and connected to by a host for which the Worm Activity alarm has been raised. Worm propagated from Source Host using smb (445/tcp)","@version":"1","@timestamp":"2014-07-17T16:17:05.184Z","type":"syslog","host":"10.122.196.201"}
+{"message":"<131>Jul 17 16:20:00 smc-01 StealthWatch[12365]: 2014-07-17T16:20:00Z 10.50.10.254 10.60.100.254 Minor Worm Propagation The host has scanned and connected on a particular port across more than one subnet, and the host was previously scanned and connected to by a host for which the Worm Activity alarm has been raised. Worm propagated from Source Host using smb (445/tcp)","@version":"1","@timestamp":"2014-07-17T16:17:05.184Z","type":"syslog","host":"10.122.196.201"}
+{"message":"<131>Jul 17 16:20:00 smc-01 StealthWatch[12365]: 2014-07-17T16:20:00Z 10.40.10.254 10.50.100.254 Minor Worm Propagation The host has scanned and connected on a particular port across more than one subnet, and the host was previously scanned and connected to by a host for which the Worm Activity alarm has been raised. Worm propagated from Source Host using smb (445/tcp)","@version":"1","@timestamp":"2014-07-17T16:17:05.184Z","type":"syslog","host":"10.122.196.201"}
+{"message":"<131>Jul 17 16:20:00 smc-01 StealthWatch[12365]: 2014-07-17T16:19:30Z 10.40.10.254 0.0.0.0 Minor High Concern Index The host's concern index has either exceeded the CI threshold or rapidly increased. Observed 36.63M points. Policy maximum allows up to 20M points.","@version":"1","@timestamp":"2014-07-17T16:17:05.168Z","type":"syslog","host":"192.249.113.37"}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/df8d682e/metron-platform/metron-integration-test/src/main/sample/data/SampleInput/PCAPExampleOutput
----------------------------------------------------------------------
diff --git a/metron-platform/metron-integration-test/src/main/sample/data/SampleInput/PCAPExampleOutput b/metron-platform/metron-integration-test/src/main/sample/data/SampleInput/PCAPExampleOutput
new file mode 100644
index 0000000..e730181
Binary files /dev/null and b/metron-platform/metron-integration-test/src/main/sample/data/SampleInput/PCAPExampleOutput differ

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/df8d682e/metron-platform/metron-integration-test/src/main/sample/data/SampleInput/PaloaltoOutput
----------------------------------------------------------------------
diff --git a/metron-platform/metron-integration-test/src/main/sample/data/SampleInput/PaloaltoOutput b/metron-platform/metron-integration-test/src/main/sample/data/SampleInput/PaloaltoOutput
new file mode 100644
index 0000000..16793a2
--- /dev/null
+++ b/metron-platform/metron-integration-test/src/main/sample/data/SampleInput/PaloaltoOutput
@@ -0,0 +1,100 @@
+<11>Jan  5 05:38:59 PAN1.exampleCustomer.com 1,2015/01/05 05:38:58,0006C110285,THREAT,vulnerability,1,2015/01/05 05:38:58,10.0.0.115,216.0.10.198,0.0.0.0,0.0.0.0,EX-Allow,example\user.name,,web-browsing,vsys1,internal,external,ethernet1/2,ethernet1/1,LOG-Default,2015/01/05 05:38:58,12031,1,54180,80,0,0,0x80004000,tcp,reset-both,"ad.aspx?f=300x250&id=12;tile=1;ord=67AF705D60B1119C0F18BEA336F9",HTTP: IIS Denial Of Service Attempt(40019),any,high,client-to-server,347368099,0x0,10.0.0.0-10.255.255.255,US,0,,1200568889751109656,, 
+<11>Jan  5 05:38:59 PAN1.exampleCustomer.com 1,2015/01/05 05:38:59,0006C110285,THREAT,vulnerability,1,2015/01/05 05:38:59,10.0.0.115,216.0.10.198,0.0.0.0,0.0.0.0,EX-Allow,example\user.name,,web-browsing,vsys1,internal,external,ethernet1/2,ethernet1/1,LOG-Default,2015/01/05 05:38:59,9399,1,54185,80,0,0,0x80004000,tcp,reset-both,"ad.aspx?f=300x250&id=8;tile=1;ord=F7315B6954238BE7FAE19D6EE0ECD",HTTP: IIS Denial Of Service Attempt(40019),any,high,client-to-server,347368106,0x0,10.0.0.0-10.255.255.255,US,0,,1200568889751109661,, 
+<11>Jan  5 05:39:00 PAN1.exampleCustomer.com 1,2015/01/05 05:38:59,0006C110285,THREAT,vulnerability,1,2015/01/05 05:38:59,10.0.0.115,216.0.10.198,0.0.0.0,0.0.0.0,EX-Allow,example\user.name,,web-browsing,vsys1,internal,external,ethernet1/2,ethernet1/1,LOG-Default,2015/01/05 05:38:59,50636,1,54181,80,0,0,0x80004000,tcp,reset-both,"ad.aspx?f=300x250&id=12;tile=1;ord=ECA531364D3B6522F9B89EE09381",HTTP: IIS Denial Of Service Attempt(40019),any,high,client-to-server,347368111,0x0,10.0.0.0-10.255.255.255,US,0,,1200568889751109663,, 
+<11>Jan  5 05:39:00 PAN1.exampleCustomer.com 1,2015/01/05 05:38:59,0006C110285,THREAT,vulnerability,1,2015/01/05 05:38:59,10.0.0.115,216.0.10.198,0.0.0.0,0.0.0.0,EX-Allow,example\user.name,,web-browsing,vsys1,internal,external,ethernet1/2,ethernet1/1,LOG-Default,2015/01/05 05:38:59,19582,1,54177,80,0,0,0x80004000,tcp,reset-both,"ad.aspx?f=160x600&id=14;tile=1;ord=9DB9E71EB91389C954E499B68203",HTTP: IIS Denial Of Service Attempt(40019),any,high,client-to-server,347368112,0x0,10.0.0.0-10.255.255.255,US,0,,1200568889751109664,, 
+<11>Jan  5 05:39:00 PAN1.exampleCustomer.com 1,2015/01/05 05:38:59,0006C110285,THREAT,vulnerability,1,2015/01/05 05:38:59,10.0.0.115,216.0.10.198,0.0.0.0,0.0.0.0,EX-Allow,example\user.name,,web-browsing,vsys1,internal,external,ethernet1/2,ethernet1/1,LOG-Default,2015/01/05 05:38:59,38426,1,54202,80,0,0,0x80004000,tcp,reset-both,"ad.aspx?f=336x288&id=4;tile=1;ord=B1B8DA9446290140922C4F6E092D8",HTTP: IIS Denial Of Service Attempt(40019),any,high,client-to-server,347368119,0x0,10.0.0.0-10.255.255.255,US,0,,1200568889751109668,, 
+<11>Jan  5 07:11:37 PAN1.exampleCustomer.com 1,2015/01/05 07:11:36,0006C110285,THREAT,vulnerability,1,2015/01/05 07:11:36,10.0.0.115,216.0.10.198,0.0.0.0,0.0.0.0,EX-Allow,example\user.name,,web-browsing,vsys1,internal,external,ethernet1/2,ethernet1/1,LOG-Default,2015/01/05 07:11:36,28124,1,56475,80,0,0,0x80004000,tcp,reset-both,"ad.aspx?f=300x250&id=6;tile=1;ord=E526836F078EB22491799C6373ED3",HTTP: IIS Denial Of Service Attempt(40019),any,high,client-to-server,347431967,0x0,10.0.0.0-10.255.255.255,US,0,,1200568889751109692,, 
+<11>Jan  5 07:11:37 PAN1.exampleCustomer.com 1,2015/01/05 07:11:37,0006C110285,THREAT,vulnerability,1,2015/01/05 07:11:37,10.0.0.115,216.0.10.198,0.0.0.0,0.0.0.0,EX-Allow,example\user.name,,web-browsing,vsys1,internal,external,ethernet1/2,ethernet1/1,LOG-Default,2015/01/05 07:11:37,36574,1,56485,80,0,0,0x80004000,tcp,reset-both,"ad.aspx?f=300x250&id=6;tile=1;ord=E526836F078EB22491799C6373ED3",HTTP: IIS Denial Of Service Attempt(40019),any,high,client-to-server,347431978,0x0,10.0.0.0-10.255.255.255,US,0,,1200568889751109694,, 
+<11>Jan  5 07:11:37 PAN1.exampleCustomer.com 1,2015/01/05 07:11:37,0006C110285,THREAT,vulnerability,1,2015/01/05 07:11:37,10.0.0.115,216.0.10.198,0.0.0.0,0.0.0.0,EX-Allow,example\user.name,,web-browsing,vsys1,internal,external,ethernet1/2,ethernet1/1,LOG-Default,2015/01/05 07:11:37,3892,1,56486,80,0,0,0x80004000,tcp,reset-both,"ad.aspx?f=300x250&id=12;tile=1;ord=E052042F211E553D6E1E44921E49",HTTP: IIS Denial Of Service Attempt(40019),any,high,client-to-server,347431979,0x0,10.0.0.0-10.255.255.255,US,0,,1200568889751109695,, 
+<11>Jan  5 07:15:23 PAN1.exampleCustomer.com 1,2015/01/05 07:15:23,0006C110285,THREAT,vulnerability,1,2015/01/05 07:15:23,10.0.0.115,216.0.10.230,0.0.0.0,0.0.0.0,EX-Allow,example\user.name,,web-browsing,vsys1,internal,external,ethernet1/2,ethernet1/1,LOG-Default,2015/01/05 07:15:23,15102,1,56706,80,0,0,0x80004000,tcp,reset-both,"ad.aspx?f=300x250&id=12;tile=1;ord=EB863BEB8809A5598F62C4CEDED7",HTTP: IIS Denial Of Service Attempt(40019),any,high,client-to-server,347434790,0x0,10.0.0.0-10.255.255.255,US,0,,1200568889751109701,, 
+<11>Jan  5 07:15:23 PAN1.exampleCustomer.com 1,2015/01/05 07:15:23,0006C110285,THREAT,vulnerability,1,2015/01/05 07:15:23,10.0.0.115,216.0.10.230,0.0.0.0,0.0.0.0,EX-Allow,example\user.name,,web-browsing,vsys1,internal,external,ethernet1/2,ethernet1/1,LOG-Default,2015/01/05 07:15:23,54920,1,56704,80,0,0,0x80004000,tcp,reset-both,"ad.aspx?f=300x250&id=12;tile=1;ord=4FB22ED5B7A0C344DB28AB34C1B3",HTTP: IIS Denial Of Service Attempt(40019),any,high,client-to-server,347434799,0x0,10.0.0.0-10.255.255.255,US,0,,1200568889751109706,, 
+<11>Jan  5 06:57:50 PAN1.exampleCustomer.com 1,2015/01/05 06:57:50,0006C110285,THREAT,vulnerability,1,2015/01/05 06:57:50,10.0.0.115,216.0.10.198,0.0.0.0,0.0.0.0,EX-Allow,example\user.name,,web-browsing,vsys1,internal,external,ethernet1/2,ethernet1/1,LOG-Default,2015/01/05 06:57:50,59603,1,56051,80,0,0,0x80004000,tcp,reset-both,"ad.aspx?f=300x250&id=12;tile=1;ord=6845CCF1045EE15B60F30B807684",HTTP: IIS Denial Of Service Attempt(40019),any,high,client-to-server,347421830,0x0,10.0.0.0-10.255.255.255,US,0,,1200568889751109684,, 
+<11>Jan  5 06:57:50 PAN1.exampleCustomer.com 1,2015/01/05 06:57:50,0006C110285,THREAT,vulnerability,1,2015/01/05 06:57:50,10.0.0.115,216.0.10.198,0.0.0.0,0.0.0.0,EX-Allow,example\user.name,,web-browsing,vsys1,internal,external,ethernet1/2,ethernet1/1,LOG-Default,2015/01/05 06:57:50,24223,1,56042,80,0,0,0x80004000,tcp,reset-both,"ad.aspx?f=300x250&id=12;tile=1;ord=256A9BBB8867977D118E2E511742",HTTP: IIS Denial Of Service Attempt(40019),any,high,client-to-server,347421831,0x0,10.0.0.0-10.255.255.255,US,0,,1200568889751109685,, 
+<11>Jan  5 06:57:50 PAN1.exampleCustomer.com 1,2015/01/05 06:57:50,0006C110285,THREAT,vulnerability,1,2015/01/05 06:57:50,10.0.0.115,216.0.10.198,0.0.0.0,0.0.0.0,EX-Allow,example\user.name,,web-browsing,vsys1,internal,external,ethernet1/2,ethernet1/1,LOG-Default,2015/01/05 06:57:50,61627,1,56043,80,0,0,0x80004000,tcp,reset-both,"ad.aspx?f=300x250&id=12;tile=1;ord=6845CCF1045EE15B60F30B807684",HTTP: IIS Denial Of Service Attempt(40019),any,high,client-to-server,347421828,0x0,10.0.0.0-10.255.255.255,US,0,,1200568889751109682,, 
+<11>Jan  5 07:11:36 PAN1.exampleCustomer.com 1,2015/01/05 07:11:36,0006C110285,THREAT,vulnerability,1,2015/01/05 07:11:36,10.0.0.115,216.0.10.198,0.0.0.0,0.0.0.0,EX-Allow,example\user.name,,web-browsing,vsys1,internal,external,ethernet1/2,ethernet1/1,LOG-Default,2015/01/05 07:11:36,37087,1,56307,80,0,0,0x80004000,tcp,reset-both,"ad.aspx?f=300x250&id=12;tile=1;ord=E052042F211E553D6E1E44921E49",HTTP: IIS Denial Of Service Attempt(40019),any,high,client-to-server,347431965,0x0,10.0.0.0-10.255.255.255,US,0,,1200568889751109691,, 
+<11>Jan  5 05:48:38 PAN1.exampleCustomer.com 1,2015/01/05 05:48:38,0006C110285,THREAT,vulnerability,1,2015/01/05 05:48:38,10.0.0.115,216.0.10.198,0.0.0.0,0.0.0.0,EX-Allow,example\user.name,,web-browsing,vsys1,internal,external,ethernet1/2,ethernet1/1,LOG-Default,2015/01/05 05:48:38,48136,1,54557,80,0,0,0x80004000,tcp,reset-both,"ad.aspx?f=300x250&id=12;tile=1;ord=EDD821C39BC0A49777874E02F7FA",HTTP: IIS Denial Of Service Attempt(40019),any,high,client-to-server,347373997,0x0,10.0.0.0-10.255.255.255,US,0,,1200568889751109676,, 
+<11>Jan  5 05:39:01 PAN1.exampleCustomer.com 1,2015/01/05 05:39:00,0006C110285,THREAT,vulnerability,1,2015/01/05 05:39:00,10.0.0.115,216.0.10.198,0.0.0.0,0.0.0.0,EX-Allow,example\user.name,,web-browsing,vsys1,internal,external,ethernet1/2,ethernet1/1,LOG-Default,2015/01/05 05:39:00,60649,1,54209,80,0,0,0x80004000,tcp,reset-both,"ad.aspx?f=728x90&id=1;tile=1;ord=6510BF66C3B427ED44AC521752E695",HTTP: IIS Denial Of Service Attempt(40019),any,high,client-to-server,347368140,0x0,10.0.0.0-10.255.255.255,US,0,,1200568889751109674,, 
+<12>Jan  5 06:41:35 PAN1.exampleCustomer.com 1,2015/01/05 06:41:34,0006C113118,THREAT,virus,1,2015/01/05 06:41:34,94.0.0.3,10.0.0.208,94.0.0.3,211.0.10.226,EX-Allow,,example\user.name,web-browsing,vsys1,untrust,trust,ethernet1/1,ethernet1/2,LOG-Default,2015/01/05 06:41:34,16864,2,80,60194,80,56595,0x404000,tcp,deny,"FreemakeVideoConverterSetup.exe",Virus/Win32.WGeneric.dyxeh(2367869),any,medium,server-to-client,40462931,0x0,GB,10.0.0.0-10.255.255.255,0,,0,, 
+<10>Jan  5 05:58:47 PAN1 1,2015/01/05 05:58:46,009401011564,THREAT,vulnerability,1,2015/01/05 05:58:46,10.0.0.38,10.3.0.31,0.0.0.0,0.0.0.0,INT_out,,,ms-ds-smb,vsys1,v_internal,v_external,ethernet1/4,ethernet1/3,LOG-Default,2015/01/05 05:58:46,44183,1,60510,445,0,0,0x80004000,tcp,reset-both,"",Microsoft Windows SMBv2 Remote Code Execution Vulnerability(32541),any,critical,client-to-server,724178,0x0,Unknown,Unknown,0,,1200515273392656547,, 
+<11>Jan  5 07:41:48 PAN1.exampleCustomer.com 1,2015/01/05 07:41:47,0006C110285,THREAT,vulnerability,1,2015/01/05 07:41:47,10.0.0.115,216.0.10.230,0.0.0.0,0.0.0.0,EX-Allow,example\user.name,,web-browsing,vsys1,internal,external,ethernet1/2,ethernet1/1,LOG-Default,2015/01/05 07:41:47,20240,1,65530,80,0,0,0x80004000,tcp,reset-both,"ad.aspx?f=300x250&id=12;tile=1;ord=9944D12C8FB4EB798036CAD371C6",HTTP: IIS Denial Of Service Attempt(40019),any,high,client-to-server,347454781,0x0,10.0.0.0-10.255.255.255,US,0,,1200568889751109719,, 
+<11>Jan  5 07:41:48 PAN1.exampleCustomer.com 1,2015/01/05 07:41:47,0006C110285,THREAT,vulnerability,1,2015/01/05 07:41:47,10.0.0.115,216.0.10.230,0.0.0.0,0.0.0.0,EX-Allow,example\user.name,,web-browsing,vsys1,internal,external,ethernet1/2,ethernet1/1,LOG-Default,2015/01/05 07:41:47,2518,1,65531,80,0,0,0x80004000,tcp,reset-both,"ad.aspx?f=300x250&id=8;tile=1;ord=E0827A4B1C6179DF64205E13AECDF",HTTP: IIS Denial Of Service Attempt(40019),any,high,client-to-server,347454775,0x0,10.0.0.0-10.255.255.255,US,0,,1200568889751109715,, 
+<12>Jan  5 09:08:53 PAN1.exampleCustomer.com 1,2015/01/05 09:08:52,0011C103117,THREAT,virus,1,2015/01/05 09:08:52,61.0.0.202,10.0.0.81,0.0.0.0,0.0.0.0,EX-Allow,,example\user.name,web-browsing,vsys1,v_external,v_internal,ethernet1/2,ethernet1/1,LOG-Default,2015/01/05 09:08:52,72686,1,80,60538,0,0,0x4000,tcp,deny,"haozip_v5.0_up6.exe",Virus/Win32.WGeneric.dpqqf(2516743),any,medium,server-to-client,3422073984,0x0,CN,10.0.0.0-10.255.255.255,0,,0,, 
+<12>Jan  5 09:10:14 PAN1.exampleCustomer.com 1,2015/01/05 09:10:13,001606003946,THREAT,virus,1,2015/01/05 09:10:13,8.30.222.22,10.0.0.109,8.30.222.22,172.13.0.21,EX-Allow,,example\user.name,web-browsing,vsys1,untrust,trust,ethernet1/1,ethernet1/2,LOG-Default,2015/01/05 09:10:13,17060,1,80,64672,80,21754,0x404000,tcp,deny,"youdaogouwu-3.13-dictupdate.exe",Virus/Win32.WGeneric.dyugt(2272380),any,medium,server-to-client,38698043,0x0,US,10.0.0.0-10.255.255.255,0,,0,, 
+<11>Jan  5 09:10:37 PAN1 1,2015/01/05 09:10:36,0003C105690,THREAT,vulnerability,1,2015/01/05 09:10:36,10.0.0.222,95.0.0.154,192.168.100.11,95.0.0.154,Guest_to_Internet,,,web-browsing,vsys1,GuestAccess,untrust,vlan.84,vlan.200,LOG-Default,2015/01/05 09:10:36,97395,1,59784,80,46548,80,0x80400000,tcp,reset-both,"8-134.0-87.0.zip",HTTP Unauthorized Brute-force Attack(40031),any,high,client-to-server,247195018,0x0,10.0.0.0-10.255.255.255,IT,0,,1200340530903386781,, 
+<11>Jan  5 09:02:24 PAN1 1,2015/01/05 09:02:24,0003C105690,THREAT,vulnerability,1,2015/01/05 09:02:24,10.0.0.222,95.0.0.154,192.168.100.11,95.0.0.154,Guest_to_Internet,,,web-browsing,vsys1,GuestAccess,untrust,vlan.84,vlan.200,LOG-Default,2015/01/05 09:02:24,137904,1,59762,80,7021,80,0x80400000,tcp,reset-both,"8-136.0-83.0.zip",HTTP Unauthorized Brute-force Attack(40031),any,high,client-to-server,247188168,0x0,10.0.0.0-10.255.255.255,IT,0,,1200340530903386777,, 
+<11>Jan  5 09:23:52 PAN1 1,2015/01/05 09:23:51,009401011564,THREAT,vulnerability,1,2015/01/05 09:23:51,10.0.0.135,10.1.0.42,0.0.0.0,0.0.0.0,INT_out,,,sccp,vsys1,v_internal,v_external,ethernet1/4,ethernet1/3,LOG-Default,2015/01/05 09:23:51,15299,1,49643,2000,0,0,0x80004000,tcp,reset-both,"",Digium Asterisk Skinny Channel NULL-Pointer Dereference Vulnerability(35378),any,high,client-to-server,732393,0x0,Unknown,Unknown,0,,1200515273392656561,, 
+<10>Jan  5 10:03:58 PAN1 1,2015/01/05 10:03:58,009401011564,THREAT,vulnerability,1,2015/01/05 10:03:58,10.0.0.38,10.3.0.37,0.0.0.0,0.0.0.0,INT_out,,,ms-ds-smb,vsys1,v_internal,v_external,ethernet1/4,ethernet1/3,LOG-Default,2015/01/05 10:03:58,57935,1,11648,445,0,0,0x80004000,tcp,reset-both,"",Microsoft Windows SMBv2 Remote Code Execution Vulnerability(32541),any,critical,client-to-server,733522,0x0,Unknown,Unknown,0,,1200515273392656570,, 
+<11>Jan  5 07:19:09 PAN1 1,2015/01/05 07:19:08,009401011564,THREAT,vulnerability,1,2015/01/05 07:19:08,10.0.0.135,10.1.0.42,0.0.0.0,0.0.0.0,INT_out,,,sccp,vsys1,v_internal,v_external,ethernet1/4,ethernet1/3,LOG-Default,2015/01/05 07:19:08,22557,1,49638,2000,0,0,0x80004000,tcp,reset-both,"",Digium Asterisk Skinny Channel NULL-Pointer Dereference Vulnerability(35378),any,high,client-to-server,727520,0x0,Unknown,Unknown,0,,1200515273392656555,, 
+<10>Jan  5 10:04:00 PAN1 1,2015/01/05 10:04:00,009401011564,THREAT,vulnerability,1,2015/01/05 10:04:00,10.0.0.38,10.2.0.40,0.0.0.0,0.0.0.0,INT_out,,,ms-ds-smb,vsys1,v_internal,v_external,ethernet1/4,ethernet1/3,LOG-Default,2015/01/05 10:04:00,37972,1,43861,445,0,0,0x80004000,tcp,reset-both,"",Microsoft Windows SMBv2 Remote Code Execution Vulnerability(32541),any,critical,client-to-server,733536,0x0,Unknown,Unknown,0,,1200515273392656584,, 
+<10>Jan  5 10:04:01 PAN1 1,2015/01/05 10:04:01,009401011564,THREAT,vulnerability,1,2015/01/05 10:04:01,10.0.0.38,172.13.0.68,0.0.0.0,0.0.0.0,INT_out,,,ms-ds-smb,vsys1,v_internal,v_external,ethernet1/4,ethernet1/3,LOG-Default,2015/01/05 10:04:01,49163,1,43869,445,0,0,0x80004000,tcp,reset-both,"",Microsoft Windows SMBv2 Remote Code Execution Vulnerability(32541),any,critical,client-to-server,733543,0x0,Unknown,US,0,,1200515273392656591,, 
+<10>Jan  5 02:16:00 PAN1.exampleCustomer.com 1,2015/01/05 02:16:00,009401009421,THREAT,spyware,1,2015/01/05 02:16:00,10.0.0.67,54.0.0.140,68.1.100.154,54.0.0.140,EX-Allow,,,web-browsing,vsys1,internal,external,ethernet1/1,ethernet1/2,LOG-Default,2015/01/05 02:16:00,2898,1,50429,80,13954,80,0x400000,tcp,reset-both,"install.ashx",WGeneric.Gen Command and Control Traffic(13600),any,critical,client-to-server,3841944,0x0,10.0.0.0-10.255.255.255,US,0,,0,, 
+<10>Jan  5 02:16:17 PAN1.exampleCustomer.com 1,2015/01/05 02:16:17,009401009421,THREAT,spyware,1,2015/01/05 02:16:17,10.0.0.67,54.0.0.140,68.1.100.154,54.0.0.140,EX-Allow,,,web-browsing,vsys1,internal,external,ethernet1/1,ethernet1/2,LOG-Default,2015/01/05 02:16:17,21959,1,50459,80,45933,80,0x400000,tcp,reset-both,"install.ashx",WGeneric.Gen Command and Control Traffic(13600),any,critical,client-to-server,3842040,0x0,10.0.0.0-10.255.255.255,US,0,,0,, 
+<10>Jan  5 10:55:21 PAN1.exampleCustomer.com 1,2015/01/05 10:55:21,0011C103117,THREAT,vulnerability,1,2015/01/05 10:55:21,172.13.0.44,10.0.0.48,0.0.0.0,0.0.0.0,EX-Allow,,,ssl,vsys1,v_dmz-external,v_dmz-internal,ethernet1/3,ethernet1/4,LOG-Default,2015/01/05 10:55:21,116502,1,55910,443,0,0,0x80004000,tcp,reset-both,"bar.exampleCustomer.com/",OpenSSL SSL/TLS MITM vulnerability(36485),any,critical,client-to-server,3422361316,0x0,NO,10.0.0.0-10.255.255.255,0,,1200269920802300348,, 
+<12>Jan  5 11:31:36 PAN1.exampleCustomer.com 1,2015/01/05 11:31:36,0011C103117,THREAT,vulnerability,1,2015/01/05 11:31:36,31.0.0.198,10.0.0.210,0.0.0.0,0.0.0.0,EX-Allow,,,twitter-base,vsys1,v_dmz-external,v_dmz-internal,ethernet1/3,ethernet1/4,LOG-Default,2015/01/05 11:31:36,181928,1,55325,443,0,0,0x80004000,tcp,alert,"foo.exampleCustomer.com/",OpenSSL TLS Malformed Heartbeat Request Found - Heartbleed(36397),any,medium,client-to-server,3422463820,0x0,CH,10.0.0.0-10.255.255.255,0,,1200269920802300355,, 
+<12>Jan  5 11:31:17 PAN1.exampleCustomer.com 1,2015/01/05 11:31:17,0011C103117,THREAT,vulnerability,1,2015/01/05 11:31:17,31.0.0.198,10.0.0.56,0.0.0.0,0.0.0.0,EX-Allow,,,twitter-base,vsys1,v_dmz-external,v_dmz-internal,ethernet1/3,ethernet1/4,LOG-Default,2015/01/05 11:31:17,33936654,1,55325,443,0,0,0x80004000,tcp,alert,"*.exampleCustomer.com/",OpenSSL TLS Malformed Heartbeat Request Found - Heartbleed(36397),any,medium,client-to-server,3422463061,0x0,CH,10.0.0.0-10.255.255.255,0,,1344385108878191554,, 
+<12>Jan  5 11:07:20 PAN1.exampleCustomer.com 1,2015/01/05 11:07:20,0011C103117,THREAT,vulnerability,1,2015/01/05 11:07:20,31.0.0.198,10.0.0.70,0.0.0.0,0.0.0.0,EX-EasyAV,,,twitter-base,vsys1,v_dmz-external,v_dmz-internal,ethernet1/3,ethernet1/4,LOG-Default,2015/01/05 11:07:20,142520,1,55325,443,0,0,0x4000,tcp,alert,"fizzbuzz.exampleCustomer.com/",OpenSSL TLS Malformed Heartbeat Request Found - Heartbleed(36397),any,medium,client-to-server,3422395620,0x0,CH,10.0.0.0-10.255.255.255,0,,0,, 
+<10>Jan  5 10:04:06 PAN1 1,2015/01/05 10:04:05,009401011564,THREAT,vulnerability,1,2015/01/05 10:04:05,10.0.0.38,10.2.0.20,0.0.0.0,0.0.0.0,INT_out,,,ms-ds-smb,vsys1,v_internal,v_external,ethernet1/4,ethernet1/3,LOG-Default,2015/01/05 10:04:05,58977,1,43882,445,0,0,0x80004000,tcp,reset-both,"",Microsoft Windows SMBv2 Remote Code Execution Vulnerability(32541),any,critical,client-to-server,733556,0x0,Unknown,Unknown,0,,1200515273392656603,, 
+<11>Jan  5 11:20:02 PAN1 1,2015/01/05 11:20:02,009401011564,THREAT,vulnerability,1,2015/01/05 11:20:02,10.0.0.131,10.1.0.42,0.0.0.0,0.0.0.0,INT_out,,,sccp,vsys1,v_internal,v_external,ethernet1/4,ethernet1/3,LOG-Default,2015/01/05 11:20:02,25219,1,49569,2000,0,0,0x80004000,tcp,reset-both,"",Digium Asterisk Skinny Channel NULL-Pointer Dereference Vulnerability(35378),any,high,client-to-server,735575,0x0,Unknown,Unknown,0,,1200515273392656605,, 
+<11>Jan  5 12:31:01 PAN1.exampleCustomer.com 1,2015/01/05 12:31:01,0006C110285,THREAT,vulnerability,1,2015/01/05 12:31:01,10.0.0.115,216.0.10.198,0.0.0.0,0.0.0.0,EX-Allow,example\user.name,,web-browsing,vsys1,internal,external,ethernet1/2,ethernet1/1,LOG-Default,2015/01/05 12:31:01,12971,1,56879,80,0,0,0x80004000,tcp,reset-both,"ad.aspx?f=160x600&id=14;tile=1;ord=339DEA400FDFBF9127DA196347F1",HTTP: IIS Denial Of Service Attempt(40019),any,high,client-to-server,347631498,0x0,10.0.0.0-10.255.255.255,US,0,,1200568889751109742,, 
+<11>Jan  5 12:31:01 PAN1.exampleCustomer.com 1,2015/01/05 12:31:01,0006C110285,THREAT,vulnerability,1,2015/01/05 12:31:01,10.0.0.115,216.0.10.198,0.0.0.0,0.0.0.0,EX-Allow,example\user.name,,web-browsing,vsys1,internal,external,ethernet1/2,ethernet1/1,LOG-Default,2015/01/05 12:31:01,52846,1,56881,80,0,0,0x80004000,tcp,reset-both,"ad.aspx?f=160x600&id=14;tile=1;ord=A501E1CAA93F3B256222F902C051",HTTP: IIS Denial Of Service Attempt(40019),any,high,client-to-server,347631499,0x0,10.0.0.0-10.255.255.255,US,0,,1200568889751109743,, 
+<11>Jan  5 12:31:01 PAN1.exampleCustomer.com 1,2015/01/05 12:31:01,0006C110285,THREAT,vulnerability,1,2015/01/05 12:31:01,10.0.0.115,216.0.10.198,0.0.0.0,0.0.0.0,EX-Allow,example\user.name,,web-browsing,vsys1,internal,external,ethernet1/2,ethernet1/1,LOG-Default,2015/01/05 12:31:01,132,1,56880,80,0,0,0x80004000,tcp,reset-both,"ad.aspx?f=300x250&id=12;tile=1;ord=A01019D3E75E253C81B9DBE60AF0",HTTP: IIS Denial Of Service Attempt(40019),any,high,client-to-server,347631500,0x0,10.0.0.0-10.255.255.255,US,0,,1200568889751109744,, 
+<11>Jan  5 11:39:28 PAN1.exampleCustomer.com 1,2015/01/05 11:39:28,0006C110285,THREAT,vulnerability,1,2015/01/05 11:39:28,10.0.0.115,216.0.10.198,0.0.0.0,0.0.0.0,EX-Allow,example\user.name,,web-browsing,vsys1,internal,external,ethernet1/2,ethernet1/1,LOG-Default,2015/01/05 11:39:28,55273,1,55241,80,0,0,0x80004000,tcp,reset-both,"ad.aspx?f=160x600&id=13;tile=1;ord=F20325FB397BD62AFCE60C004651",HTTP: IIS Denial Of Service Attempt(40019),any,high,client-to-server,347599433,0x0,10.0.0.0-10.255.255.255,US,0,,1200568889751109725,, 
+<11>Jan  5 12:09:04 PAN1.exampleCustomer.com 1,2015/01/05 12:09:03,0006C110285,THREAT,vulnerability,1,2015/01/05 12:09:03,10.0.0.115,216.0.10.198,0.0.0.0,0.0.0.0,EX-Allow,example\user.name,,web-browsing,vsys1,internal,external,ethernet1/2,ethernet1/1,LOG-Default,2015/01/05 12:09:03,40131,1,61994,80,0,0,0x80004000,tcp,reset-both,"ad.aspx?f=300x250&id=12;tile=1;ord=9C998477823511B311AA24EC53D6",HTTP: IIS Denial Of Service Attempt(40019),any,high,client-to-server,347617382,0x0,10.0.0.0-10.255.255.255,US,0,,1200568889751109736,, 
+<12>Jan  5 13:45:24 PAN1.exampleCustomer.com 1,2015/01/05 13:45:23,0011C103117,THREAT,vulnerability,1,2015/01/05 13:45:23,31.0.0.198,10.0.0.60,0.0.0.0,0.0.0.0,EX-Allow,,,twitter-base,vsys1,v_dmz-external,v_dmz-internal,ethernet1/3,ethernet1/4,LOG-Default,2015/01/05 13:45:23,179279,1,55325,443,0,0,0x80004000,tcp,alert,"*.exampleCustomer.com/",Unknown(36397),any,medium,client-to-server,3423036992,0x0,CH,10.0.0.0-10.255.255.255,0,,1200269920802300367,, 
+<12>Jan  5 13:45:24 PAN1.exampleCustomer.com 1,2015/01/05 13:45:23,0011C103117,THREAT,vulnerability,1,2015/01/05 13:45:23,10.0.0.10,10.1.0.81,0.0.0.0,0.0.0.0,EX-Allow,example\user.name,,twitter-base,vsys1,v_external,v_internal,ethernet1/2,ethernet1/1,LOG-Default,2015/01/05 13:45:23,32298,1,55325,443,0,0,0x80004000,tcp,alert,"*.exampleCustomer.com/",OpenSSL TLS Malformed Heartbeat Request Found - Heartbleed(36397),any,medium,client-to-server,3423036994,0x0,10.0.0.0-10.255.255.255,10.0.0.0-10.255.255.255,0,,1200269920802300369,, 
+<10>Jan  5 04:24:30 PAN1.exampleCustomer.com 1,2015/01/05 04:24:29,009401009421,THREAT,spyware,1,2015/01/05 04:24:29,10.0.0.67,54.0.0.133,68.1.100.154,54.0.0.133,EX-Allow,,,web-browsing,vsys1,internal,external,ethernet1/1,ethernet1/2,LOG-Default,2015/01/05 04:24:29,18282,1,49800,80,13532,80,0x400000,tcp,reset-both,"install.ashx",WGeneric.Gen Command and Control Traffic(13600),any,critical,client-to-server,3875271,0x0,10.0.0.0-10.255.255.255,US,0,,0,, 
+<12>Jan  5 11:32:12 PAN1.exampleCustomer.com 1,2015/01/05 11:32:12,0011C103117,THREAT,vulnerability,1,2015/01/05 11:32:12,31.0.0.198,10.0.0.102,0.0.0.0,0.0.0.0,EX-Allow,,,twitter-base,vsys1,v_dmz-external,v_dmz-internal,ethernet1/3,ethernet1/4,LOG-Default,2015/01/05 11:32:12,255259,1,55325,443,0,0,0x80004000,tcp,alert,"foo.exampleCustomer.com/",OpenSSL TLS Malformed Heartbeat Request Found - Heartbleed(36397),any,medium,client-to-server,3422465396,0x0,CH,10.0.0.0-10.255.255.255,0,,1200269920802300360,, 
+<12>Jan  5 11:31:46 PAN1.exampleCustomer.com 1,2015/01/05 11:31:46,0011C103117,THREAT,vulnerability,1,2015/01/05 11:31:46,31.0.0.198,10.0.0.50,0.0.0.0,0.0.0.0,EX-Allow,,,twitter-base,vsys1,v_dmz-external,v_dmz-internal,ethernet1/3,ethernet1/4,LOG-Default,2015/01/05 11:31:46,33699961,1,55325,443,0,0,0x80004000,tcp,alert,"*.exampleCustomer.com/",OpenSSL TLS Malformed Heartbeat Request Found - Heartbleed(36397),any,medium,client-to-server,3422464320,0x0,CH,10.0.0.0-10.255.255.255,0,,1344385108878191555,, 
+<12>Jan  5 11:36:03 PAN1.exampleCustomer.com 1,2015/01/05 11:36:02,0006C113555,THREAT,vulnerability,1,2015/01/05 11:36:02,10.0.0.62,10.1.0.11,0.0.0.0,0.0.0.0,EX-Allow,example\user.name,,msrpc,vsys1,Inside,Inside,ethernet1/4,tunnel.1,LOG-Default,2015/01/05 11:36:02,16469,1,51461,445,0,0,0x80004000,tcp,alert,"",Microsoft DCE RPC Big Endian Evasion Vulnerability(33510),any,medium,client-to-server,46375536,0x0,10.0.0.0-10.255.255.255,10.0.0.0-10.255.255.255,0,,1200283142590569503,, 
+<11>Jan  5 13:26:50 PAN1.exampleCustomer.com 1,2015/01/05 13:26:49,0011C103117,THREAT,vulnerability,1,2015/01/05 13:26:49,10.0.0.167,10.1.0.41,0.0.0.0,0.0.0.0,EX-EasyAV,example\user.name.hernandez,,ssh,vsys1,v_internal,v_external,ethernet1/1,ethernet1/2,LOG-Default,2015/01/05 13:26:49,121926,1,49754,9101,0,0,0x4000,tcp,reset-both,"",SSH User Authentication Brute-force Attempt(40015),any,high,client-to-server,3422922092,0x0,10.0.0.0-10.255.255.255,10.0.0.0-10.255.255.255,0,,0,, 
+<11>Jan  5 10:18:37 NTOR1FWPAN1 1,2015/01/05 10:18:37,009401008933,THREAT,vulnerability,1,2015/01/05 10:18:37,10.0.0.50,54.0.0.7,38.140.11.98,54.0.0.7,TOR-outbound,,,web-browsing,vsys1,Inside,Outside,ethernet1/2,ethernet1/1,LOG-Default,2015/01/05 10:18:37,7226,1,51724,80,58706,80,0x80400000,tcp,reset-both,"_PhotoXML.php",Microsoft Office Sharepoint Server Elevation of Privilege Vulnerability(32001),any,high,client-to-server,1252593,0x0,10.0.0.0-10.255.255.255,US,0,,1200584606076633093,, 
+<14>Jan  5 12:51:34 PAN1.exampleCustomer.com 1,2015/01/05 12:51:33,0011C103117,TRAFFIC,end,1,2015/01/05 12:51:33,10.0.0.53,10.1.0.174,0.0.0.0,0.0.0.0,EX-EasyAV2,,,mssql-db,vsys1,v_dmz-internal,v_dmz-external,ethernet1/4,ethernet1/3,LOG-Default,2015/01/05 12:51:33,34103936,1,54270,40004,0,0,0x401c,tcp,allow,5385,3299,2086,26,2015/01/05 12:51:01,30,any,0,17754932047,0x0,10.0.0.0-10.255.255.255,10.0.0.0-10.255.255.255,0,11,15 
+<14>Jan  5 12:51:34 PAN1.exampleCustomer.com 1,2015/01/05 12:51:33,0011C103117,TRAFFIC,end,1,2015/01/05 12:51:33,10.0.0.22,10.1.0.28,0.0.0.0,0.0.0.0,EX-Allow,,example\user.name,vmware,vsys1,v_external,v_internal,ethernet1/2,ethernet1/1,LOG-Default,2015/01/05 12:51:33,33888863,1,62961,902,0,0,0x4019,udp,allow,108,108,0,1,2015/01/05 12:51:01,30,any,0,17754932051,0x0,10.0.0.0-10.255.255.255,10.0.0.0-10.255.255.255,0,1,0 
+<14>Jan  5 12:51:34 PAN1.exampleCustomer.com 1,2015/01/05 12:51:33,0011C103117,TRAFFIC,end,1,2015/01/05 12:51:33,172.13.0.2,10.0.0.32,0.0.0.0,0.0.0.0,EX-Allow,,,dns,vsys1,v_dmz-external,v_dmz-internal,ethernet1/3,ethernet1/4,LOG-Default,2015/01/05 12:51:33,33841444,1,17294,53,0,0,0x4019,udp,allow,94,94,0,1,2015/01/05 12:51:01,30,any,0,17754932054,0x0,US,10.0.0.0-10.255.255.255,0,1,0 
+<14>Jan  5 12:51:34 PAN1.exampleCustomer.com 1,2015/01/05 12:51:33,0011C103117,TRAFFIC,end,1,2015/01/05 12:51:33,71.0.0.174,10.0.0.32,0.0.0.0,0.0.0.0,EX-Allow,,,dns,vsys1,v_dmz-external,v_dmz-internal,ethernet1/3,ethernet1/4,LOG-Default,2015/01/05 12:51:33,33992062,1,57783,53,0,0,0x4019,udp,allow,247,86,161,2,2015/01/05 12:51:01,30,any,0,17754932055,0x0,US,10.0.0.0-10.255.255.255,0,1,1 
+<14>Jan  5 12:51:34 PAN1.exampleCustomer.com 1,2015/01/05 12:51:33,0011C103117,TRAFFIC,end,1,2015/01/05 12:51:33,58.0.0.196,10.0.0.17,0.0.0.0,0.0.0.0,EX-Allow,,,ssl,vsys1,v_dmz-external,v_dmz-internal,ethernet1/3,ethernet1/4,LOG-Default,2015/01/05 12:51:33,34310602,1,25678,443,0,0,0x4053,tcp,allow,21935,11456,10479,44,2015/01/05 12:48:44,167,EX-Allowed,0,17754932059,0x0,IN,10.0.0.0-10.255.255.255,0,20,24 
+<14>Jan  5 12:51:34 PAN1.exampleCustomer.com 1,2015/01/05 12:51:33,0011C103117,TRAFFIC,end,1,2015/01/05 12:51:33,10.0.0.39,10.1.0.163,0.0.0.0,0.0.0.0,EX-Allow,,example\user.name,ms-ds-smb,vsys1,v_external,v_internal,ethernet1/2,ethernet1/1,LOG-Default,2015/01/05 12:51:33,33760927,1,52688,445,0,0,0x401a,tcp,allow,2229,1287,942,10,2015/01/05 12:51:01,30,any,0,17754932062,0x0,10.0.0.0-10.255.255.255,10.0.0.0-10.255.255.255,0,6,4 
+<14>Jan  5 12:51:34 PAN1.exampleCustomer.com 1,2015/01/05 12:51:33,0011C103117,TRAFFIC,end,1,2015/01/05 12:51:33,10.0.0.39,10.1.0.163,0.0.0.0,0.0.0.0,EX-Allow,,example\user.name,ms-ds-smb,vsys1,v_external,v_internal,ethernet1/2,ethernet1/1,LOG-Default,2015/01/05 12:51:33,33595018,1,52689,445,0,0,0x401a,tcp,allow,2229,1287,942,10,2015/01/05 12:51:01,30,any,0,17754932064,0x0,10.0.0.0-10.255.255.255,10.0.0.0-10.255.255.255,0,6,4 
+<14>Jan  5 12:51:34 PAN1.exampleCustomer.com 1,2015/01/05 12:51:33,0011C103117,TRAFFIC,end,1,2015/01/05 12:51:33,10.0.0.7,10.1.0.81,0.0.0.0,0.0.0.0,EX-Allow,,,netbios-ns,vsys1,v_external,v_internal,ethernet1/2,ethernet1/1,LOG-Default,2015/01/05 12:51:33,34098107,1,137,137,0,0,0x4019,udp,allow,532,220,312,6,2015/01/05 12:51:01,30,any,0,17754932070,0x0,10.0.0.0-10.255.255.255,10.0.0.0-10.255.255.255,0,3,3 
+<14>Jan  5 12:51:34 PAN1.exampleCustomer.com 1,2015/01/05 12:51:33,0011C103117,TRAFFIC,end,1,2015/01/05 12:51:33,10.0.0.39,10.1.0.163,0.0.0.0,0.0.0.0,EX-Allow,,example\user.name,ms-ds-smb,vsys1,v_external,v_internal,ethernet1/2,ethernet1/1,LOG-Default,2015/01/05 12:51:33,34326343,1,52690,445,0,0,0x401a,tcp,allow,2229,1287,942,10,2015/01/05 12:51:01,30,any,0,17754932071,0x0,10.0.0.0-10.255.255.255,10.0.0.0-10.255.255.255,0,6,4 
+<14>Jan  5 12:51:34 PAN1.exampleCustomer.com 1,2015/01/05 12:51:33,0011C103117,TRAFFIC,end,1,2015/01/05 12:51:33,172.13.0.15,10.0.0.53,0.0.0.0,0.0.0.0,EX-EasyAV,,,eset-remote-admin,vsys1,v_dmz-external,v_dmz-internal,ethernet1/3,ethernet1/4,LOG-Default,2015/01/05 12:51:33,33859365,1,23390,443,0,0,0x405e,tcp,allow,725,405,320,11,2015/01/05 12:51:01,30,any,0,17754932073,0x0,US,10.0.0.0-10.255.255.255,0,6,5 
+<14>Jan  5 12:51:34 PAN1.exampleCustomer.com 1,2015/01/05 12:51:33,0011C103117,TRAFFIC,end,1,2015/01/05 12:51:33,10.0.0.53,10.1.0.174,0.0.0.0,0.0.0.0,EX-EasyAV2,,,mssql-db,vsys1,v_external,v_internal,ethernet1/2,ethernet1/1,LOG-Default,2015/01/05 12:51:33,33621086,1,54266,40004,0,0,0x401c,tcp,allow,5325,3299,2026,25,2015/01/05 12:51:01,30,any,0,17754932075,0x0,10.0.0.0-10.255.255.255,10.0.0.0-10.255.255.255,0,11,14 
+<14>Jan  5 12:51:34 PAN1.exampleCustomer.com 1,2015/01/05 12:51:33,0011C103117,TRAFFIC,end,1,2015/01/05 12:51:33,96.0.0.102,10.0.0.57,0.0.0.0,0.0.0.0,EX-Allow,,,ssl,vsys1,v_dmz-external,v_dmz-internal,ethernet1/3,ethernet1/4,LOG-Default,2015/01/05 12:51:33,33924142,1,51230,443,0,0,0x4053,tcp,allow,18350,9280,9070,41,2015/01/05 12:51:01,30,EX-Allowed,0,17754932080,0x0,US,10.0.0.0-10.255.255.255,0,19,22 
+<14>Jan  5 12:51:34 PAN1.exampleCustomer.com 1,2015/01/05 12:51:33,0011C103117,TRAFFIC,end,1,2015/01/05 12:51:33,72.0.0.131,10.0.0.174,0.0.0.0,0.0.0.0,EX-Allow,,,ssl,vsys1,v_dmz-external,v_dmz-internal,ethernet1/3,ethernet1/4,LOG-Default,2015/01/05 12:51:33,34186774,1,28203,443,0,0,0x4053,tcp,allow,4121,2209,1912,20,2015/01/05 12:51:01,30,EX-Allowed,0,17754932086,0x0,US,10.0.0.0-10.255.255.255,0,10,10 
+<14>Jan  5 12:51:34 PAN1.exampleCustomer.com 1,2015/01/05 12:51:33,0011C103117,TRAFFIC,end,1,2015/01/05 12:51:33,216.0.10.244,10.0.0.53,0.0.0.0,0.0.0.0,EX-EasyAV,,,ssl,vsys1,v_dmz-external,v_dmz-internal,ethernet1/3,ethernet1/4,LOG-Default,2015/01/05 12:51:33,33988765,1,45150,443,0,0,0x401c,tcp,allow,626,358,268,9,2015/01/05 12:50:41,50,any,0,17754932095,0x0,US,10.0.0.0-10.255.255.255,0,5,4 
+<14>Jan  5 12:51:34 PAN1.exampleCustomer.com 1,2015/01/05 12:51:33,0011C103117,TRAFFIC,end,1,2015/01/05 12:51:33,172.12.216.82,10.0.0.53,0.0.0.0,0.0.0.0,EX-EasyAV,,,eset-update,vsys1,v_dmz-external,v_dmz-internal,ethernet1/3,ethernet1/4,LOG-Default,2015/01/05 12:51:33,33577240,1,3882,80,0,0,0x401c,tcp,allow,94947,2570,92377,106,2015/01/05 12:50:47,44,EX-Allowed,0,17754932107,0x0,US,10.0.0.0-10.255.255.255,0,38,68 
+<14>Jan  5 12:51:34 PAN1.exampleCustomer.com 1,2015/01/05 12:51:33,0011C103117,TRAFFIC,end,1,2015/01/05 12:51:33,10.0.0.33,10.1.0.85,0.0.0.0,0.0.0.0,EX-Allow,,,zabbix,vsys1,v_dmz-external,v_dmz-internal,ethernet1/3,ethernet1/4,LOG-Default,2015/01/05 12:51:33,34078885,1,46056,10050,0,0,0x405e,tcp,allow,728,367,361,11,2015/01/05 12:51:01,30,any,0,17754932117,0x0,10.0.0.0-10.255.255.255,10.0.0.0-10.255.255.255,0,6,5 
+<14>Jan  5 12:51:34 PAN1.exampleCustomer.com 1,2015/01/05 12:51:33,0011C103117,TRAFFIC,end,1,2015/01/05 12:51:33,10.0.0.11,10.1.0.33,0.0.0.0,0.0.0.0,EX-Allow,,,incomplete,vsys1,v_external,v_internal,ethernet1/2,ethernet1/1,LOG-Default,2015/01/05 12:51:33,34222137,1,59966,443,0,0,0x401c,tcp,allow,404,198,206,7,2015/01/05 12:51:01,30,any,0,17754932131,0x0,10.0.0.0-10.255.255.255,10.0.0.0-10.255.255.255,0,4,3 
+<14>Jan  5 12:51:34 PAN1.exampleCustomer.com 1,2015/01/05 12:51:33,0011C103117,TRAFFIC,end,1,2015/01/05 12:51:33,10.0.0.12,172.13.0.23,0.0.0.0,0.0.0.0,EX-Allow,,,dns,vsys1,v_dmz-internal,v_dmz-external,ethernet1/4,ethernet1/3,LOG-Default,2015/01/05 12:51:33,33560784,1,52991,53,0,0,0x4019,udp,allow,815,96,719,2,2015/01/05 12:51:01,30,any,0,17754932142,0x0,10.0.0.0-10.255.255.255,US,0,1,1 
+<14>Jan  5 12:51:34 PAN1.exampleCustomer.com 1,2015/01/05 12:51:33,0011C103117,TRAFFIC,end,1,2015/01/05 12:51:33,10.0.0.52,10.1.0.174,0.0.0.0,0.0.0.0,EX-EasyAV2,,,mssql-db,vsys1,v_dmz-internal,v_dmz-external,ethernet1/4,ethernet1/3,LOG-Default,2015/01/05 12:51:33,109384,1,50721,40004,0,0,0x401c,tcp,allow,4211,2125,2086,25,2015/01/05 12:51:02,30,any,0,17754932194,0x0,10.0.0.0-10.255.255.255,10.0.0.0-10.255.255.255,0,10,15 
+<14>Jan  5 12:51:34 PAN1.exampleCustomer.com 1,2015/01/05 12:51:33,0011C103117,TRAFFIC,end,1,2015/01/05 12:51:33,10.0.0.53,10.1.0.174,0.0.0.0,0.0.0.0,EX-EasyAV2,,,mssql-db,vsys1,v_external,v_internal,ethernet1/2,ethernet1/1,LOG-Default,2015/01/05 12:51:33,134519,1,54273,40004,0,0,0x401c,tcp,allow,5375,3289,2086,26,2015/01/05 12:51:02,30,any,0,17754932204,0x0,10.0.0.0-10.255.255.255,10.0.0.0-10.255.255.255,0,11,15 
+<14>Jan  5 12:51:34 PAN1.exampleCustomer.com 1,2015/01/05 12:51:33,0011C103117,TRAFFIC,end,1,2015/01/05 12:51:33,10.0.0.53,10.1.0.174,0.0.0.0,0.0.0.0,EX-EasyAV2,,,mssql-db,vsys1,v_external,v_internal,ethernet1/2,ethernet1/1,LOG-Default,2015/01/05 12:51:33,15005,1,54268,40004,0,0,0x401c,tcp,allow,7084,3787,3297,26,2015/01/05 12:51:02,30,any,0,17754932228,0x0,10.0.0.0-10.255.255.255,10.0.0.0-10.255.255.255,0,11,15 
+<14>Jan  5 12:51:34 PAN1 1,2015/01/05 12:51:33,0003C105690,TRAFFIC,drop,1,2015/01/05 12:51:33,10.0.0.219,10.3.0.21,0.0.0.0,0.0.0.0,catch all deny,,,not-applicable,vsys1,GuestAccess,trust,vlan.84,,LOG-Default,2015/01/05 12:51:33,0,1,62063,389,0,0,0x0,tcp,deny,70,70,0,1,2015/01/05 12:51:34,0,any,0,956329030,0x0,10.0.0.0-10.255.255.255,10.0.0.0-10.255.255.255,0,1,0 
+<14>Jan  5 12:51:34 PAN1.exampleCustomer.com 1,2015/01/05 12:51:33,0006C113555,TRAFFIC,end,1,2015/01/05 12:51:33,10.0.0.217,172.13.0.168,186.225.121.238,172.13.0.168,Guest WiFi to Internet,,,skype-probe,vsys1,Guest WiFi,Ext_Internet,ethernet1/3.109,ethernet1/2,LOG-Default,2015/01/05 12:51:33,46888,1,11566,40023,55962,40023,0x404050,udp,allow,1446,79,1367,2,2015/01/05 12:51:03,0,any,0,265102737,0x0,10.0.0.0-10.255.255.255,US,0,1,1 
+<14>Jan  5 12:51:34 PAN1.exampleCustomer.com 1,2015/01/05 12:51:33,0006C113555,TRAFFIC,end,1,2015/01/05 12:51:33,10.0.0.20,10.1.0.28,0.0.0.0,0.0.0.0,EX-Allow,,example\user.name,vmware,vsys1,Inside,Inside,ethernet1/4,tunnel.1,LOG-Default,2015/01/05 12:51:33,46821,1,61199,902,0,0,0x4019,udp,allow,108,108,0,1,2015/01/05 12:51:03,0,any,0,265102739,0x0,10.0.0.0-10.255.255.255,10.0.0.0-10.255.255.255,0,1,0 
+<14>Jan  5 12:51:34 PAN1 1,2015/01/05 12:51:33,0003C105690,TRAFFIC,end,1,2015/01/05 12:51:33,10.0.0.147,4.2.2.2,192.168.100.11,4.2.2.2,Guest_to_Internet,,,dns,vsys1,GuestAccess,untrust,vlan.84,vlan.200,LOG-Default,2015/01/05 12:51:33,188024,1,57269,53,59952,53,0x400019,udp,allow,194,73,121,2,2015/01/05 12:50:49,0,any,0,956329037,0x0,10.0.0.0-10.255.255.255,US,0,1,1 
+<14>Jan  5 12:51:34 PAN1.exampleCustomer.com 1,2015/01/05 12:51:33,0011C103117,TRAFFIC,end,1,2015/01/05 12:51:33,10.0.0.11,172.13.0.110,0.0.0.0,0.0.0.0,EX-Allow,,,dns,vsys1,v_internal,v_external,ethernet1/1,ethernet1/2,LOG-Default,2015/01/05 12:51:33,51569,1,60390,53,0,0,0x4019,udp,allow,815,96,719,2,2015/01/05 12:51:02,30,any,0,17754932369,0x0,10.0.0.0-10.255.255.255,US,0,1,1 
+<14>Jan  5 12:51:34 PAN1.exampleCustomer.com 1,2015/01/05 12:51:33,0011C103117,TRAFFIC,end,1,2015/01/05 12:51:33,10.0.0.11,10.1.0.81,0.0.0.0,0.0.0.0,EX-Allow,,,ping,vsys1,v_dmz-internal,v_dmz-external,ethernet1/4,ethernet1/3,LOG-Default,2015/01/05 12:51:33,185459,1,0,0,0,0,0x4019,icmp,allow,120,60,60,2,2015/01/05 12:51:29,0,any,0,17754932372,0x0,10.0.0.0-10.255.255.255,10.0.0.0-10.255.255.255,0,1,1 
+<14>Jan  5 12:51:34 PAN1.exampleCustomer.com 1,2015/01/05 12:51:33,0011C103117,TRAFFIC,end,1,2015/01/05 12:51:33,10.0.0.11,10.1.0.44,0.0.0.0,0.0.0.0,EX-Allow,,,ping,vsys1,v_external,v_internal,ethernet1/2,ethernet1/1,LOG-Default,2015/01/05 12:51:33,84730,1,0,0,0,0,0x4019,icmp,allow,120,60,60,2,2015/01/05 12:51:29,0,any,0,17754932379,0x0,10.0.0.0-10.255.255.255,10.0.0.0-10.255.255.255,0,1,1 
+<14>Jan  5 12:51:34 PAN1.exampleCustomer.com 1,2015/01/05 12:51:33,0006C110285,TRAFFIC,end,1,2015/01/05 12:51:33,10.0.0.73,10.1.0.12,0.0.0.0,0.0.0.0,EX-Allow,,,dns,vsys1,internal,external,ethernet1/2,ethernet1/1,LOG-Default,2015/01/05 12:51:33,12561,1,57334,53,0,0,0x4019,udp,allow,206,95,111,2,2015/01/05 12:51:03,0,any,0,803406326,0x0,10.0.0.0-10.255.255.255,10.0.0.0-10.255.255.255,0,1,1 
+<14>Jan  5 12:51:34 PAN1.exampleCustomer.com 1,2015/01/05 12:51:33,0006C110285,TRAFFIC,end,1,2015/01/05 12:51:33,10.0.0.35,10.3.0.65,0.0.0.0,0.0.0.0,EX-Allow,,,web-browsing,vsys1,external,internal,ethernet1/1,ethernet1/2,LOG-Default,2015/01/05 12:51:33,3286,1,57095,80,0,0,0x401c,tcp,allow,3506,899,2607,12,2015/01/05 12:51:03,0,private-ip-addresses,0,803406334,0x0,10.0.0.0-10.255.255.255,10.0.0.0-10.255.255.255,0,7,5 
+<14>Jan  5 12:51:34 PAN1.exampleCustomer.com 1,2015/01/05 12:51:33,0006C110285,TRAFFIC,end,1,2015/01/05 12:51:33,10.0.0.35,10.3.0.65,0.0.0.0,0.0.0.0,EX-Allow,,,web-browsing,vsys1,external,internal,ethernet1/1,ethernet1/2,LOG-Default,2015/01/05 12:51:33,42426,1,57096,80,0,0,0x401c,tcp,allow,3386,1390,1996,12,2015/01/05 12:51:03,0,private-ip-addresses,0,803406335,0x0,10.0.0.0-10.255.255.255,10.0.0.0-10.255.255.255,0,7,5 
+<14>Jan  5 12:51:34 PAN1.exampleCustomer.com 1,2015/01/05 12:51:33,0006C110285,TRAFFIC,end,1,2015/01/05 12:51:33,10.0.0.35,10.3.0.65,0.0.0.0,0.0.0.0,EX-Allow,,,web-browsing,vsys1,external,internal,ethernet1/1,ethernet1/2,LOG-Default,2015/01/05 12:51:33,15733,1,57130,80,0,0,0x401c,tcp,allow,1661,926,735,12,2015/01/05 12:51:03,0,private-ip-addresses,0,803406337,0x0,10.0.0.0-10.255.255.255,10.0.0.0-10.255.255.255,0,7,5 
+<14>Jan  5 12:51:34 PAN1.exampleCustomer.com 1,2015/01/05 12:51:33,0011C103117,TRAFFIC,end,1,2015/01/05 12:51:33,10.0.0.11,10.1.0.60,0.0.0.0,0.0.0.0,EX-Allow,,,ping,vsys1,v_external,v_internal,ethernet1/2,ethernet1/1,LOG-Default,2015/01/05 12:51:33,239420,1,0,0,0,0,0x4019,icmp,allow,120,60,60,2,2015/01/05 12:51:29,0,any,0,17754932383,0x0,10.0.0.0-10.255.255.255,10.0.0.0-10.255.255.255,0,1,1 
+<14>Jan  5 04:51:34 PAN1.exampleCustomer.com 1,2015/01/05 04:51:33,009401009421,TRAFFIC,end,1,2015/01/05 04:51:33,10.0.0.67,63.0.0.78,68.1.100.154,63.0.0.78,EX-Allow,,,web-browsing,vsys1,internal,external,ethernet1/1,ethernet1/2,LOG-Default,2015/01/05 04:51:33,13687,1,53152,80,64294,80,0x40001c,tcp,allow,1039,576,463,12,2015/01/05 04:51:03,1,search-engines,0,8195211,0x0,10.0.0.0-10.255.255.255,US,0,6,6 
+<14>Jan  5 04:51:34 PAN1.exampleCustomer.com 1,2015/01/05 04:51:33,009401009421,TRAFFIC,end,1,2015/01/05 04:51:33,10.0.0.67,77.0.0.59,68.1.100.154,77.0.0.59,EX-Allow,,,web-browsing,vsys1,internal,external,ethernet1/1,ethernet1/2,LOG-Default,2015/01/05 04:51:33,36193,1,53155,80,48756,80,0x40001c,tcp,allow,946,540,406,10,2015/01/05 04:51:04,0,computer-and-internet-security,0,8195212,0x0,10.0.0.0-10.255.255.255,CZ,0,5,5 
+<14>Jan  5 04:51:34 PAN1.exampleCustomer.com 1,2015/01/05 04:51:33,009401009421,TRAFFIC,end,1,2015/01/05 04:51:33,10.0.0.67,63.0.0.78,68.1.100.154,63.0.0.78,EX-Allow,,,web-browsing,vsys1,internal,external,ethernet1/1,ethernet1/2,LOG-Default,2015/01/05 04:51:33,8727,1,53154,80,6852,80,0x40001c,tcp,allow,1039,576,463,12,2015/01/05 04:51:04,0,search-engines,0,8195213,0x0,10.0.0.0-10.255.255.255,US,0,6,6 
+<14>Jan  5 04:51:34 PAN1.exampleCustomer.com 1,2015/01/05 04:51:33,009401009421,TRAFFIC,end,1,2015/01/05 04:51:33,10.0.0.67,77.0.0.59,68.1.100.154,77.0.0.59,EX-Allow,,,web-browsing,vsys1,internal,external,ethernet1/1,ethernet1/2,LOG-Default,2015/01/05 04:51:33,16955,1,53153,80,19440,80,0x40001c,tcp,allow,946,540,406,10,2015/01/05 04:51:03,1,computer-and-internet-security,0,8195216,0x0,10.0.0.0-10.255.255.255,CZ,0,5,5 
+<14>Jan  5 04:51:34 PAN1.exampleCustomer.com 1,2015/01/05 04:51:33,009401009421,TRAFFIC,end,1,2015/01/05 04:51:33,10.0.0.101,23.200,10,217,68.0.0.154,23.200,10,217,EX-WebControlRestrict,,,itunes-base,vsys1,internal,external,ethernet1/1,ethernet1/2,LOG-Default,2015/01/05 04:51:33,14851,1,55137,443,29553,443,0x400019,tcp,allow,654,580,74,7,2015/01/05 04:50:34,0,shopping,0,8195217,0x0,10.0.0.0-10.255.255.255,US,0,6,1 
+<14>Jan  5 12:51:34 PAN1.exampleCustomer.com 1,2015/01/05 12:51:34,0006C113555,TRAFFIC,end,1,2015/01/05 12:51:34,10.0.0.46,172.13.0.2,0.0.0.0,0.0.0.0,EX-Allow,example\user.name,,incomplete,vsys1,Inside,Inside,ethernet1/4,tunnel.1,LOG-Default,2015/01/05 12:51:34,57850,1,65286,139,0,0,0x4019,tcp,allow,62,62,0,1,2015/01/05 12:51:29,0,any,0,265102746,0x0,10.0.0.0-10.255.255.255,192.168.0.0-192.168.255.255,0,1,0 
+<14>Jan  5 12:51:34 PAN1 1,2015/01/05 12:51:34,0003C105690,TRAFFIC,end,1,2015/01/05 12:51:34,216.0.10.194,192.168.100.11,0.0.0.0,0.0.0.0,Internet to Internet,,,insufficient-data,vsys1,untrust,untrust,vlan.200,vlan.200,LOG-Default,2015/01/05 12:51:34,259007,1,80,11347,0,0,0xc,udp,allow,90,90,0,1,2015/01/05 12:50:25,0,any,0,956329050,0x0,US,US,0,1,0 
+<14>Jan  5 12:51:34 PAN1 1,2015/01/05 12:51:34,0003C105690,TRAFFIC,end,1,2015/01/05 12:51:34,10.0.0.147,4.2.2.2,192.168.100.11,4.2.2.2,Guest_to_Internet,,,dns,vsys1,GuestAccess,untrust,vlan.84,vlan.200,LOG-Default,2015/01/05 12:51:34,13024,1,56694,53,51398,53,0x400019,udp,allow,222,82,140,2,2015/01/05 12:50:49,0,any,0,956329055,0x0,10.0.0.0-10.255.255.255,US,0,1,1 
+<14>Jan  5 12:51:34 PAN1 1,2015/01/05 12:51:34,0003C105690,TRAFFIC,end,1,2015/01/05 12:51:34,10.0.0.147,4.2.2.2,192.168.100.11,4.2.2.2,Guest_to_Internet,,,dns,vsys1,GuestAccess,untrust,vlan.84,vlan.200,LOG-Default,2015/01/05 12:51:34,62999,1,58277,53,5576,53,0x400019,udp,allow,328,96,232,2,2015/01/05 12:50:49,0,any,0,956329056,0x0,10.0.0.0-10.255.255.255,US,0,1,1 
+<14>Jan  5 12:51:34 PAN1.exampleCustomer.com 1,2015/01/05 12:51:34,001606007155,TRAFFIC,end,1,2015/01/05 12:51:34,10.0.0.156,96.0.0.138,172.13.0.35,96.0.0.138,EX-Allow,example\user.name,,web-browsing,vsys1,trust,untrust,ethernet1/2,ethernet1/1,LOG-Default,2015/01/05 12:51:34,61348,1,65231,80,48623,80,0x40401a,tcp,allow,50316,4297,46019,67,2015/01/05 12:51:03,1,travel,0,179851307,0x0,10.0.0.0-10.255.255.255,US,0,28,39 
+<14>Jan  5 12:51:34 PAN1.exampleCustomer.com 1,2015/01/05 12:51:34,001606007155,TRAFFIC,end,1,2015/01/05 12:51:34,10.0.0.148,96.0.0.35,172.13.0.35,96.0.0.35,EX-Allow,example\user.name,,symantec-av-update,vsys1,trust,untrust,ethernet1/2,ethernet1/1,LOG-Default,2015/01/05 12:51:34,61220,1,60900,80,12964,80,0x40401a,tcp,allow,39350,3087,36263,56,2015/01/05 12:50:07,57,computer-and-internet-security,0,179851311,0x0,10.0.0.0-10.255.255.255,US,0,23,33 
+<14>Jan  5 12:51:34 PAN1 1,2015/01/05 12:51:34,009401003136,TRAFFIC,end,1,2015/01/05 12:51:34,10.0.0.138,213.0.10.101,172.13.0.142,213.0.10.101,Outbound,,,ssl,vsys1,internal,external,ethernet1/4,ethernet1/1,LOG-Default,2015/01/05 12:51:34,62600,1,55014,443,22537,443,0x40001c,tcp,allow,2956,1853,1103,20,2015/01/05 12:51:04,0,travel,0,54644537,0x0,10.0.0.0-10.255.255.255,CH,0,9,11 
+<14>Jan  5 12:51:34 PAN1 1,2015/01/05 12:51:34,009401003136,TRAFFIC,end,1,2015/01/05 12:51:34,10.0.0.138,213.0.10.101,172.13.0.142,213.0.10.101,Outbound,,,ssl,vsys1,internal,external,ethernet1/4,ethernet1/1,LOG-Default,2015/01/05 12:51:34,45328,1,55025,443,48646,443,0x40001c,tcp,allow,2828,1845,983,18,2015/01/05 12:51:04,0,travel,0,54644544,0x0,10.0.0.0-10.255.255.255,CH,0,9,9 
+<14>Jan  5 12:51:34 PAN1.exampleCustomer.com 1,2015/01/05 12:51:34,0004C103634,TRAFFIC,end,1,2015/01/05 12:51:34,10.0.0.165,93.0.0.200,0.0.0.0,0.0.0.0,EX-Allow,example\user.name,,ssl,vsys1,v_internal,v_external,ethernet1/2,ethernet1/1,LOG-Default,2015/01/05 12:51:34,15787,1,53105,443,0,0,0x4053,tcp,allow,10222,1275,8947,22,2015/01/05 12:48:03,181,business-and-economy,0,307579464,0x0,10.0.0.0-10.255.255.255,EU,0,10,12 
+<14>Jan  5 12:51:35 PAN1 1,2015/01/05 12:51:34,0003C105690,TRAFFIC,end,1,2015/01/05 12:51:34,10.0.0.11,10.3.0.26,0.0.0.0,0.0.0.0,ICMP DMZ to In,,,ping,vsys1,F5_DMZ_WAN,trust,vlan.81,vlan.399,LOG-Default,2015/01/05 12:51:34,33876,1,0,0,0,0,0x19,icmp,allow,128,64,64,2,2015/01/05 12:51:20,0,any,0,956329058,0x0,10.0.0.0-10.255.255.255,10.0.0.0-10.255.255.255,0,1,1 
+<14>Jan  5 12:51:35 PAN1.exampleCustomer.com 1,2015/01/05 12:51:34,0006C113555,TRAFFIC,end,1,2015/01/05 12:51:34,10.0.0.53,8.8.8.8,172.13.0.238,8.8.8.8,Guest WiFi to Internet,,,dns,vsys1,Guest WiFi,Ext_Internet,ethernet1/3.109,ethernet1/2,LOG-Default,2015/01/05 12:51:34,53079,1,59288,53,31746,53,0x404019,udp,allow,194,91,103,2,2015/01/05 12:51:04,0,any,0,265102750,0x0,10.0.0.0-10.255.255.255,US,0,1,1 
+<14>Jan  5 12:51:35 PAN1.exampleCustomer.com 1,2015/01/05 12:51:34,0011C103117,TRAFFIC,end,1,2015/01/05 12:51:34,10.0.0.53,10.1.0.174,0.0.0.0,0.0.0.0,EX-EasyAV2,,,mssql-db,vsys1,v_external,v_internal,ethernet1/2,ethernet1/1,LOG-Default,2015/01/05 12:51:34,141372,1,54279,40004,0,0,0x401c,tcp,allow,3783,1697,2086,25,2015/01/05 12:51:03,30,any,0,17754932394,0x0,10.0.0.0-10.255.255.255,10.0.0.0-10.255.255.255,0,10,15 

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/df8d682e/metron-platform/metron-integration-test/src/main/sample/data/SampleInput/SourcefireExampleOutput
----------------------------------------------------------------------
diff --git a/metron-platform/metron-integration-test/src/main/sample/data/SampleInput/SourcefireExampleOutput b/metron-platform/metron-integration-test/src/main/sample/data/SampleInput/SourcefireExampleOutput
new file mode 100644
index 0000000..5f177df
--- /dev/null
+++ b/metron-platform/metron-integration-test/src/main/sample/data/SampleInput/SourcefireExampleOutput
@@ -0,0 +1,2 @@
+SFIMS: [Primary Detection Engine (a7213248-6423-11e3-8537-fac6a92b7d9d)][MTD Access Control] Connection Type: Start, User: Unknown, Client: Unknown, Application Protocol: Unknown, Web App: Unknown, Firewall Rule Name: MTD Access Control, Firewall Rule Action: Allow, Firewall Rule Reasons: Unknown, URL Category: Unknown, URL_Reputation: Risk unknown, URL: Unknown, Interface Ingress: s1p1, Interface Egress: N/A, Security Zone Ingress: Unknown, Security Zone Egress: N/A, Security Intelligence Matching IP: None, Security Intelligence Category: None, {TCP} 72.163.0.129:60517 -> 10.1.128.236:443
+SFIMS: [Primary Detection Engine (a7213248-6423-11e3-8537-fac6a92b7d9d)][MTD Access Control] Connection Type: Start, User: Unknown, Client: Unknown, Application Protocol: Unknown, Web App: Unknown, Firewall Rule Name: MTD Access Control, Firewall Rule Action: Allow, Firewall Rule Reasons: Unknown, URL Category: Unknown, URL_Reputation: Risk unknown, URL: Unknown, Interface Ingress: s1p1, Interface Egress: N/A, Security Zone Ingress: Unknown, Security Zone Egress: N/A, Security Intelligence Matching IP: None, Security Intelligence Category: None, {TCP} 10.5.200.245:45283 -> 72.163.0.129:21
\ No newline at end of file


[09/15] incubator-metron git commit: METRON-142 Simplify Parser configuration (merrimanr via cestella) closes apache/incubator-metron#120

Posted by ce...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/df8d682e/metron-platform/metron-integration-test/src/main/resources/sample/data/SampleInput/LancopeExampleOutput
----------------------------------------------------------------------
diff --git a/metron-platform/metron-integration-test/src/main/resources/sample/data/SampleInput/LancopeExampleOutput b/metron-platform/metron-integration-test/src/main/resources/sample/data/SampleInput/LancopeExampleOutput
deleted file mode 100644
index b1bccf9..0000000
--- a/metron-platform/metron-integration-test/src/main/resources/sample/data/SampleInput/LancopeExampleOutput
+++ /dev/null
@@ -1,40 +0,0 @@
-{"message":"<131>Jul 17 15:27:27 smc-01 StealthWatch[12365]: 2014-06-24T14:37:58Z 192.168.200.9 199.237.198.232 Critical Bad Host The host has been observed doing something bad to another host. Source Host is http (80/tcp) client to target.host.name (199.237.198.232)","@version":"1","@timestamp":"2014-07-17T15:24:32.217Z","type":"syslog","host":"10.122.196.201"}
-{"message":"<131>Jul 17 15:35:00 smc-01 StealthWatch[12365]: 2014-07-17T15:34:30Z 10.201.3.83 0.0.0.0 Minor High Concern Index The host's concern index has either exceeded the CI threshold or rapidly increased. Observed 92.64M points. Policy maximum allows up to 20M points.","@version":"1","@timestamp":"2014-07-17T15:32:05.934Z","type":"syslog","host":"10.122.196.201"}
-{"message":"<131>Jul 17 15:35:00 smc-01 StealthWatch[12365]: 2014-07-17T15:34:30Z 10.201.3.145 0.0.0.0 Minor High Concern Index The host's concern index has either exceeded the CI threshold or rapidly increased. Observed 45.2M points. Policy maximum allows up to 20M points.","@version":"1","@timestamp":"2014-07-17T15:32:05.935Z","type":"syslog","host":"10.122.196.201"}
-{"message":"<131>Jul 17 15:35:00 smc-01 StealthWatch[12365]: 2014-07-17T15:34:30Z 10.201.3.50 0.0.0.0 Minor High Concern Index The host's concern index has either exceeded the CI threshold or rapidly increased. Observed 41.46M points. Policy maximum allows up to 20M points.","@version":"1","@timestamp":"2014-07-17T15:32:05.936Z","type":"syslog","host":"10.122.196.201"}
-{"message":"<131>Jul 17 15:42:01 smc-01 StealthWatch[12365]: 2014-07-17T15:42:00Z 10.10.101.24 0.0.0.0 Minor High Concern Index The host's concern index has either exceeded the CI threshold or rapidly increased. Observed 39.37M points. Policy maximum allows up to 20M points.","@version":"1","@timestamp":"2014-07-17T15:39:05.976Z","type":"syslog","host":"10.122.196.201"}
-{"message":"<131>Jul 17 15:56:01 smc-01 StealthWatch[12365]: 2014-07-17T15:55:00Z 0.0.100.0 0.0.0.0 Major ICMP Flood The source IP has sent an excessive number of ICMP packets in the last 5 minutes. Observed 262.4k pp5m. Policy maximum allows up to 100k pp5m.","@version":"1","@timestamp":"2014-07-17T15:53:05.995Z","type":"syslog","host":"10.122.196.201"}
-{"message":"<131>Jul 17 15:56:01 smc-01 StealthWatch[12365]: 2014-07-17T15:55:00Z 0.0.88.0 0.0.0.0 Major High Total Traffic The total traffic inbound + outbound exceeds the acceptable total traffic values. Observed 16.26G bytes. Expected 4.17G bytes, tolerance of 50 allows up to 15.06G bytes.","@version":"1","@timestamp":"2014-07-17T15:53:05.996Z","type":"syslog","host":"10.122.196.201"}
-{"message":"<131>Jul 17 15:57:01 smc-01 StealthWatch[12365]: 2014-07-17T15:56:30Z 10.201.3.50 0.0.0.0 Minor High Concern Index The host's concern index has either exceeded the CI threshold or rapidly increased. Observed 42.49M points. Policy maximum allows up to 20M points.","@version":"1","@timestamp":"2014-07-17T15:54:05.984Z","type":"syslog","host":"10.122.196.201"}
-{"message":"<131>Jul 17 15:59:01 smc-01 StealthWatch[12365]: 2014-07-17T15:58:30Z 10.40.10.254 0.0.0.0 Minor High Concern Index The host's concern index has either exceeded the CI threshold or rapidly increased. Observed 36.55M points. Policy maximum allows up to 20M points.","@version":"1","@timestamp":"2014-07-17T15:56:05.992Z","type":"syslog","host":"10.122.196.201"}
-{"message":"<131>Jul 17 15:59:01 smc-01 StealthWatch[12365]: 2014-07-17T15:58:30Z 10.30.10.254 0.0.0.0 Minor High Concern Index The host's concern index has either exceeded the CI threshold or rapidly increased. Observed 36.47M points. Policy maximum allows up to 20M points.","@version":"1","@timestamp":"2014-07-17T15:56:05.995Z","type":"syslog","host":"10.122.196.201"}
-{"message":"<131>Jul 17 15:59:01 smc-01 StealthWatch[12365]: 2014-07-17T15:58:30Z 10.20.10.254 0.0.0.0 Minor High Concern Index The host's concern index has either exceeded the CI threshold or rapidly increased. Observed 40.48M points. Policy maximum allows up to 20M points.","@version":"1","@timestamp":"2014-07-17T15:56:05.995Z","type":"syslog","host":"10.122.196.201"}
-{"message":"<131>Jul 17 15:59:01 smc-01 StealthWatch[12365]: 2014-07-17T15:58:30Z 10.201.3.83 0.0.0.0 Minor High Concern Index The host's concern index has either exceeded the CI threshold or rapidly increased. Observed 96.74M points. Policy maximum allows up to 20M points.","@version":"1","@timestamp":"2014-07-17T15:56:05.992Z","type":"syslog","host":"10.122.196.201"}
-{"message":"<131>Jul 17 15:59:01 smc-01 StealthWatch[12365]: 2014-07-17T15:58:30Z 10.100.10.254 0.0.0.0 Minor High Concern Index The host's concern index has either exceeded the CI threshold or rapidly increased. Observed 32.95M points. Policy maximum allows up to 20M points.","@version":"1","@timestamp":"2014-07-17T15:56:05.997Z","type":"syslog","host":"10.122.196.201"}
-{"message":"<131>Jul 17 15:59:01 smc-01 StealthWatch[12365]: 2014-07-17T15:58:30Z 10.90.10.254 0.0.0.0 Minor High Concern Index The host's concern index has either exceeded the CI threshold or rapidly increased. Observed 36.52M points. Policy maximum allows up to 20M points.","@version":"1","@timestamp":"2014-07-17T15:56:06.000Z","type":"syslog","host":"10.122.196.201"}
-{"message":"<131>Jul 17 15:59:01 smc-01 StealthWatch[12365]: 2014-07-17T15:58:30Z 10.80.10.254 0.0.0.0 Minor High Concern Index The host's concern index has either exceeded the CI threshold or rapidly increased. Observed 36.51M points. Policy maximum allows up to 20M points.","@version":"1","@timestamp":"2014-07-17T15:56:06.002Z","type":"syslog","host":"10.122.196.201"}
-{"message":"<131>Jul 17 15:59:01 smc-01 StealthWatch[12365]: 2014-07-17T15:58:30Z 10.70.10.254 0.0.0.0 Minor High Concern Index The host's concern index has either exceeded the CI threshold or rapidly increased. Observed 36.49M points. Policy maximum allows up to 20M points.","@version":"1","@timestamp":"2014-07-17T15:56:06.002Z","type":"syslog","host":"10.122.196.201"}
-{"message":"<131>Jul 17 15:59:01 smc-01 StealthWatch[12365]: 2014-07-17T15:58:30Z 10.110.10.254 0.0.0.0 Minor High Concern Index The host's concern index has either exceeded the CI threshold or rapidly increased. Observed 32.92M points. Policy maximum allows up to 20M points.","@version":"1","@timestamp":"2014-07-17T15:56:05.997Z","type":"syslog","host":"10.122.196.201"}
-{"message":"<131>Jul 17 15:59:01 smc-01 StealthWatch[12365]: 2014-07-17T15:58:30Z 10.60.10.254 0.0.0.0 Minor High Concern Index The host's concern index has either exceeded the CI threshold or rapidly increased. Observed 36.49M points. Policy maximum allows up to 20M points.","@version":"1","@timestamp":"2014-07-17T15:56:06.003Z","type":"syslog","host":"10.122.196.201"}
-{"message":"<131>Jul 17 15:59:01 smc-01 StealthWatch[12365]: 2014-07-17T15:58:30Z 10.50.10.254 0.0.0.0 Minor High Concern Index The host's concern index has either exceeded the CI threshold or rapidly increased. Observed 36.48M points. Policy maximum allows up to 20M points.","@version":"1","@timestamp":"2014-07-17T15:56:06.004Z","type":"syslog","host":"10.122.196.201"}
-{"message":"<131>Jul 17 16:06:01 smc-01 StealthWatch[12365]: 2014-07-17T16:05:00Z 10.10.101.46 0.0.0.0 Major New Flows Initiated The host has exceeded the acceptable total number of new flows initiated in a 5-minute period. ","@version":"1","@timestamp":"2014-07-17T16:03:06.046Z","type":"syslog","host":"10.122.196.201"}
-{"message":"<131>Jul 17 16:06:01 smc-01 StealthWatch[12365]: 2014-07-17T16:05:00Z 10.10.101.46 0.0.0.0 Major Max Flows Initiated The host has initiated more than an acceptable maximum number of flows. ","@version":"1","@timestamp":"2014-07-17T16:03:06.046Z","type":"syslog","host":"10.122.196.201"}
-{"message":"<131>Jul 17 16:20:00 smc-01 StealthWatch[12365]: 2014-07-17T16:19:30Z 10.110.10.254 0.0.0.0 Minor High Concern Index The host's concern index has either exceeded the CI threshold or rapidly increased. Observed 33.01M points. Policy maximum allows up to 20M points.","@version":"1","@timestamp":"2014-07-17T16:17:05.146Z","type":"syslog","host":"10.122.196.201"}
-{"message":"<131>Jul 17 16:20:00 smc-01 StealthWatch[12365]: 2014-07-17T16:19:30Z 10.100.10.254 0.0.0.0 Minor High Concern Index The host's concern index has either exceeded the CI threshold or rapidly increased. Observed 33.03M points. Policy maximum allows up to 20M points.","@version":"1","@timestamp":"2014-07-17T16:17:05.147Z","type":"syslog","host":"10.122.196.201"}
-{"message":"<131>Jul 17 16:20:00 smc-01 StealthWatch[12365]: 2014-07-17T16:19:30Z 10.90.10.254 0.0.0.0 Minor High Concern Index The host's concern index has either exceeded the CI threshold or rapidly increased. Observed 36.59M points. Policy maximum allows up to 20M points.","@version":"1","@timestamp":"2014-07-17T16:17:05.148Z","type":"syslog","host":"10.122.196.201"}
-{"message":"<131>Jul 17 16:20:00 smc-01 StealthWatch[12365]: 2014-07-17T16:19:30Z 10.80.10.254 0.0.0.0 Minor High Concern Index The host's concern index has either exceeded the CI threshold or rapidly increased. Observed 36.58M points. Policy maximum allows up to 20M points.","@version":"1","@timestamp":"2014-07-17T16:17:05.157Z","type":"syslog","host":"10.122.196.201"}
-{"message":"<131>Jul 17 16:20:00 smc-01 StealthWatch[12365]: 2014-07-17T16:19:30Z 10.70.10.254 0.0.0.0 Minor High Concern Index The host's concern index has either exceeded the CI threshold or rapidly increased. Observed 36.56M points. Policy maximum allows up to 20M points.","@version":"1","@timestamp":"2014-07-17T16:17:05.157Z","type":"syslog","host":"10.122.196.201"}
-{"message":"<131>Jul 17 16:20:00 smc-01 StealthWatch[12365]: 2014-07-17T16:19:30Z 10.60.10.254 0.0.0.0 Minor High Concern Index The host's concern index has either exceeded the CI threshold or rapidly increased. Observed 36.56M points. Policy maximum allows up to 20M points.","@version":"1","@timestamp":"2014-07-17T16:17:05.158Z","type":"syslog","host":"10.122.196.201"}
-{"message":"<131>Jul 17 16:20:00 smc-01 StealthWatch[12365]: 2014-07-17T16:19:30Z 10.50.10.254 0.0.0.0 Minor High Concern Index The host's concern index has either exceeded the CI threshold or rapidly increased. Observed 36.55M points. Policy maximum allows up to 20M points.","@version":"1","@timestamp":"2014-07-17T16:17:05.160Z","type":"syslog","host":"10.122.196.201"}
-{"message":"<131>Jul 17 16:20:00 smc-01 StealthWatch[12365]: 2014-07-17T16:19:30Z 10.30.10.254 0.0.0.0 Minor High Concern Index The host's concern index has either exceeded the CI threshold or rapidly increased. Observed 36.55M points. Policy maximum allows up to 20M points.","@version":"1","@timestamp":"2014-07-17T16:17:05.173Z","type":"syslog","host":"10.122.196.201"}
-{"message":"<131>Jul 17 16:20:00 smc-01 StealthWatch[12365]: 2014-07-17T16:19:30Z 10.201.3.83 0.0.0.0 Minor High Concern Index The host's concern index has either exceeded the CI threshold or rapidly increased. Observed 96.82M points. Policy maximum allows up to 20M points.","@version":"1","@timestamp":"2014-07-17T16:17:05.173Z","type":"syslog","host":"10.122.196.201"}
-{"message":"<131>Jul 17 16:20:00 smc-01 StealthWatch[12365]: 2014-07-17T16:19:30Z 10.20.10.254 0.0.0.0 Minor High Concern Index The host's concern index has either exceeded the CI threshold or rapidly increased. Observed 40.55M points. Policy maximum allows up to 20M points.","@version":"1","@timestamp":"2014-07-17T16:17:05.174Z","type":"syslog","host":"10.122.196.201"}
-{"message":"<131>Jul 17 16:20:00 smc-01 StealthWatch[12365]: 2014-07-17T16:20:00Z 10.110.10.254 10.120.80.254 Minor Worm Propagation The host has scanned and connected on a particular port across more than one subnet, and the host was previously scanned and connected to by a host for which the Worm Activity alarm has been raised. Worm propagated from Source Host using smb (445/tcp)","@version":"1","@timestamp":"2014-07-17T16:17:05.174Z","type":"syslog","host":"10.122.196.201"}
-{"message":"<131>Jul 17 16:20:00 smc-01 StealthWatch[12365]: 2014-07-17T16:20:00Z 10.100.10.254 10.110.100.254 Minor Worm Propagation The host has scanned and connected on a particular port across more than one subnet, and the host was previously scanned and connected to by a host for which the Worm Activity alarm has been raised. Worm propagated from Source Host using smb (445/tcp)","@version":"1","@timestamp":"2014-07-17T16:17:05.174Z","type":"syslog","host":"10.122.196.201"}
-{"message":"<131>Jul 17 16:20:00 smc-01 StealthWatch[12365]: 2014-07-17T16:20:00Z 10.90.10.254 10.100.100.254 Minor Worm Propagation The host has scanned and connected on a particular port across more than one subnet, and the host was previously scanned and connected to by a host for which the Worm Activity alarm has been raised. Worm propagated from Source Host using smb (445/tcp)","@version":"1","@timestamp":"2014-07-17T16:17:05.174Z","type":"syslog","host":"10.122.196.201"}
-{"message":"<131>Jul 17 16:20:00 smc-01 StealthWatch[12365]: 2014-07-17T16:20:00Z 10.80.10.254 10.90.100.254 Minor Worm Propagation The host has scanned and connected on a particular port across more than one subnet, and the host was previously scanned and connected to by a host for which the Worm Activity alarm has been raised. Worm propagated from Source Host using smb (445/tcp)","@version":"1","@timestamp":"2014-07-17T16:17:05.175Z","type":"syslog","host":"10.122.196.201"}
-{"message":"<131>Jul 17 16:20:00 smc-01 StealthWatch[12365]: 2014-07-17T16:20:00Z 10.70.10.254 10.80.100.254 Minor Worm Propagation The host has scanned and connected on a particular port across more than one subnet, and the host was previously scanned and connected to by a host for which the Worm Activity alarm has been raised. Worm propagated from Source Host using smb (445/tcp)","@version":"1","@timestamp":"2014-07-17T16:17:05.183Z","type":"syslog","host":"10.122.196.201"}
-{"message":"<131>Jul 17 16:20:00 smc-01 StealthWatch[12365]: 2014-07-17T16:20:00Z 10.60.10.254 10.70.100.254 Minor Worm Propagation The host has scanned and connected on a particular port across more than one subnet, and the host was previously scanned and connected to by a host for which the Worm Activity alarm has been raised. Worm propagated from Source Host using smb (445/tcp)","@version":"1","@timestamp":"2014-07-17T16:17:05.184Z","type":"syslog","host":"10.122.196.201"}
-{"message":"<131>Jul 17 16:20:00 smc-01 StealthWatch[12365]: 2014-07-17T16:20:00Z 10.50.10.254 10.60.100.254 Minor Worm Propagation The host has scanned and connected on a particular port across more than one subnet, and the host was previously scanned and connected to by a host for which the Worm Activity alarm has been raised. Worm propagated from Source Host using smb (445/tcp)","@version":"1","@timestamp":"2014-07-17T16:17:05.184Z","type":"syslog","host":"10.122.196.201"}
-{"message":"<131>Jul 17 16:20:00 smc-01 StealthWatch[12365]: 2014-07-17T16:20:00Z 10.40.10.254 10.50.100.254 Minor Worm Propagation The host has scanned and connected on a particular port across more than one subnet, and the host was previously scanned and connected to by a host for which the Worm Activity alarm has been raised. Worm propagated from Source Host using smb (445/tcp)","@version":"1","@timestamp":"2014-07-17T16:17:05.184Z","type":"syslog","host":"10.122.196.201"}
-{"message":"<131>Jul 17 16:20:00 smc-01 StealthWatch[12365]: 2014-07-17T16:19:30Z 10.40.10.254 0.0.0.0 Minor High Concern Index The host's concern index has either exceeded the CI threshold or rapidly increased. Observed 36.63M points. Policy maximum allows up to 20M points.","@version":"1","@timestamp":"2014-07-17T16:17:05.168Z","type":"syslog","host":"192.249.113.37"}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/df8d682e/metron-platform/metron-integration-test/src/main/resources/sample/data/SampleInput/PCAPExampleOutput
----------------------------------------------------------------------
diff --git a/metron-platform/metron-integration-test/src/main/resources/sample/data/SampleInput/PCAPExampleOutput b/metron-platform/metron-integration-test/src/main/resources/sample/data/SampleInput/PCAPExampleOutput
deleted file mode 100644
index e730181..0000000
Binary files a/metron-platform/metron-integration-test/src/main/resources/sample/data/SampleInput/PCAPExampleOutput and /dev/null differ

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/df8d682e/metron-platform/metron-integration-test/src/main/resources/sample/data/SampleInput/PaloaltoOutput
----------------------------------------------------------------------
diff --git a/metron-platform/metron-integration-test/src/main/resources/sample/data/SampleInput/PaloaltoOutput b/metron-platform/metron-integration-test/src/main/resources/sample/data/SampleInput/PaloaltoOutput
deleted file mode 100644
index 16793a2..0000000
--- a/metron-platform/metron-integration-test/src/main/resources/sample/data/SampleInput/PaloaltoOutput
+++ /dev/null
@@ -1,100 +0,0 @@
-<11>Jan  5 05:38:59 PAN1.exampleCustomer.com 1,2015/01/05 05:38:58,0006C110285,THREAT,vulnerability,1,2015/01/05 05:38:58,10.0.0.115,216.0.10.198,0.0.0.0,0.0.0.0,EX-Allow,example\user.name,,web-browsing,vsys1,internal,external,ethernet1/2,ethernet1/1,LOG-Default,2015/01/05 05:38:58,12031,1,54180,80,0,0,0x80004000,tcp,reset-both,"ad.aspx?f=300x250&id=12;tile=1;ord=67AF705D60B1119C0F18BEA336F9",HTTP: IIS Denial Of Service Attempt(40019),any,high,client-to-server,347368099,0x0,10.0.0.0-10.255.255.255,US,0,,1200568889751109656,, 
-<11>Jan  5 05:38:59 PAN1.exampleCustomer.com 1,2015/01/05 05:38:59,0006C110285,THREAT,vulnerability,1,2015/01/05 05:38:59,10.0.0.115,216.0.10.198,0.0.0.0,0.0.0.0,EX-Allow,example\user.name,,web-browsing,vsys1,internal,external,ethernet1/2,ethernet1/1,LOG-Default,2015/01/05 05:38:59,9399,1,54185,80,0,0,0x80004000,tcp,reset-both,"ad.aspx?f=300x250&id=8;tile=1;ord=F7315B6954238BE7FAE19D6EE0ECD",HTTP: IIS Denial Of Service Attempt(40019),any,high,client-to-server,347368106,0x0,10.0.0.0-10.255.255.255,US,0,,1200568889751109661,, 
-<11>Jan  5 05:39:00 PAN1.exampleCustomer.com 1,2015/01/05 05:38:59,0006C110285,THREAT,vulnerability,1,2015/01/05 05:38:59,10.0.0.115,216.0.10.198,0.0.0.0,0.0.0.0,EX-Allow,example\user.name,,web-browsing,vsys1,internal,external,ethernet1/2,ethernet1/1,LOG-Default,2015/01/05 05:38:59,50636,1,54181,80,0,0,0x80004000,tcp,reset-both,"ad.aspx?f=300x250&id=12;tile=1;ord=ECA531364D3B6522F9B89EE09381",HTTP: IIS Denial Of Service Attempt(40019),any,high,client-to-server,347368111,0x0,10.0.0.0-10.255.255.255,US,0,,1200568889751109663,, 
-<11>Jan  5 05:39:00 PAN1.exampleCustomer.com 1,2015/01/05 05:38:59,0006C110285,THREAT,vulnerability,1,2015/01/05 05:38:59,10.0.0.115,216.0.10.198,0.0.0.0,0.0.0.0,EX-Allow,example\user.name,,web-browsing,vsys1,internal,external,ethernet1/2,ethernet1/1,LOG-Default,2015/01/05 05:38:59,19582,1,54177,80,0,0,0x80004000,tcp,reset-both,"ad.aspx?f=160x600&id=14;tile=1;ord=9DB9E71EB91389C954E499B68203",HTTP: IIS Denial Of Service Attempt(40019),any,high,client-to-server,347368112,0x0,10.0.0.0-10.255.255.255,US,0,,1200568889751109664,, 
-<11>Jan  5 05:39:00 PAN1.exampleCustomer.com 1,2015/01/05 05:38:59,0006C110285,THREAT,vulnerability,1,2015/01/05 05:38:59,10.0.0.115,216.0.10.198,0.0.0.0,0.0.0.0,EX-Allow,example\user.name,,web-browsing,vsys1,internal,external,ethernet1/2,ethernet1/1,LOG-Default,2015/01/05 05:38:59,38426,1,54202,80,0,0,0x80004000,tcp,reset-both,"ad.aspx?f=336x288&id=4;tile=1;ord=B1B8DA9446290140922C4F6E092D8",HTTP: IIS Denial Of Service Attempt(40019),any,high,client-to-server,347368119,0x0,10.0.0.0-10.255.255.255,US,0,,1200568889751109668,, 
-<11>Jan  5 07:11:37 PAN1.exampleCustomer.com 1,2015/01/05 07:11:36,0006C110285,THREAT,vulnerability,1,2015/01/05 07:11:36,10.0.0.115,216.0.10.198,0.0.0.0,0.0.0.0,EX-Allow,example\user.name,,web-browsing,vsys1,internal,external,ethernet1/2,ethernet1/1,LOG-Default,2015/01/05 07:11:36,28124,1,56475,80,0,0,0x80004000,tcp,reset-both,"ad.aspx?f=300x250&id=6;tile=1;ord=E526836F078EB22491799C6373ED3",HTTP: IIS Denial Of Service Attempt(40019),any,high,client-to-server,347431967,0x0,10.0.0.0-10.255.255.255,US,0,,1200568889751109692,, 
-<11>Jan  5 07:11:37 PAN1.exampleCustomer.com 1,2015/01/05 07:11:37,0006C110285,THREAT,vulnerability,1,2015/01/05 07:11:37,10.0.0.115,216.0.10.198,0.0.0.0,0.0.0.0,EX-Allow,example\user.name,,web-browsing,vsys1,internal,external,ethernet1/2,ethernet1/1,LOG-Default,2015/01/05 07:11:37,36574,1,56485,80,0,0,0x80004000,tcp,reset-both,"ad.aspx?f=300x250&id=6;tile=1;ord=E526836F078EB22491799C6373ED3",HTTP: IIS Denial Of Service Attempt(40019),any,high,client-to-server,347431978,0x0,10.0.0.0-10.255.255.255,US,0,,1200568889751109694,, 
-<11>Jan  5 07:11:37 PAN1.exampleCustomer.com 1,2015/01/05 07:11:37,0006C110285,THREAT,vulnerability,1,2015/01/05 07:11:37,10.0.0.115,216.0.10.198,0.0.0.0,0.0.0.0,EX-Allow,example\user.name,,web-browsing,vsys1,internal,external,ethernet1/2,ethernet1/1,LOG-Default,2015/01/05 07:11:37,3892,1,56486,80,0,0,0x80004000,tcp,reset-both,"ad.aspx?f=300x250&id=12;tile=1;ord=E052042F211E553D6E1E44921E49",HTTP: IIS Denial Of Service Attempt(40019),any,high,client-to-server,347431979,0x0,10.0.0.0-10.255.255.255,US,0,,1200568889751109695,, 
-<11>Jan  5 07:15:23 PAN1.exampleCustomer.com 1,2015/01/05 07:15:23,0006C110285,THREAT,vulnerability,1,2015/01/05 07:15:23,10.0.0.115,216.0.10.230,0.0.0.0,0.0.0.0,EX-Allow,example\user.name,,web-browsing,vsys1,internal,external,ethernet1/2,ethernet1/1,LOG-Default,2015/01/05 07:15:23,15102,1,56706,80,0,0,0x80004000,tcp,reset-both,"ad.aspx?f=300x250&id=12;tile=1;ord=EB863BEB8809A5598F62C4CEDED7",HTTP: IIS Denial Of Service Attempt(40019),any,high,client-to-server,347434790,0x0,10.0.0.0-10.255.255.255,US,0,,1200568889751109701,, 
-<11>Jan  5 07:15:23 PAN1.exampleCustomer.com 1,2015/01/05 07:15:23,0006C110285,THREAT,vulnerability,1,2015/01/05 07:15:23,10.0.0.115,216.0.10.230,0.0.0.0,0.0.0.0,EX-Allow,example\user.name,,web-browsing,vsys1,internal,external,ethernet1/2,ethernet1/1,LOG-Default,2015/01/05 07:15:23,54920,1,56704,80,0,0,0x80004000,tcp,reset-both,"ad.aspx?f=300x250&id=12;tile=1;ord=4FB22ED5B7A0C344DB28AB34C1B3",HTTP: IIS Denial Of Service Attempt(40019),any,high,client-to-server,347434799,0x0,10.0.0.0-10.255.255.255,US,0,,1200568889751109706,, 
-<11>Jan  5 06:57:50 PAN1.exampleCustomer.com 1,2015/01/05 06:57:50,0006C110285,THREAT,vulnerability,1,2015/01/05 06:57:50,10.0.0.115,216.0.10.198,0.0.0.0,0.0.0.0,EX-Allow,example\user.name,,web-browsing,vsys1,internal,external,ethernet1/2,ethernet1/1,LOG-Default,2015/01/05 06:57:50,59603,1,56051,80,0,0,0x80004000,tcp,reset-both,"ad.aspx?f=300x250&id=12;tile=1;ord=6845CCF1045EE15B60F30B807684",HTTP: IIS Denial Of Service Attempt(40019),any,high,client-to-server,347421830,0x0,10.0.0.0-10.255.255.255,US,0,,1200568889751109684,, 
-<11>Jan  5 06:57:50 PAN1.exampleCustomer.com 1,2015/01/05 06:57:50,0006C110285,THREAT,vulnerability,1,2015/01/05 06:57:50,10.0.0.115,216.0.10.198,0.0.0.0,0.0.0.0,EX-Allow,example\user.name,,web-browsing,vsys1,internal,external,ethernet1/2,ethernet1/1,LOG-Default,2015/01/05 06:57:50,24223,1,56042,80,0,0,0x80004000,tcp,reset-both,"ad.aspx?f=300x250&id=12;tile=1;ord=256A9BBB8867977D118E2E511742",HTTP: IIS Denial Of Service Attempt(40019),any,high,client-to-server,347421831,0x0,10.0.0.0-10.255.255.255,US,0,,1200568889751109685,, 
-<11>Jan  5 06:57:50 PAN1.exampleCustomer.com 1,2015/01/05 06:57:50,0006C110285,THREAT,vulnerability,1,2015/01/05 06:57:50,10.0.0.115,216.0.10.198,0.0.0.0,0.0.0.0,EX-Allow,example\user.name,,web-browsing,vsys1,internal,external,ethernet1/2,ethernet1/1,LOG-Default,2015/01/05 06:57:50,61627,1,56043,80,0,0,0x80004000,tcp,reset-both,"ad.aspx?f=300x250&id=12;tile=1;ord=6845CCF1045EE15B60F30B807684",HTTP: IIS Denial Of Service Attempt(40019),any,high,client-to-server,347421828,0x0,10.0.0.0-10.255.255.255,US,0,,1200568889751109682,, 
-<11>Jan  5 07:11:36 PAN1.exampleCustomer.com 1,2015/01/05 07:11:36,0006C110285,THREAT,vulnerability,1,2015/01/05 07:11:36,10.0.0.115,216.0.10.198,0.0.0.0,0.0.0.0,EX-Allow,example\user.name,,web-browsing,vsys1,internal,external,ethernet1/2,ethernet1/1,LOG-Default,2015/01/05 07:11:36,37087,1,56307,80,0,0,0x80004000,tcp,reset-both,"ad.aspx?f=300x250&id=12;tile=1;ord=E052042F211E553D6E1E44921E49",HTTP: IIS Denial Of Service Attempt(40019),any,high,client-to-server,347431965,0x0,10.0.0.0-10.255.255.255,US,0,,1200568889751109691,, 
-<11>Jan  5 05:48:38 PAN1.exampleCustomer.com 1,2015/01/05 05:48:38,0006C110285,THREAT,vulnerability,1,2015/01/05 05:48:38,10.0.0.115,216.0.10.198,0.0.0.0,0.0.0.0,EX-Allow,example\user.name,,web-browsing,vsys1,internal,external,ethernet1/2,ethernet1/1,LOG-Default,2015/01/05 05:48:38,48136,1,54557,80,0,0,0x80004000,tcp,reset-both,"ad.aspx?f=300x250&id=12;tile=1;ord=EDD821C39BC0A49777874E02F7FA",HTTP: IIS Denial Of Service Attempt(40019),any,high,client-to-server,347373997,0x0,10.0.0.0-10.255.255.255,US,0,,1200568889751109676,, 
-<11>Jan  5 05:39:01 PAN1.exampleCustomer.com 1,2015/01/05 05:39:00,0006C110285,THREAT,vulnerability,1,2015/01/05 05:39:00,10.0.0.115,216.0.10.198,0.0.0.0,0.0.0.0,EX-Allow,example\user.name,,web-browsing,vsys1,internal,external,ethernet1/2,ethernet1/1,LOG-Default,2015/01/05 05:39:00,60649,1,54209,80,0,0,0x80004000,tcp,reset-both,"ad.aspx?f=728x90&id=1;tile=1;ord=6510BF66C3B427ED44AC521752E695",HTTP: IIS Denial Of Service Attempt(40019),any,high,client-to-server,347368140,0x0,10.0.0.0-10.255.255.255,US,0,,1200568889751109674,, 
-<12>Jan  5 06:41:35 PAN1.exampleCustomer.com 1,2015/01/05 06:41:34,0006C113118,THREAT,virus,1,2015/01/05 06:41:34,94.0.0.3,10.0.0.208,94.0.0.3,211.0.10.226,EX-Allow,,example\user.name,web-browsing,vsys1,untrust,trust,ethernet1/1,ethernet1/2,LOG-Default,2015/01/05 06:41:34,16864,2,80,60194,80,56595,0x404000,tcp,deny,"FreemakeVideoConverterSetup.exe",Virus/Win32.WGeneric.dyxeh(2367869),any,medium,server-to-client,40462931,0x0,GB,10.0.0.0-10.255.255.255,0,,0,, 
-<10>Jan  5 05:58:47 PAN1 1,2015/01/05 05:58:46,009401011564,THREAT,vulnerability,1,2015/01/05 05:58:46,10.0.0.38,10.3.0.31,0.0.0.0,0.0.0.0,INT_out,,,ms-ds-smb,vsys1,v_internal,v_external,ethernet1/4,ethernet1/3,LOG-Default,2015/01/05 05:58:46,44183,1,60510,445,0,0,0x80004000,tcp,reset-both,"",Microsoft Windows SMBv2 Remote Code Execution Vulnerability(32541),any,critical,client-to-server,724178,0x0,Unknown,Unknown,0,,1200515273392656547,, 
-<11>Jan  5 07:41:48 PAN1.exampleCustomer.com 1,2015/01/05 07:41:47,0006C110285,THREAT,vulnerability,1,2015/01/05 07:41:47,10.0.0.115,216.0.10.230,0.0.0.0,0.0.0.0,EX-Allow,example\user.name,,web-browsing,vsys1,internal,external,ethernet1/2,ethernet1/1,LOG-Default,2015/01/05 07:41:47,20240,1,65530,80,0,0,0x80004000,tcp,reset-both,"ad.aspx?f=300x250&id=12;tile=1;ord=9944D12C8FB4EB798036CAD371C6",HTTP: IIS Denial Of Service Attempt(40019),any,high,client-to-server,347454781,0x0,10.0.0.0-10.255.255.255,US,0,,1200568889751109719,, 
-<11>Jan  5 07:41:48 PAN1.exampleCustomer.com 1,2015/01/05 07:41:47,0006C110285,THREAT,vulnerability,1,2015/01/05 07:41:47,10.0.0.115,216.0.10.230,0.0.0.0,0.0.0.0,EX-Allow,example\user.name,,web-browsing,vsys1,internal,external,ethernet1/2,ethernet1/1,LOG-Default,2015/01/05 07:41:47,2518,1,65531,80,0,0,0x80004000,tcp,reset-both,"ad.aspx?f=300x250&id=8;tile=1;ord=E0827A4B1C6179DF64205E13AECDF",HTTP: IIS Denial Of Service Attempt(40019),any,high,client-to-server,347454775,0x0,10.0.0.0-10.255.255.255,US,0,,1200568889751109715,, 
-<12>Jan  5 09:08:53 PAN1.exampleCustomer.com 1,2015/01/05 09:08:52,0011C103117,THREAT,virus,1,2015/01/05 09:08:52,61.0.0.202,10.0.0.81,0.0.0.0,0.0.0.0,EX-Allow,,example\user.name,web-browsing,vsys1,v_external,v_internal,ethernet1/2,ethernet1/1,LOG-Default,2015/01/05 09:08:52,72686,1,80,60538,0,0,0x4000,tcp,deny,"haozip_v5.0_up6.exe",Virus/Win32.WGeneric.dpqqf(2516743),any,medium,server-to-client,3422073984,0x0,CN,10.0.0.0-10.255.255.255,0,,0,, 
-<12>Jan  5 09:10:14 PAN1.exampleCustomer.com 1,2015/01/05 09:10:13,001606003946,THREAT,virus,1,2015/01/05 09:10:13,8.30.222.22,10.0.0.109,8.30.222.22,172.13.0.21,EX-Allow,,example\user.name,web-browsing,vsys1,untrust,trust,ethernet1/1,ethernet1/2,LOG-Default,2015/01/05 09:10:13,17060,1,80,64672,80,21754,0x404000,tcp,deny,"youdaogouwu-3.13-dictupdate.exe",Virus/Win32.WGeneric.dyugt(2272380),any,medium,server-to-client,38698043,0x0,US,10.0.0.0-10.255.255.255,0,,0,, 
-<11>Jan  5 09:10:37 PAN1 1,2015/01/05 09:10:36,0003C105690,THREAT,vulnerability,1,2015/01/05 09:10:36,10.0.0.222,95.0.0.154,192.168.100.11,95.0.0.154,Guest_to_Internet,,,web-browsing,vsys1,GuestAccess,untrust,vlan.84,vlan.200,LOG-Default,2015/01/05 09:10:36,97395,1,59784,80,46548,80,0x80400000,tcp,reset-both,"8-134.0-87.0.zip",HTTP Unauthorized Brute-force Attack(40031),any,high,client-to-server,247195018,0x0,10.0.0.0-10.255.255.255,IT,0,,1200340530903386781,, 
-<11>Jan  5 09:02:24 PAN1 1,2015/01/05 09:02:24,0003C105690,THREAT,vulnerability,1,2015/01/05 09:02:24,10.0.0.222,95.0.0.154,192.168.100.11,95.0.0.154,Guest_to_Internet,,,web-browsing,vsys1,GuestAccess,untrust,vlan.84,vlan.200,LOG-Default,2015/01/05 09:02:24,137904,1,59762,80,7021,80,0x80400000,tcp,reset-both,"8-136.0-83.0.zip",HTTP Unauthorized Brute-force Attack(40031),any,high,client-to-server,247188168,0x0,10.0.0.0-10.255.255.255,IT,0,,1200340530903386777,, 
-<11>Jan  5 09:23:52 PAN1 1,2015/01/05 09:23:51,009401011564,THREAT,vulnerability,1,2015/01/05 09:23:51,10.0.0.135,10.1.0.42,0.0.0.0,0.0.0.0,INT_out,,,sccp,vsys1,v_internal,v_external,ethernet1/4,ethernet1/3,LOG-Default,2015/01/05 09:23:51,15299,1,49643,2000,0,0,0x80004000,tcp,reset-both,"",Digium Asterisk Skinny Channel NULL-Pointer Dereference Vulnerability(35378),any,high,client-to-server,732393,0x0,Unknown,Unknown,0,,1200515273392656561,, 
-<10>Jan  5 10:03:58 PAN1 1,2015/01/05 10:03:58,009401011564,THREAT,vulnerability,1,2015/01/05 10:03:58,10.0.0.38,10.3.0.37,0.0.0.0,0.0.0.0,INT_out,,,ms-ds-smb,vsys1,v_internal,v_external,ethernet1/4,ethernet1/3,LOG-Default,2015/01/05 10:03:58,57935,1,11648,445,0,0,0x80004000,tcp,reset-both,"",Microsoft Windows SMBv2 Remote Code Execution Vulnerability(32541),any,critical,client-to-server,733522,0x0,Unknown,Unknown,0,,1200515273392656570,, 
-<11>Jan  5 07:19:09 PAN1 1,2015/01/05 07:19:08,009401011564,THREAT,vulnerability,1,2015/01/05 07:19:08,10.0.0.135,10.1.0.42,0.0.0.0,0.0.0.0,INT_out,,,sccp,vsys1,v_internal,v_external,ethernet1/4,ethernet1/3,LOG-Default,2015/01/05 07:19:08,22557,1,49638,2000,0,0,0x80004000,tcp,reset-both,"",Digium Asterisk Skinny Channel NULL-Pointer Dereference Vulnerability(35378),any,high,client-to-server,727520,0x0,Unknown,Unknown,0,,1200515273392656555,, 
-<10>Jan  5 10:04:00 PAN1 1,2015/01/05 10:04:00,009401011564,THREAT,vulnerability,1,2015/01/05 10:04:00,10.0.0.38,10.2.0.40,0.0.0.0,0.0.0.0,INT_out,,,ms-ds-smb,vsys1,v_internal,v_external,ethernet1/4,ethernet1/3,LOG-Default,2015/01/05 10:04:00,37972,1,43861,445,0,0,0x80004000,tcp,reset-both,"",Microsoft Windows SMBv2 Remote Code Execution Vulnerability(32541),any,critical,client-to-server,733536,0x0,Unknown,Unknown,0,,1200515273392656584,, 
-<10>Jan  5 10:04:01 PAN1 1,2015/01/05 10:04:01,009401011564,THREAT,vulnerability,1,2015/01/05 10:04:01,10.0.0.38,172.13.0.68,0.0.0.0,0.0.0.0,INT_out,,,ms-ds-smb,vsys1,v_internal,v_external,ethernet1/4,ethernet1/3,LOG-Default,2015/01/05 10:04:01,49163,1,43869,445,0,0,0x80004000,tcp,reset-both,"",Microsoft Windows SMBv2 Remote Code Execution Vulnerability(32541),any,critical,client-to-server,733543,0x0,Unknown,US,0,,1200515273392656591,, 
-<10>Jan  5 02:16:00 PAN1.exampleCustomer.com 1,2015/01/05 02:16:00,009401009421,THREAT,spyware,1,2015/01/05 02:16:00,10.0.0.67,54.0.0.140,68.1.100.154,54.0.0.140,EX-Allow,,,web-browsing,vsys1,internal,external,ethernet1/1,ethernet1/2,LOG-Default,2015/01/05 02:16:00,2898,1,50429,80,13954,80,0x400000,tcp,reset-both,"install.ashx",WGeneric.Gen Command and Control Traffic(13600),any,critical,client-to-server,3841944,0x0,10.0.0.0-10.255.255.255,US,0,,0,, 
-<10>Jan  5 02:16:17 PAN1.exampleCustomer.com 1,2015/01/05 02:16:17,009401009421,THREAT,spyware,1,2015/01/05 02:16:17,10.0.0.67,54.0.0.140,68.1.100.154,54.0.0.140,EX-Allow,,,web-browsing,vsys1,internal,external,ethernet1/1,ethernet1/2,LOG-Default,2015/01/05 02:16:17,21959,1,50459,80,45933,80,0x400000,tcp,reset-both,"install.ashx",WGeneric.Gen Command and Control Traffic(13600),any,critical,client-to-server,3842040,0x0,10.0.0.0-10.255.255.255,US,0,,0,, 
-<10>Jan  5 10:55:21 PAN1.exampleCustomer.com 1,2015/01/05 10:55:21,0011C103117,THREAT,vulnerability,1,2015/01/05 10:55:21,172.13.0.44,10.0.0.48,0.0.0.0,0.0.0.0,EX-Allow,,,ssl,vsys1,v_dmz-external,v_dmz-internal,ethernet1/3,ethernet1/4,LOG-Default,2015/01/05 10:55:21,116502,1,55910,443,0,0,0x80004000,tcp,reset-both,"bar.exampleCustomer.com/",OpenSSL SSL/TLS MITM vulnerability(36485),any,critical,client-to-server,3422361316,0x0,NO,10.0.0.0-10.255.255.255,0,,1200269920802300348,, 
-<12>Jan  5 11:31:36 PAN1.exampleCustomer.com 1,2015/01/05 11:31:36,0011C103117,THREAT,vulnerability,1,2015/01/05 11:31:36,31.0.0.198,10.0.0.210,0.0.0.0,0.0.0.0,EX-Allow,,,twitter-base,vsys1,v_dmz-external,v_dmz-internal,ethernet1/3,ethernet1/4,LOG-Default,2015/01/05 11:31:36,181928,1,55325,443,0,0,0x80004000,tcp,alert,"foo.exampleCustomer.com/",OpenSSL TLS Malformed Heartbeat Request Found - Heartbleed(36397),any,medium,client-to-server,3422463820,0x0,CH,10.0.0.0-10.255.255.255,0,,1200269920802300355,, 
-<12>Jan  5 11:31:17 PAN1.exampleCustomer.com 1,2015/01/05 11:31:17,0011C103117,THREAT,vulnerability,1,2015/01/05 11:31:17,31.0.0.198,10.0.0.56,0.0.0.0,0.0.0.0,EX-Allow,,,twitter-base,vsys1,v_dmz-external,v_dmz-internal,ethernet1/3,ethernet1/4,LOG-Default,2015/01/05 11:31:17,33936654,1,55325,443,0,0,0x80004000,tcp,alert,"*.exampleCustomer.com/",OpenSSL TLS Malformed Heartbeat Request Found - Heartbleed(36397),any,medium,client-to-server,3422463061,0x0,CH,10.0.0.0-10.255.255.255,0,,1344385108878191554,, 
-<12>Jan  5 11:07:20 PAN1.exampleCustomer.com 1,2015/01/05 11:07:20,0011C103117,THREAT,vulnerability,1,2015/01/05 11:07:20,31.0.0.198,10.0.0.70,0.0.0.0,0.0.0.0,EX-EasyAV,,,twitter-base,vsys1,v_dmz-external,v_dmz-internal,ethernet1/3,ethernet1/4,LOG-Default,2015/01/05 11:07:20,142520,1,55325,443,0,0,0x4000,tcp,alert,"fizzbuzz.exampleCustomer.com/",OpenSSL TLS Malformed Heartbeat Request Found - Heartbleed(36397),any,medium,client-to-server,3422395620,0x0,CH,10.0.0.0-10.255.255.255,0,,0,, 
-<10>Jan  5 10:04:06 PAN1 1,2015/01/05 10:04:05,009401011564,THREAT,vulnerability,1,2015/01/05 10:04:05,10.0.0.38,10.2.0.20,0.0.0.0,0.0.0.0,INT_out,,,ms-ds-smb,vsys1,v_internal,v_external,ethernet1/4,ethernet1/3,LOG-Default,2015/01/05 10:04:05,58977,1,43882,445,0,0,0x80004000,tcp,reset-both,"",Microsoft Windows SMBv2 Remote Code Execution Vulnerability(32541),any,critical,client-to-server,733556,0x0,Unknown,Unknown,0,,1200515273392656603,, 
-<11>Jan  5 11:20:02 PAN1 1,2015/01/05 11:20:02,009401011564,THREAT,vulnerability,1,2015/01/05 11:20:02,10.0.0.131,10.1.0.42,0.0.0.0,0.0.0.0,INT_out,,,sccp,vsys1,v_internal,v_external,ethernet1/4,ethernet1/3,LOG-Default,2015/01/05 11:20:02,25219,1,49569,2000,0,0,0x80004000,tcp,reset-both,"",Digium Asterisk Skinny Channel NULL-Pointer Dereference Vulnerability(35378),any,high,client-to-server,735575,0x0,Unknown,Unknown,0,,1200515273392656605,, 
-<11>Jan  5 12:31:01 PAN1.exampleCustomer.com 1,2015/01/05 12:31:01,0006C110285,THREAT,vulnerability,1,2015/01/05 12:31:01,10.0.0.115,216.0.10.198,0.0.0.0,0.0.0.0,EX-Allow,example\user.name,,web-browsing,vsys1,internal,external,ethernet1/2,ethernet1/1,LOG-Default,2015/01/05 12:31:01,12971,1,56879,80,0,0,0x80004000,tcp,reset-both,"ad.aspx?f=160x600&id=14;tile=1;ord=339DEA400FDFBF9127DA196347F1",HTTP: IIS Denial Of Service Attempt(40019),any,high,client-to-server,347631498,0x0,10.0.0.0-10.255.255.255,US,0,,1200568889751109742,, 
-<11>Jan  5 12:31:01 PAN1.exampleCustomer.com 1,2015/01/05 12:31:01,0006C110285,THREAT,vulnerability,1,2015/01/05 12:31:01,10.0.0.115,216.0.10.198,0.0.0.0,0.0.0.0,EX-Allow,example\user.name,,web-browsing,vsys1,internal,external,ethernet1/2,ethernet1/1,LOG-Default,2015/01/05 12:31:01,52846,1,56881,80,0,0,0x80004000,tcp,reset-both,"ad.aspx?f=160x600&id=14;tile=1;ord=A501E1CAA93F3B256222F902C051",HTTP: IIS Denial Of Service Attempt(40019),any,high,client-to-server,347631499,0x0,10.0.0.0-10.255.255.255,US,0,,1200568889751109743,, 
-<11>Jan  5 12:31:01 PAN1.exampleCustomer.com 1,2015/01/05 12:31:01,0006C110285,THREAT,vulnerability,1,2015/01/05 12:31:01,10.0.0.115,216.0.10.198,0.0.0.0,0.0.0.0,EX-Allow,example\user.name,,web-browsing,vsys1,internal,external,ethernet1/2,ethernet1/1,LOG-Default,2015/01/05 12:31:01,132,1,56880,80,0,0,0x80004000,tcp,reset-both,"ad.aspx?f=300x250&id=12;tile=1;ord=A01019D3E75E253C81B9DBE60AF0",HTTP: IIS Denial Of Service Attempt(40019),any,high,client-to-server,347631500,0x0,10.0.0.0-10.255.255.255,US,0,,1200568889751109744,, 
-<11>Jan  5 11:39:28 PAN1.exampleCustomer.com 1,2015/01/05 11:39:28,0006C110285,THREAT,vulnerability,1,2015/01/05 11:39:28,10.0.0.115,216.0.10.198,0.0.0.0,0.0.0.0,EX-Allow,example\user.name,,web-browsing,vsys1,internal,external,ethernet1/2,ethernet1/1,LOG-Default,2015/01/05 11:39:28,55273,1,55241,80,0,0,0x80004000,tcp,reset-both,"ad.aspx?f=160x600&id=13;tile=1;ord=F20325FB397BD62AFCE60C004651",HTTP: IIS Denial Of Service Attempt(40019),any,high,client-to-server,347599433,0x0,10.0.0.0-10.255.255.255,US,0,,1200568889751109725,, 
-<11>Jan  5 12:09:04 PAN1.exampleCustomer.com 1,2015/01/05 12:09:03,0006C110285,THREAT,vulnerability,1,2015/01/05 12:09:03,10.0.0.115,216.0.10.198,0.0.0.0,0.0.0.0,EX-Allow,example\user.name,,web-browsing,vsys1,internal,external,ethernet1/2,ethernet1/1,LOG-Default,2015/01/05 12:09:03,40131,1,61994,80,0,0,0x80004000,tcp,reset-both,"ad.aspx?f=300x250&id=12;tile=1;ord=9C998477823511B311AA24EC53D6",HTTP: IIS Denial Of Service Attempt(40019),any,high,client-to-server,347617382,0x0,10.0.0.0-10.255.255.255,US,0,,1200568889751109736,, 
-<12>Jan  5 13:45:24 PAN1.exampleCustomer.com 1,2015/01/05 13:45:23,0011C103117,THREAT,vulnerability,1,2015/01/05 13:45:23,31.0.0.198,10.0.0.60,0.0.0.0,0.0.0.0,EX-Allow,,,twitter-base,vsys1,v_dmz-external,v_dmz-internal,ethernet1/3,ethernet1/4,LOG-Default,2015/01/05 13:45:23,179279,1,55325,443,0,0,0x80004000,tcp,alert,"*.exampleCustomer.com/",Unknown(36397),any,medium,client-to-server,3423036992,0x0,CH,10.0.0.0-10.255.255.255,0,,1200269920802300367,, 
-<12>Jan  5 13:45:24 PAN1.exampleCustomer.com 1,2015/01/05 13:45:23,0011C103117,THREAT,vulnerability,1,2015/01/05 13:45:23,10.0.0.10,10.1.0.81,0.0.0.0,0.0.0.0,EX-Allow,example\user.name,,twitter-base,vsys1,v_external,v_internal,ethernet1/2,ethernet1/1,LOG-Default,2015/01/05 13:45:23,32298,1,55325,443,0,0,0x80004000,tcp,alert,"*.exampleCustomer.com/",OpenSSL TLS Malformed Heartbeat Request Found - Heartbleed(36397),any,medium,client-to-server,3423036994,0x0,10.0.0.0-10.255.255.255,10.0.0.0-10.255.255.255,0,,1200269920802300369,, 
-<10>Jan  5 04:24:30 PAN1.exampleCustomer.com 1,2015/01/05 04:24:29,009401009421,THREAT,spyware,1,2015/01/05 04:24:29,10.0.0.67,54.0.0.133,68.1.100.154,54.0.0.133,EX-Allow,,,web-browsing,vsys1,internal,external,ethernet1/1,ethernet1/2,LOG-Default,2015/01/05 04:24:29,18282,1,49800,80,13532,80,0x400000,tcp,reset-both,"install.ashx",WGeneric.Gen Command and Control Traffic(13600),any,critical,client-to-server,3875271,0x0,10.0.0.0-10.255.255.255,US,0,,0,, 
-<12>Jan  5 11:32:12 PAN1.exampleCustomer.com 1,2015/01/05 11:32:12,0011C103117,THREAT,vulnerability,1,2015/01/05 11:32:12,31.0.0.198,10.0.0.102,0.0.0.0,0.0.0.0,EX-Allow,,,twitter-base,vsys1,v_dmz-external,v_dmz-internal,ethernet1/3,ethernet1/4,LOG-Default,2015/01/05 11:32:12,255259,1,55325,443,0,0,0x80004000,tcp,alert,"foo.exampleCustomer.com/",OpenSSL TLS Malformed Heartbeat Request Found - Heartbleed(36397),any,medium,client-to-server,3422465396,0x0,CH,10.0.0.0-10.255.255.255,0,,1200269920802300360,, 
-<12>Jan  5 11:31:46 PAN1.exampleCustomer.com 1,2015/01/05 11:31:46,0011C103117,THREAT,vulnerability,1,2015/01/05 11:31:46,31.0.0.198,10.0.0.50,0.0.0.0,0.0.0.0,EX-Allow,,,twitter-base,vsys1,v_dmz-external,v_dmz-internal,ethernet1/3,ethernet1/4,LOG-Default,2015/01/05 11:31:46,33699961,1,55325,443,0,0,0x80004000,tcp,alert,"*.exampleCustomer.com/",OpenSSL TLS Malformed Heartbeat Request Found - Heartbleed(36397),any,medium,client-to-server,3422464320,0x0,CH,10.0.0.0-10.255.255.255,0,,1344385108878191555,, 
-<12>Jan  5 11:36:03 PAN1.exampleCustomer.com 1,2015/01/05 11:36:02,0006C113555,THREAT,vulnerability,1,2015/01/05 11:36:02,10.0.0.62,10.1.0.11,0.0.0.0,0.0.0.0,EX-Allow,example\user.name,,msrpc,vsys1,Inside,Inside,ethernet1/4,tunnel.1,LOG-Default,2015/01/05 11:36:02,16469,1,51461,445,0,0,0x80004000,tcp,alert,"",Microsoft DCE RPC Big Endian Evasion Vulnerability(33510),any,medium,client-to-server,46375536,0x0,10.0.0.0-10.255.255.255,10.0.0.0-10.255.255.255,0,,1200283142590569503,, 
-<11>Jan  5 13:26:50 PAN1.exampleCustomer.com 1,2015/01/05 13:26:49,0011C103117,THREAT,vulnerability,1,2015/01/05 13:26:49,10.0.0.167,10.1.0.41,0.0.0.0,0.0.0.0,EX-EasyAV,example\user.name.hernandez,,ssh,vsys1,v_internal,v_external,ethernet1/1,ethernet1/2,LOG-Default,2015/01/05 13:26:49,121926,1,49754,9101,0,0,0x4000,tcp,reset-both,"",SSH User Authentication Brute-force Attempt(40015),any,high,client-to-server,3422922092,0x0,10.0.0.0-10.255.255.255,10.0.0.0-10.255.255.255,0,,0,, 
-<11>Jan  5 10:18:37 NTOR1FWPAN1 1,2015/01/05 10:18:37,009401008933,THREAT,vulnerability,1,2015/01/05 10:18:37,10.0.0.50,54.0.0.7,38.140.11.98,54.0.0.7,TOR-outbound,,,web-browsing,vsys1,Inside,Outside,ethernet1/2,ethernet1/1,LOG-Default,2015/01/05 10:18:37,7226,1,51724,80,58706,80,0x80400000,tcp,reset-both,"_PhotoXML.php",Microsoft Office Sharepoint Server Elevation of Privilege Vulnerability(32001),any,high,client-to-server,1252593,0x0,10.0.0.0-10.255.255.255,US,0,,1200584606076633093,, 
-<14>Jan  5 12:51:34 PAN1.exampleCustomer.com 1,2015/01/05 12:51:33,0011C103117,TRAFFIC,end,1,2015/01/05 12:51:33,10.0.0.53,10.1.0.174,0.0.0.0,0.0.0.0,EX-EasyAV2,,,mssql-db,vsys1,v_dmz-internal,v_dmz-external,ethernet1/4,ethernet1/3,LOG-Default,2015/01/05 12:51:33,34103936,1,54270,40004,0,0,0x401c,tcp,allow,5385,3299,2086,26,2015/01/05 12:51:01,30,any,0,17754932047,0x0,10.0.0.0-10.255.255.255,10.0.0.0-10.255.255.255,0,11,15 
-<14>Jan  5 12:51:34 PAN1.exampleCustomer.com 1,2015/01/05 12:51:33,0011C103117,TRAFFIC,end,1,2015/01/05 12:51:33,10.0.0.22,10.1.0.28,0.0.0.0,0.0.0.0,EX-Allow,,example\user.name,vmware,vsys1,v_external,v_internal,ethernet1/2,ethernet1/1,LOG-Default,2015/01/05 12:51:33,33888863,1,62961,902,0,0,0x4019,udp,allow,108,108,0,1,2015/01/05 12:51:01,30,any,0,17754932051,0x0,10.0.0.0-10.255.255.255,10.0.0.0-10.255.255.255,0,1,0 
-<14>Jan  5 12:51:34 PAN1.exampleCustomer.com 1,2015/01/05 12:51:33,0011C103117,TRAFFIC,end,1,2015/01/05 12:51:33,172.13.0.2,10.0.0.32,0.0.0.0,0.0.0.0,EX-Allow,,,dns,vsys1,v_dmz-external,v_dmz-internal,ethernet1/3,ethernet1/4,LOG-Default,2015/01/05 12:51:33,33841444,1,17294,53,0,0,0x4019,udp,allow,94,94,0,1,2015/01/05 12:51:01,30,any,0,17754932054,0x0,US,10.0.0.0-10.255.255.255,0,1,0 
-<14>Jan  5 12:51:34 PAN1.exampleCustomer.com 1,2015/01/05 12:51:33,0011C103117,TRAFFIC,end,1,2015/01/05 12:51:33,71.0.0.174,10.0.0.32,0.0.0.0,0.0.0.0,EX-Allow,,,dns,vsys1,v_dmz-external,v_dmz-internal,ethernet1/3,ethernet1/4,LOG-Default,2015/01/05 12:51:33,33992062,1,57783,53,0,0,0x4019,udp,allow,247,86,161,2,2015/01/05 12:51:01,30,any,0,17754932055,0x0,US,10.0.0.0-10.255.255.255,0,1,1 
-<14>Jan  5 12:51:34 PAN1.exampleCustomer.com 1,2015/01/05 12:51:33,0011C103117,TRAFFIC,end,1,2015/01/05 12:51:33,58.0.0.196,10.0.0.17,0.0.0.0,0.0.0.0,EX-Allow,,,ssl,vsys1,v_dmz-external,v_dmz-internal,ethernet1/3,ethernet1/4,LOG-Default,2015/01/05 12:51:33,34310602,1,25678,443,0,0,0x4053,tcp,allow,21935,11456,10479,44,2015/01/05 12:48:44,167,EX-Allowed,0,17754932059,0x0,IN,10.0.0.0-10.255.255.255,0,20,24 
-<14>Jan  5 12:51:34 PAN1.exampleCustomer.com 1,2015/01/05 12:51:33,0011C103117,TRAFFIC,end,1,2015/01/05 12:51:33,10.0.0.39,10.1.0.163,0.0.0.0,0.0.0.0,EX-Allow,,example\user.name,ms-ds-smb,vsys1,v_external,v_internal,ethernet1/2,ethernet1/1,LOG-Default,2015/01/05 12:51:33,33760927,1,52688,445,0,0,0x401a,tcp,allow,2229,1287,942,10,2015/01/05 12:51:01,30,any,0,17754932062,0x0,10.0.0.0-10.255.255.255,10.0.0.0-10.255.255.255,0,6,4 
-<14>Jan  5 12:51:34 PAN1.exampleCustomer.com 1,2015/01/05 12:51:33,0011C103117,TRAFFIC,end,1,2015/01/05 12:51:33,10.0.0.39,10.1.0.163,0.0.0.0,0.0.0.0,EX-Allow,,example\user.name,ms-ds-smb,vsys1,v_external,v_internal,ethernet1/2,ethernet1/1,LOG-Default,2015/01/05 12:51:33,33595018,1,52689,445,0,0,0x401a,tcp,allow,2229,1287,942,10,2015/01/05 12:51:01,30,any,0,17754932064,0x0,10.0.0.0-10.255.255.255,10.0.0.0-10.255.255.255,0,6,4 
-<14>Jan  5 12:51:34 PAN1.exampleCustomer.com 1,2015/01/05 12:51:33,0011C103117,TRAFFIC,end,1,2015/01/05 12:51:33,10.0.0.7,10.1.0.81,0.0.0.0,0.0.0.0,EX-Allow,,,netbios-ns,vsys1,v_external,v_internal,ethernet1/2,ethernet1/1,LOG-Default,2015/01/05 12:51:33,34098107,1,137,137,0,0,0x4019,udp,allow,532,220,312,6,2015/01/05 12:51:01,30,any,0,17754932070,0x0,10.0.0.0-10.255.255.255,10.0.0.0-10.255.255.255,0,3,3 
-<14>Jan  5 12:51:34 PAN1.exampleCustomer.com 1,2015/01/05 12:51:33,0011C103117,TRAFFIC,end,1,2015/01/05 12:51:33,10.0.0.39,10.1.0.163,0.0.0.0,0.0.0.0,EX-Allow,,example\user.name,ms-ds-smb,vsys1,v_external,v_internal,ethernet1/2,ethernet1/1,LOG-Default,2015/01/05 12:51:33,34326343,1,52690,445,0,0,0x401a,tcp,allow,2229,1287,942,10,2015/01/05 12:51:01,30,any,0,17754932071,0x0,10.0.0.0-10.255.255.255,10.0.0.0-10.255.255.255,0,6,4 
-<14>Jan  5 12:51:34 PAN1.exampleCustomer.com 1,2015/01/05 12:51:33,0011C103117,TRAFFIC,end,1,2015/01/05 12:51:33,172.13.0.15,10.0.0.53,0.0.0.0,0.0.0.0,EX-EasyAV,,,eset-remote-admin,vsys1,v_dmz-external,v_dmz-internal,ethernet1/3,ethernet1/4,LOG-Default,2015/01/05 12:51:33,33859365,1,23390,443,0,0,0x405e,tcp,allow,725,405,320,11,2015/01/05 12:51:01,30,any,0,17754932073,0x0,US,10.0.0.0-10.255.255.255,0,6,5 
-<14>Jan  5 12:51:34 PAN1.exampleCustomer.com 1,2015/01/05 12:51:33,0011C103117,TRAFFIC,end,1,2015/01/05 12:51:33,10.0.0.53,10.1.0.174,0.0.0.0,0.0.0.0,EX-EasyAV2,,,mssql-db,vsys1,v_external,v_internal,ethernet1/2,ethernet1/1,LOG-Default,2015/01/05 12:51:33,33621086,1,54266,40004,0,0,0x401c,tcp,allow,5325,3299,2026,25,2015/01/05 12:51:01,30,any,0,17754932075,0x0,10.0.0.0-10.255.255.255,10.0.0.0-10.255.255.255,0,11,14 
-<14>Jan  5 12:51:34 PAN1.exampleCustomer.com 1,2015/01/05 12:51:33,0011C103117,TRAFFIC,end,1,2015/01/05 12:51:33,96.0.0.102,10.0.0.57,0.0.0.0,0.0.0.0,EX-Allow,,,ssl,vsys1,v_dmz-external,v_dmz-internal,ethernet1/3,ethernet1/4,LOG-Default,2015/01/05 12:51:33,33924142,1,51230,443,0,0,0x4053,tcp,allow,18350,9280,9070,41,2015/01/05 12:51:01,30,EX-Allowed,0,17754932080,0x0,US,10.0.0.0-10.255.255.255,0,19,22 
-<14>Jan  5 12:51:34 PAN1.exampleCustomer.com 1,2015/01/05 12:51:33,0011C103117,TRAFFIC,end,1,2015/01/05 12:51:33,72.0.0.131,10.0.0.174,0.0.0.0,0.0.0.0,EX-Allow,,,ssl,vsys1,v_dmz-external,v_dmz-internal,ethernet1/3,ethernet1/4,LOG-Default,2015/01/05 12:51:33,34186774,1,28203,443,0,0,0x4053,tcp,allow,4121,2209,1912,20,2015/01/05 12:51:01,30,EX-Allowed,0,17754932086,0x0,US,10.0.0.0-10.255.255.255,0,10,10 
-<14>Jan  5 12:51:34 PAN1.exampleCustomer.com 1,2015/01/05 12:51:33,0011C103117,TRAFFIC,end,1,2015/01/05 12:51:33,216.0.10.244,10.0.0.53,0.0.0.0,0.0.0.0,EX-EasyAV,,,ssl,vsys1,v_dmz-external,v_dmz-internal,ethernet1/3,ethernet1/4,LOG-Default,2015/01/05 12:51:33,33988765,1,45150,443,0,0,0x401c,tcp,allow,626,358,268,9,2015/01/05 12:50:41,50,any,0,17754932095,0x0,US,10.0.0.0-10.255.255.255,0,5,4 
-<14>Jan  5 12:51:34 PAN1.exampleCustomer.com 1,2015/01/05 12:51:33,0011C103117,TRAFFIC,end,1,2015/01/05 12:51:33,172.12.216.82,10.0.0.53,0.0.0.0,0.0.0.0,EX-EasyAV,,,eset-update,vsys1,v_dmz-external,v_dmz-internal,ethernet1/3,ethernet1/4,LOG-Default,2015/01/05 12:51:33,33577240,1,3882,80,0,0,0x401c,tcp,allow,94947,2570,92377,106,2015/01/05 12:50:47,44,EX-Allowed,0,17754932107,0x0,US,10.0.0.0-10.255.255.255,0,38,68 
-<14>Jan  5 12:51:34 PAN1.exampleCustomer.com 1,2015/01/05 12:51:33,0011C103117,TRAFFIC,end,1,2015/01/05 12:51:33,10.0.0.33,10.1.0.85,0.0.0.0,0.0.0.0,EX-Allow,,,zabbix,vsys1,v_dmz-external,v_dmz-internal,ethernet1/3,ethernet1/4,LOG-Default,2015/01/05 12:51:33,34078885,1,46056,10050,0,0,0x405e,tcp,allow,728,367,361,11,2015/01/05 12:51:01,30,any,0,17754932117,0x0,10.0.0.0-10.255.255.255,10.0.0.0-10.255.255.255,0,6,5 
-<14>Jan  5 12:51:34 PAN1.exampleCustomer.com 1,2015/01/05 12:51:33,0011C103117,TRAFFIC,end,1,2015/01/05 12:51:33,10.0.0.11,10.1.0.33,0.0.0.0,0.0.0.0,EX-Allow,,,incomplete,vsys1,v_external,v_internal,ethernet1/2,ethernet1/1,LOG-Default,2015/01/05 12:51:33,34222137,1,59966,443,0,0,0x401c,tcp,allow,404,198,206,7,2015/01/05 12:51:01,30,any,0,17754932131,0x0,10.0.0.0-10.255.255.255,10.0.0.0-10.255.255.255,0,4,3 
-<14>Jan  5 12:51:34 PAN1.exampleCustomer.com 1,2015/01/05 12:51:33,0011C103117,TRAFFIC,end,1,2015/01/05 12:51:33,10.0.0.12,172.13.0.23,0.0.0.0,0.0.0.0,EX-Allow,,,dns,vsys1,v_dmz-internal,v_dmz-external,ethernet1/4,ethernet1/3,LOG-Default,2015/01/05 12:51:33,33560784,1,52991,53,0,0,0x4019,udp,allow,815,96,719,2,2015/01/05 12:51:01,30,any,0,17754932142,0x0,10.0.0.0-10.255.255.255,US,0,1,1 
-<14>Jan  5 12:51:34 PAN1.exampleCustomer.com 1,2015/01/05 12:51:33,0011C103117,TRAFFIC,end,1,2015/01/05 12:51:33,10.0.0.52,10.1.0.174,0.0.0.0,0.0.0.0,EX-EasyAV2,,,mssql-db,vsys1,v_dmz-internal,v_dmz-external,ethernet1/4,ethernet1/3,LOG-Default,2015/01/05 12:51:33,109384,1,50721,40004,0,0,0x401c,tcp,allow,4211,2125,2086,25,2015/01/05 12:51:02,30,any,0,17754932194,0x0,10.0.0.0-10.255.255.255,10.0.0.0-10.255.255.255,0,10,15 
-<14>Jan  5 12:51:34 PAN1.exampleCustomer.com 1,2015/01/05 12:51:33,0011C103117,TRAFFIC,end,1,2015/01/05 12:51:33,10.0.0.53,10.1.0.174,0.0.0.0,0.0.0.0,EX-EasyAV2,,,mssql-db,vsys1,v_external,v_internal,ethernet1/2,ethernet1/1,LOG-Default,2015/01/05 12:51:33,134519,1,54273,40004,0,0,0x401c,tcp,allow,5375,3289,2086,26,2015/01/05 12:51:02,30,any,0,17754932204,0x0,10.0.0.0-10.255.255.255,10.0.0.0-10.255.255.255,0,11,15 
-<14>Jan  5 12:51:34 PAN1.exampleCustomer.com 1,2015/01/05 12:51:33,0011C103117,TRAFFIC,end,1,2015/01/05 12:51:33,10.0.0.53,10.1.0.174,0.0.0.0,0.0.0.0,EX-EasyAV2,,,mssql-db,vsys1,v_external,v_internal,ethernet1/2,ethernet1/1,LOG-Default,2015/01/05 12:51:33,15005,1,54268,40004,0,0,0x401c,tcp,allow,7084,3787,3297,26,2015/01/05 12:51:02,30,any,0,17754932228,0x0,10.0.0.0-10.255.255.255,10.0.0.0-10.255.255.255,0,11,15 
-<14>Jan  5 12:51:34 PAN1 1,2015/01/05 12:51:33,0003C105690,TRAFFIC,drop,1,2015/01/05 12:51:33,10.0.0.219,10.3.0.21,0.0.0.0,0.0.0.0,catch all deny,,,not-applicable,vsys1,GuestAccess,trust,vlan.84,,LOG-Default,2015/01/05 12:51:33,0,1,62063,389,0,0,0x0,tcp,deny,70,70,0,1,2015/01/05 12:51:34,0,any,0,956329030,0x0,10.0.0.0-10.255.255.255,10.0.0.0-10.255.255.255,0,1,0 
-<14>Jan  5 12:51:34 PAN1.exampleCustomer.com 1,2015/01/05 12:51:33,0006C113555,TRAFFIC,end,1,2015/01/05 12:51:33,10.0.0.217,172.13.0.168,186.225.121.238,172.13.0.168,Guest WiFi to Internet,,,skype-probe,vsys1,Guest WiFi,Ext_Internet,ethernet1/3.109,ethernet1/2,LOG-Default,2015/01/05 12:51:33,46888,1,11566,40023,55962,40023,0x404050,udp,allow,1446,79,1367,2,2015/01/05 12:51:03,0,any,0,265102737,0x0,10.0.0.0-10.255.255.255,US,0,1,1 
-<14>Jan  5 12:51:34 PAN1.exampleCustomer.com 1,2015/01/05 12:51:33,0006C113555,TRAFFIC,end,1,2015/01/05 12:51:33,10.0.0.20,10.1.0.28,0.0.0.0,0.0.0.0,EX-Allow,,example\user.name,vmware,vsys1,Inside,Inside,ethernet1/4,tunnel.1,LOG-Default,2015/01/05 12:51:33,46821,1,61199,902,0,0,0x4019,udp,allow,108,108,0,1,2015/01/05 12:51:03,0,any,0,265102739,0x0,10.0.0.0-10.255.255.255,10.0.0.0-10.255.255.255,0,1,0 
-<14>Jan  5 12:51:34 PAN1 1,2015/01/05 12:51:33,0003C105690,TRAFFIC,end,1,2015/01/05 12:51:33,10.0.0.147,4.2.2.2,192.168.100.11,4.2.2.2,Guest_to_Internet,,,dns,vsys1,GuestAccess,untrust,vlan.84,vlan.200,LOG-Default,2015/01/05 12:51:33,188024,1,57269,53,59952,53,0x400019,udp,allow,194,73,121,2,2015/01/05 12:50:49,0,any,0,956329037,0x0,10.0.0.0-10.255.255.255,US,0,1,1 
-<14>Jan  5 12:51:34 PAN1.exampleCustomer.com 1,2015/01/05 12:51:33,0011C103117,TRAFFIC,end,1,2015/01/05 12:51:33,10.0.0.11,172.13.0.110,0.0.0.0,0.0.0.0,EX-Allow,,,dns,vsys1,v_internal,v_external,ethernet1/1,ethernet1/2,LOG-Default,2015/01/05 12:51:33,51569,1,60390,53,0,0,0x4019,udp,allow,815,96,719,2,2015/01/05 12:51:02,30,any,0,17754932369,0x0,10.0.0.0-10.255.255.255,US,0,1,1 
-<14>Jan  5 12:51:34 PAN1.exampleCustomer.com 1,2015/01/05 12:51:33,0011C103117,TRAFFIC,end,1,2015/01/05 12:51:33,10.0.0.11,10.1.0.81,0.0.0.0,0.0.0.0,EX-Allow,,,ping,vsys1,v_dmz-internal,v_dmz-external,ethernet1/4,ethernet1/3,LOG-Default,2015/01/05 12:51:33,185459,1,0,0,0,0,0x4019,icmp,allow,120,60,60,2,2015/01/05 12:51:29,0,any,0,17754932372,0x0,10.0.0.0-10.255.255.255,10.0.0.0-10.255.255.255,0,1,1 
-<14>Jan  5 12:51:34 PAN1.exampleCustomer.com 1,2015/01/05 12:51:33,0011C103117,TRAFFIC,end,1,2015/01/05 12:51:33,10.0.0.11,10.1.0.44,0.0.0.0,0.0.0.0,EX-Allow,,,ping,vsys1,v_external,v_internal,ethernet1/2,ethernet1/1,LOG-Default,2015/01/05 12:51:33,84730,1,0,0,0,0,0x4019,icmp,allow,120,60,60,2,2015/01/05 12:51:29,0,any,0,17754932379,0x0,10.0.0.0-10.255.255.255,10.0.0.0-10.255.255.255,0,1,1 
-<14>Jan  5 12:51:34 PAN1.exampleCustomer.com 1,2015/01/05 12:51:33,0006C110285,TRAFFIC,end,1,2015/01/05 12:51:33,10.0.0.73,10.1.0.12,0.0.0.0,0.0.0.0,EX-Allow,,,dns,vsys1,internal,external,ethernet1/2,ethernet1/1,LOG-Default,2015/01/05 12:51:33,12561,1,57334,53,0,0,0x4019,udp,allow,206,95,111,2,2015/01/05 12:51:03,0,any,0,803406326,0x0,10.0.0.0-10.255.255.255,10.0.0.0-10.255.255.255,0,1,1 
-<14>Jan  5 12:51:34 PAN1.exampleCustomer.com 1,2015/01/05 12:51:33,0006C110285,TRAFFIC,end,1,2015/01/05 12:51:33,10.0.0.35,10.3.0.65,0.0.0.0,0.0.0.0,EX-Allow,,,web-browsing,vsys1,external,internal,ethernet1/1,ethernet1/2,LOG-Default,2015/01/05 12:51:33,3286,1,57095,80,0,0,0x401c,tcp,allow,3506,899,2607,12,2015/01/05 12:51:03,0,private-ip-addresses,0,803406334,0x0,10.0.0.0-10.255.255.255,10.0.0.0-10.255.255.255,0,7,5 
-<14>Jan  5 12:51:34 PAN1.exampleCustomer.com 1,2015/01/05 12:51:33,0006C110285,TRAFFIC,end,1,2015/01/05 12:51:33,10.0.0.35,10.3.0.65,0.0.0.0,0.0.0.0,EX-Allow,,,web-browsing,vsys1,external,internal,ethernet1/1,ethernet1/2,LOG-Default,2015/01/05 12:51:33,42426,1,57096,80,0,0,0x401c,tcp,allow,3386,1390,1996,12,2015/01/05 12:51:03,0,private-ip-addresses,0,803406335,0x0,10.0.0.0-10.255.255.255,10.0.0.0-10.255.255.255,0,7,5 
-<14>Jan  5 12:51:34 PAN1.exampleCustomer.com 1,2015/01/05 12:51:33,0006C110285,TRAFFIC,end,1,2015/01/05 12:51:33,10.0.0.35,10.3.0.65,0.0.0.0,0.0.0.0,EX-Allow,,,web-browsing,vsys1,external,internal,ethernet1/1,ethernet1/2,LOG-Default,2015/01/05 12:51:33,15733,1,57130,80,0,0,0x401c,tcp,allow,1661,926,735,12,2015/01/05 12:51:03,0,private-ip-addresses,0,803406337,0x0,10.0.0.0-10.255.255.255,10.0.0.0-10.255.255.255,0,7,5 
-<14>Jan  5 12:51:34 PAN1.exampleCustomer.com 1,2015/01/05 12:51:33,0011C103117,TRAFFIC,end,1,2015/01/05 12:51:33,10.0.0.11,10.1.0.60,0.0.0.0,0.0.0.0,EX-Allow,,,ping,vsys1,v_external,v_internal,ethernet1/2,ethernet1/1,LOG-Default,2015/01/05 12:51:33,239420,1,0,0,0,0,0x4019,icmp,allow,120,60,60,2,2015/01/05 12:51:29,0,any,0,17754932383,0x0,10.0.0.0-10.255.255.255,10.0.0.0-10.255.255.255,0,1,1 
-<14>Jan  5 04:51:34 PAN1.exampleCustomer.com 1,2015/01/05 04:51:33,009401009421,TRAFFIC,end,1,2015/01/05 04:51:33,10.0.0.67,63.0.0.78,68.1.100.154,63.0.0.78,EX-Allow,,,web-browsing,vsys1,internal,external,ethernet1/1,ethernet1/2,LOG-Default,2015/01/05 04:51:33,13687,1,53152,80,64294,80,0x40001c,tcp,allow,1039,576,463,12,2015/01/05 04:51:03,1,search-engines,0,8195211,0x0,10.0.0.0-10.255.255.255,US,0,6,6 
-<14>Jan  5 04:51:34 PAN1.exampleCustomer.com 1,2015/01/05 04:51:33,009401009421,TRAFFIC,end,1,2015/01/05 04:51:33,10.0.0.67,77.0.0.59,68.1.100.154,77.0.0.59,EX-Allow,,,web-browsing,vsys1,internal,external,ethernet1/1,ethernet1/2,LOG-Default,2015/01/05 04:51:33,36193,1,53155,80,48756,80,0x40001c,tcp,allow,946,540,406,10,2015/01/05 04:51:04,0,computer-and-internet-security,0,8195212,0x0,10.0.0.0-10.255.255.255,CZ,0,5,5 
-<14>Jan  5 04:51:34 PAN1.exampleCustomer.com 1,2015/01/05 04:51:33,009401009421,TRAFFIC,end,1,2015/01/05 04:51:33,10.0.0.67,63.0.0.78,68.1.100.154,63.0.0.78,EX-Allow,,,web-browsing,vsys1,internal,external,ethernet1/1,ethernet1/2,LOG-Default,2015/01/05 04:51:33,8727,1,53154,80,6852,80,0x40001c,tcp,allow,1039,576,463,12,2015/01/05 04:51:04,0,search-engines,0,8195213,0x0,10.0.0.0-10.255.255.255,US,0,6,6 
-<14>Jan  5 04:51:34 PAN1.exampleCustomer.com 1,2015/01/05 04:51:33,009401009421,TRAFFIC,end,1,2015/01/05 04:51:33,10.0.0.67,77.0.0.59,68.1.100.154,77.0.0.59,EX-Allow,,,web-browsing,vsys1,internal,external,ethernet1/1,ethernet1/2,LOG-Default,2015/01/05 04:51:33,16955,1,53153,80,19440,80,0x40001c,tcp,allow,946,540,406,10,2015/01/05 04:51:03,1,computer-and-internet-security,0,8195216,0x0,10.0.0.0-10.255.255.255,CZ,0,5,5 
-<14>Jan  5 04:51:34 PAN1.exampleCustomer.com 1,2015/01/05 04:51:33,009401009421,TRAFFIC,end,1,2015/01/05 04:51:33,10.0.0.101,23.200,10,217,68.0.0.154,23.200,10,217,EX-WebControlRestrict,,,itunes-base,vsys1,internal,external,ethernet1/1,ethernet1/2,LOG-Default,2015/01/05 04:51:33,14851,1,55137,443,29553,443,0x400019,tcp,allow,654,580,74,7,2015/01/05 04:50:34,0,shopping,0,8195217,0x0,10.0.0.0-10.255.255.255,US,0,6,1 
-<14>Jan  5 12:51:34 PAN1.exampleCustomer.com 1,2015/01/05 12:51:34,0006C113555,TRAFFIC,end,1,2015/01/05 12:51:34,10.0.0.46,172.13.0.2,0.0.0.0,0.0.0.0,EX-Allow,example\user.name,,incomplete,vsys1,Inside,Inside,ethernet1/4,tunnel.1,LOG-Default,2015/01/05 12:51:34,57850,1,65286,139,0,0,0x4019,tcp,allow,62,62,0,1,2015/01/05 12:51:29,0,any,0,265102746,0x0,10.0.0.0-10.255.255.255,192.168.0.0-192.168.255.255,0,1,0 
-<14>Jan  5 12:51:34 PAN1 1,2015/01/05 12:51:34,0003C105690,TRAFFIC,end,1,2015/01/05 12:51:34,216.0.10.194,192.168.100.11,0.0.0.0,0.0.0.0,Internet to Internet,,,insufficient-data,vsys1,untrust,untrust,vlan.200,vlan.200,LOG-Default,2015/01/05 12:51:34,259007,1,80,11347,0,0,0xc,udp,allow,90,90,0,1,2015/01/05 12:50:25,0,any,0,956329050,0x0,US,US,0,1,0 
-<14>Jan  5 12:51:34 PAN1 1,2015/01/05 12:51:34,0003C105690,TRAFFIC,end,1,2015/01/05 12:51:34,10.0.0.147,4.2.2.2,192.168.100.11,4.2.2.2,Guest_to_Internet,,,dns,vsys1,GuestAccess,untrust,vlan.84,vlan.200,LOG-Default,2015/01/05 12:51:34,13024,1,56694,53,51398,53,0x400019,udp,allow,222,82,140,2,2015/01/05 12:50:49,0,any,0,956329055,0x0,10.0.0.0-10.255.255.255,US,0,1,1 
-<14>Jan  5 12:51:34 PAN1 1,2015/01/05 12:51:34,0003C105690,TRAFFIC,end,1,2015/01/05 12:51:34,10.0.0.147,4.2.2.2,192.168.100.11,4.2.2.2,Guest_to_Internet,,,dns,vsys1,GuestAccess,untrust,vlan.84,vlan.200,LOG-Default,2015/01/05 12:51:34,62999,1,58277,53,5576,53,0x400019,udp,allow,328,96,232,2,2015/01/05 12:50:49,0,any,0,956329056,0x0,10.0.0.0-10.255.255.255,US,0,1,1 
-<14>Jan  5 12:51:34 PAN1.exampleCustomer.com 1,2015/01/05 12:51:34,001606007155,TRAFFIC,end,1,2015/01/05 12:51:34,10.0.0.156,96.0.0.138,172.13.0.35,96.0.0.138,EX-Allow,example\user.name,,web-browsing,vsys1,trust,untrust,ethernet1/2,ethernet1/1,LOG-Default,2015/01/05 12:51:34,61348,1,65231,80,48623,80,0x40401a,tcp,allow,50316,4297,46019,67,2015/01/05 12:51:03,1,travel,0,179851307,0x0,10.0.0.0-10.255.255.255,US,0,28,39 
-<14>Jan  5 12:51:34 PAN1.exampleCustomer.com 1,2015/01/05 12:51:34,001606007155,TRAFFIC,end,1,2015/01/05 12:51:34,10.0.0.148,96.0.0.35,172.13.0.35,96.0.0.35,EX-Allow,example\user.name,,symantec-av-update,vsys1,trust,untrust,ethernet1/2,ethernet1/1,LOG-Default,2015/01/05 12:51:34,61220,1,60900,80,12964,80,0x40401a,tcp,allow,39350,3087,36263,56,2015/01/05 12:50:07,57,computer-and-internet-security,0,179851311,0x0,10.0.0.0-10.255.255.255,US,0,23,33 
-<14>Jan  5 12:51:34 PAN1 1,2015/01/05 12:51:34,009401003136,TRAFFIC,end,1,2015/01/05 12:51:34,10.0.0.138,213.0.10.101,172.13.0.142,213.0.10.101,Outbound,,,ssl,vsys1,internal,external,ethernet1/4,ethernet1/1,LOG-Default,2015/01/05 12:51:34,62600,1,55014,443,22537,443,0x40001c,tcp,allow,2956,1853,1103,20,2015/01/05 12:51:04,0,travel,0,54644537,0x0,10.0.0.0-10.255.255.255,CH,0,9,11 
-<14>Jan  5 12:51:34 PAN1 1,2015/01/05 12:51:34,009401003136,TRAFFIC,end,1,2015/01/05 12:51:34,10.0.0.138,213.0.10.101,172.13.0.142,213.0.10.101,Outbound,,,ssl,vsys1,internal,external,ethernet1/4,ethernet1/1,LOG-Default,2015/01/05 12:51:34,45328,1,55025,443,48646,443,0x40001c,tcp,allow,2828,1845,983,18,2015/01/05 12:51:04,0,travel,0,54644544,0x0,10.0.0.0-10.255.255.255,CH,0,9,9 
-<14>Jan  5 12:51:34 PAN1.exampleCustomer.com 1,2015/01/05 12:51:34,0004C103634,TRAFFIC,end,1,2015/01/05 12:51:34,10.0.0.165,93.0.0.200,0.0.0.0,0.0.0.0,EX-Allow,example\user.name,,ssl,vsys1,v_internal,v_external,ethernet1/2,ethernet1/1,LOG-Default,2015/01/05 12:51:34,15787,1,53105,443,0,0,0x4053,tcp,allow,10222,1275,8947,22,2015/01/05 12:48:03,181,business-and-economy,0,307579464,0x0,10.0.0.0-10.255.255.255,EU,0,10,12 
-<14>Jan  5 12:51:35 PAN1 1,2015/01/05 12:51:34,0003C105690,TRAFFIC,end,1,2015/01/05 12:51:34,10.0.0.11,10.3.0.26,0.0.0.0,0.0.0.0,ICMP DMZ to In,,,ping,vsys1,F5_DMZ_WAN,trust,vlan.81,vlan.399,LOG-Default,2015/01/05 12:51:34,33876,1,0,0,0,0,0x19,icmp,allow,128,64,64,2,2015/01/05 12:51:20,0,any,0,956329058,0x0,10.0.0.0-10.255.255.255,10.0.0.0-10.255.255.255,0,1,1 
-<14>Jan  5 12:51:35 PAN1.exampleCustomer.com 1,2015/01/05 12:51:34,0006C113555,TRAFFIC,end,1,2015/01/05 12:51:34,10.0.0.53,8.8.8.8,172.13.0.238,8.8.8.8,Guest WiFi to Internet,,,dns,vsys1,Guest WiFi,Ext_Internet,ethernet1/3.109,ethernet1/2,LOG-Default,2015/01/05 12:51:34,53079,1,59288,53,31746,53,0x404019,udp,allow,194,91,103,2,2015/01/05 12:51:04,0,any,0,265102750,0x0,10.0.0.0-10.255.255.255,US,0,1,1 
-<14>Jan  5 12:51:35 PAN1.exampleCustomer.com 1,2015/01/05 12:51:34,0011C103117,TRAFFIC,end,1,2015/01/05 12:51:34,10.0.0.53,10.1.0.174,0.0.0.0,0.0.0.0,EX-EasyAV2,,,mssql-db,vsys1,v_external,v_internal,ethernet1/2,ethernet1/1,LOG-Default,2015/01/05 12:51:34,141372,1,54279,40004,0,0,0x401c,tcp,allow,3783,1697,2086,25,2015/01/05 12:51:03,30,any,0,17754932394,0x0,10.0.0.0-10.255.255.255,10.0.0.0-10.255.255.255,0,10,15 

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/df8d682e/metron-platform/metron-integration-test/src/main/resources/sample/data/SampleInput/SnortOutput
----------------------------------------------------------------------
diff --git a/metron-platform/metron-integration-test/src/main/resources/sample/data/SampleInput/SnortOutput b/metron-platform/metron-integration-test/src/main/resources/sample/data/SampleInput/SnortOutput
deleted file mode 100644
index 0497b0f..0000000
--- a/metron-platform/metron-integration-test/src/main/resources/sample/data/SampleInput/SnortOutput
+++ /dev/null
@@ -1,3 +0,0 @@
-01/27-16:01:04.877970 ,129,12,1,"Consecutive TCP small segments exceeding threshold",TCP,10.0.2.2,56642,10.0.2.15,22,52:54:00:12:35:02,08:00:27:7F:93:2D,0x4E,***AP***,0x9AFF3D7,0xC8761D52,,0xFFFF,64,0,59677,64,65536,,,,
-02/22-15:56:48.612494 ,129,12,1,"Consecutive TCP small segments exceeding threshold",TCP,96.44.142.5,80,10.0.2.15,50895,52:54:00:12:35:02,08:00:27:7F:93:2D,0x6E,***AP***,0xDB45F7A,0x7701DD5B,,0xFFFF,64,0,16785,96,98304,,,,
-02/22-15:56:48.616775 ,129,12,1,"Consecutive TCP small segments exceeding threshold",TCP,96.44.142.5,80,10.0.2.15,50895,52:54:00:12:35:02,08:00:27:7F:93:2D,0xA6,***AP***,0xDB508F2,0x7701DD5B,,0xFFFF,64,0,16824,152,155648,,,,
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/df8d682e/metron-platform/metron-integration-test/src/main/resources/sample/data/SampleInput/SourcefireExampleOutput
----------------------------------------------------------------------
diff --git a/metron-platform/metron-integration-test/src/main/resources/sample/data/SampleInput/SourcefireExampleOutput b/metron-platform/metron-integration-test/src/main/resources/sample/data/SampleInput/SourcefireExampleOutput
deleted file mode 100644
index 5f177df..0000000
--- a/metron-platform/metron-integration-test/src/main/resources/sample/data/SampleInput/SourcefireExampleOutput
+++ /dev/null
@@ -1,2 +0,0 @@
-SFIMS: [Primary Detection Engine (a7213248-6423-11e3-8537-fac6a92b7d9d)][MTD Access Control] Connection Type: Start, User: Unknown, Client: Unknown, Application Protocol: Unknown, Web App: Unknown, Firewall Rule Name: MTD Access Control, Firewall Rule Action: Allow, Firewall Rule Reasons: Unknown, URL Category: Unknown, URL_Reputation: Risk unknown, URL: Unknown, Interface Ingress: s1p1, Interface Egress: N/A, Security Zone Ingress: Unknown, Security Zone Egress: N/A, Security Intelligence Matching IP: None, Security Intelligence Category: None, {TCP} 72.163.0.129:60517 -> 10.1.128.236:443
-SFIMS: [Primary Detection Engine (a7213248-6423-11e3-8537-fac6a92b7d9d)][MTD Access Control] Connection Type: Start, User: Unknown, Client: Unknown, Application Protocol: Unknown, Web App: Unknown, Firewall Rule Name: MTD Access Control, Firewall Rule Action: Allow, Firewall Rule Reasons: Unknown, URL Category: Unknown, URL_Reputation: Risk unknown, URL: Unknown, Interface Ingress: s1p1, Interface Egress: N/A, Security Zone Ingress: Unknown, Security Zone Egress: N/A, Security Intelligence Matching IP: None, Security Intelligence Category: None, {TCP} 10.5.200.245:45283 -> 72.163.0.129:21
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/df8d682e/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
deleted file mode 100644
index ae70fb9..0000000
--- a/metron-platform/metron-integration-test/src/main/resources/sample/data/SampleInput/SquidExampleOutput
+++ /dev/null
@@ -1,2 +0,0 @@
-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

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/df8d682e/metron-platform/metron-integration-test/src/main/resources/sample/data/SampleInput/WebsphereOutput.txt
----------------------------------------------------------------------
diff --git a/metron-platform/metron-integration-test/src/main/resources/sample/data/SampleInput/WebsphereOutput.txt b/metron-platform/metron-integration-test/src/main/resources/sample/data/SampleInput/WebsphereOutput.txt
deleted file mode 100644
index 9003548..0000000
--- a/metron-platform/metron-integration-test/src/main/resources/sample/data/SampleInput/WebsphereOutput.txt
+++ /dev/null
@@ -1,5 +0,0 @@
-<133>Apr 15 17:47:28 ABCXML1413 [rojOut][0x81000033][auth][notice] user(rick007): [120.43.200.6]: User logged into 'cohlOut'.
-<134>Apr 15 18:02:27 PHIXML3RWD [0x81000019][auth][info] [14.122.2.201]: User 'hjpotter' logged out from 'default'.
-<131>Apr 15 17:36:35 ROBXML3QRS [0x80800018][auth][error] rbm(RBM-Settings): trans(3502888135)[request] gtid(3502888135): RBM: Resource access denied.
-<134>Apr 15 17:17:34 SAGPXMLQA333 [0x8240001c][audit][info] trans(191): (admin:default:system:*): ntp-service 'NTP Service' - Operational state down
-<134>Apr 15 17:46:52 DOMXML3PUZ [0x8100448e][auth][info] CLI timeout occurred.
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/df8d682e/metron-platform/metron-integration-test/src/main/resources/sample/data/SampleInput/YafExampleOutput
----------------------------------------------------------------------
diff --git a/metron-platform/metron-integration-test/src/main/resources/sample/data/SampleInput/YafExampleOutput b/metron-platform/metron-integration-test/src/main/resources/sample/data/SampleInput/YafExampleOutput
deleted file mode 100644
index 8f3ff44..0000000
--- a/metron-platform/metron-integration-test/src/main/resources/sample/data/SampleInput/YafExampleOutput
+++ /dev/null
@@ -1,10 +0,0 @@
-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
-2016-01-28 15:29:48.502|2016-01-28 15:29:48.502|   0.000|   0.000| 17|                               10.0.2.15|37299|                                10.0.2.3|   53|       A|       0|       0|       0|10000000|00000000|000|000|       1|      56|       0|       0|    0|idle
-2016-01-28 15:29:48.504|2016-01-28 15:29:48.504|   0.000|   0.000| 17|                                10.0.2.3|   53|                               10.0.2.15|37299|       A|       0|       0|       0|00000000|00000000|000|000|       1|     312|       0|       0|    0|idle
-2016-01-28 15:29:48.504|2016-01-28 15:29:48.504|   0.000|   0.000| 17|                               10.0.2.15|56303|                                10.0.2.3|   53|       A|       0|       0|       0|00000000|00000000|000|000|       1|      56|       0|       0|    0|idle
-2016-01-28 15:29:48.506|2016-01-28 15:29:48.506|   0.000|   0.000| 17|                                10.0.2.3|   53|                               10.0.2.15|56303|       A|       0|       0|       0|00000000|00000000|000|000|       1|      84|       0|       0|    0|idle
-2016-01-28 15:29:48.508|2016-01-28 15:29:48.508|   0.000|   0.000|  6|                               10.0.2.15|39468|                          216.21.170.221|   80|       S|       0|       0|       0|58c52fca|00000000|000|000|       1|      60|       0|       0|    0|idle
-2016-01-28 15:29:48.512|2016-01-28 15:29:48.512|   0.000|   0.000|  6|                               10.0.2.15|39468|                          216.21.170.221|   80|       A|       0|       0|       0|58c52fcb|00000000|000|000|       1|      40|       0|       0|    0|idle 
-2016-01-28 15:29:48.512|2016-01-28 15:29:48.512|   0.000|   0.000|  6|                               10.0.2.15|39468|                          216.21.170.221|   80|      AP|       0|       0|       0|58c52fcb|00000000|000|000|       1|     148|       0|       0|    0|idle 
-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|       A|       0|       0|       0|22efa002|00000000|000|000|       1|      40|       0|       0|    0|idle 
-2016-01-28 15:29:48.562|2016-01-28 15:29:48.562|   0.000|   0.000|  6|                          216.21.170.221|   80|                               10.0.2.15|39468|      AP|       0|       0|       0|22efa002|00000000|000|000|       1|     604|       0|       0|    0|idle
\ No newline at end of file


[04/15] incubator-metron git commit: METRON-142 Simplify Parser configuration (merrimanr via cestella) closes apache/incubator-metron#120

Posted by ce...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/df8d682e/metron-platform/metron-integration-test/src/main/sample/data/bluecoat/parsed/BluecoatParsed
----------------------------------------------------------------------
diff --git a/metron-platform/metron-integration-test/src/main/sample/data/bluecoat/parsed/BluecoatParsed b/metron-platform/metron-integration-test/src/main/sample/data/bluecoat/parsed/BluecoatParsed
new file mode 100644
index 0000000..201c972
--- /dev/null
+++ b/metron-platform/metron-integration-test/src/main/sample/data/bluecoat/parsed/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


[02/15] incubator-metron git commit: METRON-142 Simplify Parser configuration (merrimanr via cestella) closes apache/incubator-metron#120

Posted by ce...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/df8d682e/metron-platform/metron-integration-test/src/main/sample/data/yaf/indexed/YafIndexed
----------------------------------------------------------------------
diff --git a/metron-platform/metron-integration-test/src/main/sample/data/yaf/indexed/YafIndexed b/metron-platform/metron-integration-test/src/main/sample/data/yaf/indexed/YafIndexed
new file mode 100644
index 0000000..1c38406
--- /dev/null
+++ b/metron-platform/metron-integration-test/src/main/sample/data/yaf/indexed/YafIndexed
@@ -0,0 +1,10 @@
+{"adapter.threatinteladapter.end.ts":"1457102731219","enrichments.geo.dip.location_point":"test longitude,test latitude","isn":"22efa001","index.elasticsearchwriter.ts":"1457102731220","dip":"10.0.2.15","dp":39468,"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","enrichments.geo.dip.locID":"1","enrichments.geo.sip.city":"test city","enrichmentjoinbolt.joiner.ts":"1457102731206","adapter.hostfromjsonlistadapter.begin.ts":"1457102731185","tag":0,"enrichments.geo.dip.dmaCode":"test dmaCode","app":0,"oct":44,"end_reason":"idle","enrichments.geo.sip.locID":"1","adapter.mockgeoadapter.begin.ts":"1457102731185","threatintelsplitterbolt.splitter.ts":"1457102731207","enrichments.geo.dip.postalCode":"test postalCode","start_time":1453994988512,
 "adapter.threatinteladapter.begin.ts":"1457102731210","riflags":0,"proto":6,"enrichments.host.dip.known_info.local":"YES","enrichments.geo.dip.longitude":"test longitude","iflags":"AS","uflags":0,"adapter.mockgeoadapter.end.ts":"1457102731198","adapter.hostfromjsonlistadapter.end.ts":"1457102731197","enrichments.geo.sip.postalCode":"test postalCode","duration":"0.000","enrichments.geo.dip.country":"test country","threatinteljoinbolt.joiner.ts":"1457102731220","enrichments.geo.dip.latitude":"test latitude","enrichments.geo.sip.country":"test country","enrichments.geo.dip.city":"test city","enrichments.geo.sip.dmaCode":"test dmaCode","pkt":1,"enrichments.geo.sip.location_point":"test longitude,test latitude","ruflags":0,"roct":0,"sip":"216.21.170.221","rtag":0,"sp":80,"enrichments.geo.sip.longitude":"test longitude","enrichments.geo.sip.latitude":"test latitude","timestamp":1453994988512,"risn":0,"enrichments.host.dip.known_info.type":"printer","end_time":1453994988512,"enrichments.ho
 st.dip.known_info.asset_value":"important","source.type":"yaf","rtt":"0.000"}
+{"adapter.threatinteladapter.end.ts":"1457102731221","enrichments.geo.dip.location_point":"test longitude,test latitude","enrichments.host.sip.known_info.asset_value":"important","isn":10000000,"index.elasticsearchwriter.ts":"1457102731221","dip":"10.0.2.3","dp":53,"rpkt":0,"original_string":"2016-01-28 15:29:48.502|2016-01-28 15:29:48.502|   0.000|   0.000| 17|                               10.0.2.15|37299|                                10.0.2.3|   53|       A|       0|       0|       0|10000000|00000000|000|000|       1|      56|       0|       0|    0|idle","enrichments.geo.dip.locID":"1","enrichments.geo.sip.city":"test city","enrichments.host.sip.known_info.type":"printer","enrichmentjoinbolt.joiner.ts":"1457102731208","adapter.hostfromjsonlistadapter.begin.ts":"1457102731197","tag":0,"enrichments.geo.dip.dmaCode":"test dmaCode","app":0,"oct":56,"end_reason":"idle","enrichments.geo.sip.locID":"1","adapter.mockgeoadapter.begin.ts":"1457102731198","threatintelsplitterbolt.splitt
 er.ts":"1457102731210","enrichments.geo.dip.postalCode":"test postalCode","start_time":1453994988502,"adapter.threatinteladapter.begin.ts":"1457102731219","riflags":0,"proto":17,"enrichments.geo.dip.longitude":"test longitude","iflags":"A","uflags":0,"adapter.mockgeoadapter.end.ts":"1457102731198","adapter.hostfromjsonlistadapter.end.ts":"1457102731197","enrichments.host.sip.known_info.local":"YES","threatintels.ip.dip.ip_threat_intel":"alert","enrichments.geo.sip.postalCode":"test postalCode","duration":"0.000","enrichments.geo.dip.country":"test country","threatinteljoinbolt.joiner.ts":"1457102731221","enrichments.geo.dip.latitude":"test latitude","enrichments.geo.sip.country":"test country","enrichments.geo.dip.city":"test city","enrichments.geo.sip.dmaCode":"test dmaCode","pkt":1,"enrichments.geo.sip.location_point":"test longitude,test latitude","ruflags":0,"roct":0,"sip":"10.0.2.15","rtag":0,"sp":37299,"enrichments.geo.sip.longitude":"test longitude","enrichments.geo.sip.latit
 ude":"test latitude","timestamp":1453994988502,"risn":0,"end_time":1453994988502,"is_alert":"true","source.type":"yaf","rtt":"0.000"}
+{"adapter.threatinteladapter.end.ts":"1457102731221","enrichments.geo.dip.location_point":"test longitude,test latitude","isn":0,"index.elasticsearchwriter.ts":"1457102731222","dip":"10.0.2.15","dp":37299,"rpkt":0,"original_string":"2016-01-28 15:29:48.504|2016-01-28 15:29:48.504|   0.000|   0.000| 17|                                10.0.2.3|   53|                               10.0.2.15|37299|       A|       0|       0|       0|00000000|00000000|000|000|       1|     312|       0|       0|    0|idle","enrichments.geo.dip.locID":"1","enrichments.geo.sip.city":"test city","enrichmentjoinbolt.joiner.ts":"1457102731209","adapter.hostfromjsonlistadapter.begin.ts":"1457102731197","tag":0,"enrichments.geo.dip.dmaCode":"test dmaCode","app":0,"oct":312,"end_reason":"idle","enrichments.geo.sip.locID":"1","adapter.mockgeoadapter.begin.ts":"1457102731198","threatintelsplitterbolt.splitter.ts":"1457102731210","enrichments.geo.dip.postalCode":"test postalCode","start_time":1453994988504,"adapter
 .threatinteladapter.begin.ts":"1457102731221","riflags":0,"proto":17,"enrichments.host.dip.known_info.local":"YES","enrichments.geo.dip.longitude":"test longitude","iflags":"A","uflags":0,"adapter.mockgeoadapter.end.ts":"1457102731199","adapter.hostfromjsonlistadapter.end.ts":"1457102731198","enrichments.geo.sip.postalCode":"test postalCode","duration":"0.000","enrichments.geo.dip.country":"test country","threatinteljoinbolt.joiner.ts":"1457102731222","enrichments.geo.dip.latitude":"test latitude","enrichments.geo.sip.country":"test country","enrichments.geo.dip.city":"test city","enrichments.geo.sip.dmaCode":"test dmaCode","pkt":1,"enrichments.geo.sip.location_point":"test longitude,test latitude","ruflags":0,"roct":0,"sip":"10.0.2.3","rtag":0,"sp":53,"enrichments.geo.sip.longitude":"test longitude","enrichments.geo.sip.latitude":"test latitude","timestamp":1453994988504,"risn":0,"enrichments.host.dip.known_info.type":"printer","end_time":1453994988504,"enrichments.host.dip.known_i
 nfo.asset_value":"important","is_alert":"true","source.type":"yaf","threatintels.ip.sip.ip_threat_intel":"alert","rtt":"0.000"}
+{"adapter.threatinteladapter.end.ts":"1457102731222","enrichments.geo.dip.location_point":"test longitude,test latitude","enrichments.host.sip.known_info.asset_value":"important","isn":0,"index.elasticsearchwriter.ts":"1457102731222","dip":"10.0.2.3","dp":53,"rpkt":0,"original_string":"2016-01-28 15:29:48.504|2016-01-28 15:29:48.504|   0.000|   0.000| 17|                               10.0.2.15|56303|                                10.0.2.3|   53|       A|       0|       0|       0|00000000|00000000|000|000|       1|      56|       0|       0|    0|idle","enrichments.geo.dip.locID":"1","enrichments.geo.sip.city":"test city","enrichments.host.sip.known_info.type":"printer","enrichmentjoinbolt.joiner.ts":"1457102731209","adapter.hostfromjsonlistadapter.begin.ts":"1457102731198","tag":0,"enrichments.geo.dip.dmaCode":"test dmaCode","app":0,"oct":56,"end_reason":"idle","enrichments.geo.sip.locID":"1","adapter.mockgeoadapter.begin.ts":"1457102731199","threatintelsplitterbolt.splitter.ts":
 "1457102731211","enrichments.geo.dip.postalCode":"test postalCode","start_time":1453994988504,"adapter.threatinteladapter.begin.ts":"1457102731221","riflags":0,"proto":17,"enrichments.geo.dip.longitude":"test longitude","iflags":"A","uflags":0,"adapter.mockgeoadapter.end.ts":"1457102731199","adapter.hostfromjsonlistadapter.end.ts":"1457102731198","enrichments.host.sip.known_info.local":"YES","threatintels.ip.dip.ip_threat_intel":"alert","enrichments.geo.sip.postalCode":"test postalCode","duration":"0.000","enrichments.geo.dip.country":"test country","threatinteljoinbolt.joiner.ts":"1457102731222","enrichments.geo.dip.latitude":"test latitude","enrichments.geo.sip.country":"test country","enrichments.geo.dip.city":"test city","enrichments.geo.sip.dmaCode":"test dmaCode","pkt":1,"enrichments.geo.sip.location_point":"test longitude,test latitude","ruflags":0,"roct":0,"sip":"10.0.2.15","rtag":0,"sp":56303,"enrichments.geo.sip.longitude":"test longitude","enrichments.geo.sip.latitude":"t
 est latitude","timestamp":1453994988504,"risn":0,"end_time":1453994988504,"is_alert":"true","source.type":"yaf","rtt":"0.000"}
+{"adapter.threatinteladapter.end.ts":"1457102731222","enrichments.geo.dip.location_point":"test longitude,test latitude","isn":0,"index.elasticsearchwriter.ts":"1457102731222","dip":"10.0.2.15","dp":56303,"rpkt":0,"original_string":"2016-01-28 15:29:48.506|2016-01-28 15:29:48.506|   0.000|   0.000| 17|                                10.0.2.3|   53|                               10.0.2.15|56303|       A|       0|       0|       0|00000000|00000000|000|000|       1|      84|       0|       0|    0|idle","enrichments.geo.dip.locID":"1","enrichments.geo.sip.city":"test city","enrichmentjoinbolt.joiner.ts":"1457102731210","adapter.hostfromjsonlistadapter.begin.ts":"1457102731198","tag":0,"enrichments.geo.dip.dmaCode":"test dmaCode","app":0,"oct":84,"end_reason":"idle","enrichments.geo.sip.locID":"1","adapter.mockgeoadapter.begin.ts":"1457102731199","threatintelsplitterbolt.splitter.ts":"1457102731212","enrichments.geo.dip.postalCode":"test postalCode","start_time":1453994988506,"adapter.
 threatinteladapter.begin.ts":"1457102731222","riflags":0,"proto":17,"enrichments.host.dip.known_info.local":"YES","enrichments.geo.dip.longitude":"test longitude","iflags":"A","uflags":0,"adapter.mockgeoadapter.end.ts":"1457102731199","adapter.hostfromjsonlistadapter.end.ts":"1457102731198","enrichments.geo.sip.postalCode":"test postalCode","duration":"0.000","enrichments.geo.dip.country":"test country","threatinteljoinbolt.joiner.ts":"1457102731222","enrichments.geo.dip.latitude":"test latitude","enrichments.geo.sip.country":"test country","enrichments.geo.dip.city":"test city","enrichments.geo.sip.dmaCode":"test dmaCode","pkt":1,"enrichments.geo.sip.location_point":"test longitude,test latitude","ruflags":0,"roct":0,"sip":"10.0.2.3","rtag":0,"sp":53,"enrichments.geo.sip.longitude":"test longitude","enrichments.geo.sip.latitude":"test latitude","timestamp":1453994988506,"risn":0,"enrichments.host.dip.known_info.type":"printer","end_time":1453994988506,"enrichments.host.dip.known_in
 fo.asset_value":"important","is_alert":"true","source.type":"yaf","threatintels.ip.sip.ip_threat_intel":"alert","rtt":"0.000"}
+{"adapter.threatinteladapter.end.ts":"1457102731222","enrichments.geo.dip.location_point":"test longitude,test latitude","enrichments.host.sip.known_info.asset_value":"important","isn":"58c52fca","index.elasticsearchwriter.ts":"1457102732038","dip":"216.21.170.221","dp":80,"rpkt":0,"original_string":"2016-01-28 15:29:48.508|2016-01-28 15:29:48.508|   0.000|   0.000|  6|                               10.0.2.15|39468|                          216.21.170.221|   80|       S|       0|       0|       0|58c52fca|00000000|000|000|       1|      60|       0|       0|    0|idle","enrichments.geo.dip.locID":"1","enrichments.geo.sip.city":"test city","enrichments.host.sip.known_info.type":"printer","enrichmentjoinbolt.joiner.ts":"1457102731210","adapter.hostfromjsonlistadapter.begin.ts":"1457102731198","tag":0,"enrichments.geo.dip.dmaCode":"test dmaCode","app":0,"oct":60,"end_reason":"idle","enrichments.geo.sip.locID":"1","adapter.mockgeoadapter.begin.ts":"1457102731199","threatintelsplitterbol
 t.splitter.ts":"1457102731212","enrichments.geo.dip.postalCode":"test postalCode","start_time":1453994988508,"adapter.threatinteladapter.begin.ts":"1457102731222","riflags":0,"proto":6,"enrichments.geo.dip.longitude":"test longitude","iflags":"S","uflags":0,"adapter.mockgeoadapter.end.ts":"1457102731199","adapter.hostfromjsonlistadapter.end.ts":"1457102731198","enrichments.host.sip.known_info.local":"YES","enrichments.geo.sip.postalCode":"test postalCode","duration":"0.000","enrichments.geo.dip.country":"test country","threatinteljoinbolt.joiner.ts":"1457102731223","enrichments.geo.dip.latitude":"test latitude","enrichments.geo.sip.country":"test country","enrichments.geo.dip.city":"test city","enrichments.geo.sip.dmaCode":"test dmaCode","pkt":1,"enrichments.geo.sip.location_point":"test longitude,test latitude","ruflags":0,"roct":0,"sip":"10.0.2.15","rtag":0,"sp":39468,"enrichments.geo.sip.longitude":"test longitude","enrichments.geo.sip.latitude":"test latitude","timestamp":145399
 4988508,"risn":0,"end_time":1453994988508,"source.type":"yaf","rtt":"0.000"}
+{"adapter.threatinteladapter.end.ts":"1457102731223","enrichments.geo.dip.location_point":"test longitude,test latitude","enrichments.host.sip.known_info.asset_value":"important","isn":"58c52fcb","index.elasticsearchwriter.ts":"1457102732038","dip":"216.21.170.221","dp":80,"rpkt":0,"original_string":"2016-01-28 15:29:48.512|2016-01-28 15:29:48.512|   0.000|   0.000|  6|                               10.0.2.15|39468|                          216.21.170.221|   80|       A|       0|       0|       0|58c52fcb|00000000|000|000|       1|      40|       0|       0|    0|idle ","enrichments.geo.dip.locID":"1","enrichments.geo.sip.city":"test city","enrichments.host.sip.known_info.type":"printer","enrichmentjoinbolt.joiner.ts":"1457102731210","adapter.hostfromjsonlistadapter.begin.ts":"1457102731198","tag":0,"enrichments.geo.dip.dmaCode":"test dmaCode","app":0,"oct":40,"end_reason":"idle ","enrichments.geo.sip.locID":"1","adapter.mockgeoadapter.begin.ts":"1457102731199","threatintelsplitterb
 olt.splitter.ts":"1457102731212","enrichments.geo.dip.postalCode":"test postalCode","start_time":1453994988512,"adapter.threatinteladapter.begin.ts":"1457102731223","riflags":0,"proto":6,"enrichments.geo.dip.longitude":"test longitude","iflags":"A","uflags":0,"adapter.mockgeoadapter.end.ts":"1457102731199","adapter.hostfromjsonlistadapter.end.ts":"1457102731198","enrichments.host.sip.known_info.local":"YES","enrichments.geo.sip.postalCode":"test postalCode","duration":"0.000","enrichments.geo.dip.country":"test country","threatinteljoinbolt.joiner.ts":"1457102731223","enrichments.geo.dip.latitude":"test latitude","enrichments.geo.sip.country":"test country","enrichments.geo.dip.city":"test city","enrichments.geo.sip.dmaCode":"test dmaCode","pkt":1,"enrichments.geo.sip.location_point":"test longitude,test latitude","ruflags":0,"roct":0,"sip":"10.0.2.15","rtag":0,"sp":39468,"enrichments.geo.sip.longitude":"test longitude","enrichments.geo.sip.latitude":"test latitude","timestamp":1453
 994988512,"risn":0,"end_time":1453994988512,"source.type":"yaf","rtt":"0.000"}
+{"adapter.threatinteladapter.end.ts":"1457102731223","enrichments.geo.dip.location_point":"test longitude,test latitude","enrichments.host.sip.known_info.asset_value":"important","isn":"58c52fcb","index.elasticsearchwriter.ts":"1457102732038","dip":"216.21.170.221","dp":80,"rpkt":0,"original_string":"2016-01-28 15:29:48.512|2016-01-28 15:29:48.512|   0.000|   0.000|  6|                               10.0.2.15|39468|                          216.21.170.221|   80|      AP|       0|       0|       0|58c52fcb|00000000|000|000|       1|     148|       0|       0|    0|idle ","enrichments.geo.dip.locID":"1","enrichments.geo.sip.city":"test city","enrichments.host.sip.known_info.type":"printer","enrichmentjoinbolt.joiner.ts":"1457102731210","adapter.hostfromjsonlistadapter.begin.ts":"1457102731198","tag":0,"enrichments.geo.dip.dmaCode":"test dmaCode","app":0,"oct":148,"end_reason":"idle ","enrichments.geo.sip.locID":"1","adapter.mockgeoadapter.begin.ts":"1457102731199","threatintelsplitter
 bolt.splitter.ts":"1457102731212","enrichments.geo.dip.postalCode":"test postalCode","start_time":1453994988512,"adapter.threatinteladapter.begin.ts":"1457102731223","riflags":0,"proto":6,"enrichments.geo.dip.longitude":"test longitude","iflags":"AP","uflags":0,"adapter.mockgeoadapter.end.ts":"1457102731199","adapter.hostfromjsonlistadapter.end.ts":"1457102731198","enrichments.host.sip.known_info.local":"YES","enrichments.geo.sip.postalCode":"test postalCode","duration":"0.000","enrichments.geo.dip.country":"test country","threatinteljoinbolt.joiner.ts":"1457102731225","enrichments.geo.dip.latitude":"test latitude","enrichments.geo.sip.country":"test country","enrichments.geo.dip.city":"test city","enrichments.geo.sip.dmaCode":"test dmaCode","pkt":1,"enrichments.geo.sip.location_point":"test longitude,test latitude","ruflags":0,"roct":0,"sip":"10.0.2.15","rtag":0,"sp":39468,"enrichments.geo.sip.longitude":"test longitude","enrichments.geo.sip.latitude":"test latitude","timestamp":14
 53994988512,"risn":0,"end_time":1453994988512,"source.type":"yaf","rtt":"0.000"}
+{"adapter.threatinteladapter.end.ts":"1457102731225","enrichments.geo.dip.location_point":"test longitude,test latitude","isn":"22efa002","index.elasticsearchwriter.ts":"1457102732038","dip":"10.0.2.15","dp":39468,"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|       A|       0|       0|       0|22efa002|00000000|000|000|       1|      40|       0|       0|    0|idle ","enrichments.geo.dip.locID":"1","enrichments.geo.sip.city":"test city","enrichmentjoinbolt.joiner.ts":"1457102731211","adapter.hostfromjsonlistadapter.begin.ts":"1457102731198","tag":0,"enrichments.geo.dip.dmaCode":"test dmaCode","app":0,"oct":40,"end_reason":"idle ","enrichments.geo.sip.locID":"1","adapter.mockgeoadapter.begin.ts":"1457102731199","threatintelsplitterbolt.splitter.ts":"1457102731212","enrichments.geo.dip.postalCode":"test postalCode","start_time":145399498851
 2,"adapter.threatinteladapter.begin.ts":"1457102731223","riflags":0,"proto":6,"enrichments.host.dip.known_info.local":"YES","enrichments.geo.dip.longitude":"test longitude","iflags":"A","uflags":0,"adapter.mockgeoadapter.end.ts":"1457102731199","adapter.hostfromjsonlistadapter.end.ts":"1457102731198","enrichments.geo.sip.postalCode":"test postalCode","duration":"0.000","enrichments.geo.dip.country":"test country","threatinteljoinbolt.joiner.ts":"1457102731225","enrichments.geo.dip.latitude":"test latitude","enrichments.geo.sip.country":"test country","enrichments.geo.dip.city":"test city","enrichments.geo.sip.dmaCode":"test dmaCode","pkt":1,"enrichments.geo.sip.location_point":"test longitude,test latitude","ruflags":0,"roct":0,"sip":"216.21.170.221","rtag":0,"sp":80,"enrichments.geo.sip.longitude":"test longitude","enrichments.geo.sip.latitude":"test latitude","timestamp":1453994988512,"risn":0,"enrichments.host.dip.known_info.type":"printer","end_time":1453994988512,"enrichments.h
 ost.dip.known_info.asset_value":"important","source.type":"yaf","rtt":"0.000"}
+{"adapter.threatinteladapter.end.ts":"1457102731226","enrichments.geo.dip.location_point":"test longitude,test latitude","isn":"22efa002","index.elasticsearchwriter.ts":"1457102732038","dip":"10.0.2.15","dp":39468,"rpkt":0,"original_string":"2016-01-28 15:29:48.562|2016-01-28 15:29:48.562|   0.000|   0.000|  6|                          216.21.170.221|   80|                               10.0.2.15|39468|      AP|       0|       0|       0|22efa002|00000000|000|000|       1|     604|       0|       0|    0|idle","enrichments.geo.dip.locID":"1","enrichments.geo.sip.city":"test city","enrichmentjoinbolt.joiner.ts":"1457102731211","adapter.hostfromjsonlistadapter.begin.ts":"1457102731198","tag":0,"enrichments.geo.dip.dmaCode":"test dmaCode","app":0,"oct":604,"end_reason":"idle","enrichments.geo.sip.locID":"1","adapter.mockgeoadapter.begin.ts":"1457102731199","threatintelsplitterbolt.splitter.ts":"1457102731213","enrichments.geo.dip.postalCode":"test postalCode","start_time":1453994988562
 ,"adapter.threatinteladapter.begin.ts":"1457102731226","riflags":0,"proto":6,"enrichments.host.dip.known_info.local":"YES","enrichments.geo.dip.longitude":"test longitude","iflags":"AP","uflags":0,"adapter.mockgeoadapter.end.ts":"1457102731199","adapter.hostfromjsonlistadapter.end.ts":"1457102731198","enrichments.geo.sip.postalCode":"test postalCode","duration":"0.000","enrichments.geo.dip.country":"test country","threatinteljoinbolt.joiner.ts":"1457102731226","enrichments.geo.dip.latitude":"test latitude","enrichments.geo.sip.country":"test country","enrichments.geo.dip.city":"test city","enrichments.geo.sip.dmaCode":"test dmaCode","pkt":1,"enrichments.geo.sip.location_point":"test longitude,test latitude","ruflags":0,"roct":0,"sip":"216.21.170.221","rtag":0,"sp":80,"enrichments.geo.sip.longitude":"test longitude","enrichments.geo.sip.latitude":"test latitude","timestamp":1453994988562,"risn":0,"enrichments.host.dip.known_info.type":"printer","end_time":1453994988562,"enrichments.h
 ost.dip.known_info.asset_value":"important","source.type":"yaf","rtt":"0.000"}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/df8d682e/metron-platform/metron-integration-test/src/main/sample/data/yaf/parsed/YafExampleParsed
----------------------------------------------------------------------
diff --git a/metron-platform/metron-integration-test/src/main/sample/data/yaf/parsed/YafExampleParsed b/metron-platform/metron-integration-test/src/main/sample/data/yaf/parsed/YafExampleParsed
new file mode 100644
index 0000000..6155e98
--- /dev/null
+++ b/metron-platform/metron-integration-test/src/main/sample/data/yaf/parsed/YafExampleParsed
@@ -0,0 +1,10 @@
+{"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,"source.type":"yaf","start_time":1453994988512,"riflags":0,"rtt":"0.000","protocol":6}
+{"iflags":"A","uflags":0,"isn":10000000,"ip_dst_addr":"10.0.2.3","ip_dst_port":53,"duration":"0.000","rpkt":0,"original_string":"2016-01-28 15:29:48.502|2016-01-28 15:29:48.502|   0.000|   0.000| 17|                               10.0.2.15|37299|                                10.0.2.3|   53|       A|       0|       0|       0|10000000|00000000|000|000|       1|      56|       0|       0|    0|idle","pkt":1,"ruflags":0,"roct":0,"ip_src_addr":"10.0.2.15","tag":0,"rtag":0,"ip_src_port":37299,"timestamp":1453994988502,"app":0,"oct":56,"end_reason":"idle","risn":0,"end_time":1453994988502,"source.type":"yaf","start_time":1453994988502,"riflags":0,"rtt":"0.000","protocol":17}
+{"iflags":"A","uflags":0,"isn":0,"ip_dst_addr":"10.0.2.15","ip_dst_port":37299,"duration":"0.000","rpkt":0,"original_string":"2016-01-28 15:29:48.504|2016-01-28 15:29:48.504|   0.000|   0.000| 17|                                10.0.2.3|   53|                               10.0.2.15|37299|       A|       0|       0|       0|00000000|00000000|000|000|       1|     312|       0|       0|    0|idle","pkt":1,"ruflags":0,"roct":0,"ip_src_addr":"10.0.2.3","tag":0,"rtag":0,"ip_src_port":53,"timestamp":1453994988504,"app":0,"oct":312,"end_reason":"idle","risn":0,"end_time":1453994988504,"source.type":"yaf","start_time":1453994988504,"riflags":0,"rtt":"0.000","protocol":17}
+{"iflags":"A","uflags":0,"isn":0,"ip_dst_addr":"10.0.2.3","ip_dst_port":53,"duration":"0.000","rpkt":0,"original_string":"2016-01-28 15:29:48.504|2016-01-28 15:29:48.504|   0.000|   0.000| 17|                               10.0.2.15|56303|                                10.0.2.3|   53|       A|       0|       0|       0|00000000|00000000|000|000|       1|      56|       0|       0|    0|idle","pkt":1,"ruflags":0,"roct":0,"ip_src_addr":"10.0.2.15","tag":0,"rtag":0,"ip_src_port":56303,"timestamp":1453994988504,"app":0,"oct":56,"end_reason":"idle","risn":0,"end_time":1453994988504,"source.type":"yaf","start_time":1453994988504,"riflags":0,"rtt":"0.000","protocol":17}
+{"iflags":"A","uflags":0,"isn":0,"ip_dst_addr":"10.0.2.15","ip_dst_port":56303,"duration":"0.000","rpkt":0,"original_string":"2016-01-28 15:29:48.506|2016-01-28 15:29:48.506|   0.000|   0.000| 17|                                10.0.2.3|   53|                               10.0.2.15|56303|       A|       0|       0|       0|00000000|00000000|000|000|       1|      84|       0|       0|    0|idle","pkt":1,"ruflags":0,"roct":0,"ip_src_addr":"10.0.2.3","tag":0,"rtag":0,"ip_src_port":53,"timestamp":1453994988506,"app":0,"oct":84,"end_reason":"idle","risn":0,"end_time":1453994988506,"source.type":"yaf","start_time":1453994988506,"riflags":0,"rtt":"0.000","protocol":17}
+{"iflags":"S","uflags":0,"isn":"58c52fca","ip_dst_addr":"216.21.170.221","ip_dst_port":80,"duration":"0.000","rpkt":0,"original_string":"2016-01-28 15:29:48.508|2016-01-28 15:29:48.508|   0.000|   0.000|  6|                               10.0.2.15|39468|                          216.21.170.221|   80|       S|       0|       0|       0|58c52fca|00000000|000|000|       1|      60|       0|       0|    0|idle","pkt":1,"ruflags":0,"roct":0,"ip_src_addr":"10.0.2.15","tag":0,"rtag":0,"ip_src_port":39468,"timestamp":1453994988508,"app":0,"oct":60,"end_reason":"idle","risn":0,"end_time":1453994988508,"source.type":"yaf","start_time":1453994988508,"riflags":0,"rtt":"0.000","protocol":6}
+{"iflags":"A","uflags":0,"isn":"58c52fcb","ip_dst_addr":"216.21.170.221","ip_dst_port":80,"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|                               10.0.2.15|39468|                          216.21.170.221|   80|       A|       0|       0|       0|58c52fcb|00000000|000|000|       1|      40|       0|       0|    0|idle ","pkt":1,"ruflags":0,"roct":0,"ip_src_addr":"10.0.2.15","tag":0,"rtag":0,"ip_src_port":39468,"timestamp":1453994988512,"app":0,"oct":40,"end_reason":"idle ","risn":0,"end_time":1453994988512,"source.type":"yaf","start_time":1453994988512,"riflags":0,"rtt":"0.000","protocol":6}
+{"iflags":"AP","uflags":0,"isn":"58c52fcb","ip_dst_addr":"216.21.170.221","ip_dst_port":80,"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|                               10.0.2.15|39468|                          216.21.170.221|   80|      AP|       0|       0|       0|58c52fcb|00000000|000|000|       1|     148|       0|       0|    0|idle ","pkt":1,"ruflags":0,"roct":0,"ip_src_addr":"10.0.2.15","tag":0,"rtag":0,"ip_src_port":39468,"timestamp":1453994988512,"app":0,"oct":148,"end_reason":"idle ","risn":0,"end_time":1453994988512,"source.type":"yaf","start_time":1453994988512,"riflags":0,"rtt":"0.000","protocol":6}
+{"iflags":"A","uflags":0,"isn":"22efa002","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|       A|       0|       0|       0|22efa002|00000000|000|000|       1|      40|       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":40,"end_reason":"idle ","risn":0,"end_time":1453994988512,"source.type":"yaf","start_time":1453994988512,"riflags":0,"rtt":"0.000","protocol":6}
+{"iflags":"AP","uflags":0,"isn":"22efa002","ip_dst_addr":"10.0.2.15","ip_dst_port":39468,"duration":"0.000","rpkt":0,"original_string":"2016-01-28 15:29:48.562|2016-01-28 15:29:48.562|   0.000|   0.000|  6|                          216.21.170.221|   80|                               10.0.2.15|39468|      AP|       0|       0|       0|22efa002|00000000|000|000|       1|     604|       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":1453994988562,"app":0,"oct":604,"end_reason":"idle","risn":0,"end_time":1453994988562,"source.type":"yaf","start_time":1453994988562,"riflags":0,"rtt":"0.000","protocol":6}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/df8d682e/metron-platform/metron-integration-test/src/main/sample/data/yaf/raw/YafExampleOutput
----------------------------------------------------------------------
diff --git a/metron-platform/metron-integration-test/src/main/sample/data/yaf/raw/YafExampleOutput b/metron-platform/metron-integration-test/src/main/sample/data/yaf/raw/YafExampleOutput
new file mode 100644
index 0000000..8f3ff44
--- /dev/null
+++ b/metron-platform/metron-integration-test/src/main/sample/data/yaf/raw/YafExampleOutput
@@ -0,0 +1,10 @@
+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
+2016-01-28 15:29:48.502|2016-01-28 15:29:48.502|   0.000|   0.000| 17|                               10.0.2.15|37299|                                10.0.2.3|   53|       A|       0|       0|       0|10000000|00000000|000|000|       1|      56|       0|       0|    0|idle
+2016-01-28 15:29:48.504|2016-01-28 15:29:48.504|   0.000|   0.000| 17|                                10.0.2.3|   53|                               10.0.2.15|37299|       A|       0|       0|       0|00000000|00000000|000|000|       1|     312|       0|       0|    0|idle
+2016-01-28 15:29:48.504|2016-01-28 15:29:48.504|   0.000|   0.000| 17|                               10.0.2.15|56303|                                10.0.2.3|   53|       A|       0|       0|       0|00000000|00000000|000|000|       1|      56|       0|       0|    0|idle
+2016-01-28 15:29:48.506|2016-01-28 15:29:48.506|   0.000|   0.000| 17|                                10.0.2.3|   53|                               10.0.2.15|56303|       A|       0|       0|       0|00000000|00000000|000|000|       1|      84|       0|       0|    0|idle
+2016-01-28 15:29:48.508|2016-01-28 15:29:48.508|   0.000|   0.000|  6|                               10.0.2.15|39468|                          216.21.170.221|   80|       S|       0|       0|       0|58c52fca|00000000|000|000|       1|      60|       0|       0|    0|idle
+2016-01-28 15:29:48.512|2016-01-28 15:29:48.512|   0.000|   0.000|  6|                               10.0.2.15|39468|                          216.21.170.221|   80|       A|       0|       0|       0|58c52fcb|00000000|000|000|       1|      40|       0|       0|    0|idle 
+2016-01-28 15:29:48.512|2016-01-28 15:29:48.512|   0.000|   0.000|  6|                               10.0.2.15|39468|                          216.21.170.221|   80|      AP|       0|       0|       0|58c52fcb|00000000|000|000|       1|     148|       0|       0|    0|idle 
+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|       A|       0|       0|       0|22efa002|00000000|000|000|       1|      40|       0|       0|    0|idle 
+2016-01-28 15:29:48.562|2016-01-28 15:29:48.562|   0.000|   0.000|  6|                          216.21.170.221|   80|                               10.0.2.15|39468|      AP|       0|       0|       0|22efa002|00000000|000|000|       1|     604|       0|       0|    0|idle
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/df8d682e/metron-platform/metron-integration-test/src/main/sample/patterns/test
----------------------------------------------------------------------
diff --git a/metron-platform/metron-integration-test/src/main/sample/patterns/test b/metron-platform/metron-integration-test/src/main/sample/patterns/test
new file mode 100644
index 0000000..a88a255
--- /dev/null
+++ b/metron-platform/metron-integration-test/src/main/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/df8d682e/metron-platform/metron-parsers/src/main/config/parsers.properties
----------------------------------------------------------------------
diff --git a/metron-platform/metron-parsers/src/main/config/parsers.properties b/metron-platform/metron-parsers/src/main/config/parsers.properties
deleted file mode 100644
index 7b906d2..0000000
--- a/metron-platform/metron-parsers/src/main/config/parsers.properties
+++ /dev/null
@@ -1,21 +0,0 @@
-#  Licensed to the Apache Software Foundation (ASF) under one
-#  or more contributor license agreements.  See the NOTICE file
-#  distributed with this work for additional information
-#  regarding copyright ownership.  The ASF licenses this file
-#  to you under the Apache License, Version 2.0 (the
-#  "License"); you may not use this file except in compliance
-#  with the License.  You may obtain a copy of the License at
-#
-#      http://www.apache.org/licenses/LICENSE-2.0
-#
-#  Unless required by applicable law or agreed to in writing, software
-#  distributed under the License is distributed on an "AS IS" BASIS,
-#  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-#  See the License for the specific language governing permissions and
-#  limitations under the License.
-
-
-##### Kafka #####
-
-kafka.zk=node1:2181
-kafka.broker=node1:6667

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/df8d682e/metron-platform/metron-parsers/src/main/config/zookeeper/parsers/bluecoat.json
----------------------------------------------------------------------
diff --git a/metron-platform/metron-parsers/src/main/config/zookeeper/parsers/bluecoat.json b/metron-platform/metron-parsers/src/main/config/zookeeper/parsers/bluecoat.json
new file mode 100644
index 0000000..303bedc
--- /dev/null
+++ b/metron-platform/metron-parsers/src/main/config/zookeeper/parsers/bluecoat.json
@@ -0,0 +1,5 @@
+{
+"parserClassName":"org.apache.metron.parsers.bluecoat.BasicBluecoatParser",
+"sensorTopic":"bluecoat",
+"parserConfig": {}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/df8d682e/metron-platform/metron-parsers/src/main/config/zookeeper/parsers/bro.json
----------------------------------------------------------------------
diff --git a/metron-platform/metron-parsers/src/main/config/zookeeper/parsers/bro.json b/metron-platform/metron-parsers/src/main/config/zookeeper/parsers/bro.json
new file mode 100644
index 0000000..a9750c2
--- /dev/null
+++ b/metron-platform/metron-parsers/src/main/config/zookeeper/parsers/bro.json
@@ -0,0 +1,5 @@
+{
+  "parserClassName":"org.apache.metron.parsers.bro.BasicBroParser",
+  "sensorTopic":"bro",
+  "parserConfig": {}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/df8d682e/metron-platform/metron-parsers/src/main/config/zookeeper/parsers/snort.json
----------------------------------------------------------------------
diff --git a/metron-platform/metron-parsers/src/main/config/zookeeper/parsers/snort.json b/metron-platform/metron-parsers/src/main/config/zookeeper/parsers/snort.json
new file mode 100644
index 0000000..be36fa2
--- /dev/null
+++ b/metron-platform/metron-parsers/src/main/config/zookeeper/parsers/snort.json
@@ -0,0 +1,5 @@
+{
+  "parserClassName":"org.apache.metron.parsers.snort.BasicSnortParser",
+  "sensorTopic":"snort",
+  "parserConfig": {}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/df8d682e/metron-platform/metron-parsers/src/main/config/zookeeper/parsers/squid.json
----------------------------------------------------------------------
diff --git a/metron-platform/metron-parsers/src/main/config/zookeeper/parsers/squid.json b/metron-platform/metron-parsers/src/main/config/zookeeper/parsers/squid.json
new file mode 100644
index 0000000..6c4a69b
--- /dev/null
+++ b/metron-platform/metron-parsers/src/main/config/zookeeper/parsers/squid.json
@@ -0,0 +1,9 @@
+{
+  "parserClassName": "org.apache.metron.parsers.GrokParser",
+  "sensorTopic": "squid",
+  "parserConfig": {
+    "grokPath": "/patterns/squid",
+    "patternLabel": "SQUID_DELIMITED",
+    "timestampField": "timestamp"
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/df8d682e/metron-platform/metron-parsers/src/main/config/zookeeper/parsers/websphere.json
----------------------------------------------------------------------
diff --git a/metron-platform/metron-parsers/src/main/config/zookeeper/parsers/websphere.json b/metron-platform/metron-parsers/src/main/config/zookeeper/parsers/websphere.json
new file mode 100644
index 0000000..0f2c901
--- /dev/null
+++ b/metron-platform/metron-parsers/src/main/config/zookeeper/parsers/websphere.json
@@ -0,0 +1,11 @@
+{
+  "parserClassName":"org.apache.metron.parsers.websphere.GrokWebSphereParser",
+  "sensorTopic":"websphere",
+  "parserConfig":
+  {
+    "grokPath":"/patterns/websphere",
+    "patternLabel":"WEBSPHERE",
+    "timestampField":"timestamp_string",
+    "dateFormat":"yyyy MMM dd HH:mm:ss"
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/df8d682e/metron-platform/metron-parsers/src/main/config/zookeeper/parsers/yaf.json
----------------------------------------------------------------------
diff --git a/metron-platform/metron-parsers/src/main/config/zookeeper/parsers/yaf.json b/metron-platform/metron-parsers/src/main/config/zookeeper/parsers/yaf.json
new file mode 100644
index 0000000..6290e9f
--- /dev/null
+++ b/metron-platform/metron-parsers/src/main/config/zookeeper/parsers/yaf.json
@@ -0,0 +1,12 @@
+{
+  "parserClassName":"org.apache.metron.parsers.GrokParser",
+  "sensorTopic":"yaf",
+  "parserConfig":
+  {
+    "grokPath":"/patterns/yaf",
+    "patternLabel":"YAF_DELIMITED",
+    "timestampField":"start_time",
+    "timeFields": ["start_time", "end_time"],
+    "dateFormat":"yyyy-MM-dd HH:mm:ss.S"
+  }
+}
\ No newline at end of file

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

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

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/df8d682e/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
deleted file mode 100644
index 119f03e..0000000
--- a/metron-platform/metron-parsers/src/main/flux/squid/remote.yaml
+++ /dev/null
@@ -1,78 +0,0 @@
-# Licensed to the Apache Software Foundation (ASF) under one
-# or more contributor license agreements.  See the NOTICE file
-# distributed with this work for additional information
-# regarding copyright ownership.  The ASF licenses this file
-# to you under the Apache License, Version 2.0 (the
-# "License"); you may not use this file except in compliance
-# with the License.  You may obtain a copy of the License at
-#
-#     http://www.apache.org/licenses/LICENSE-2.0
-#
-# Unless required by applicable law or agreed to in writing, software
-# distributed under the License is distributed on an "AS IS" BASIS,
-# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-# See the License for the specific language governing permissions and
-# limitations under the License.
-
-name: "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/df8d682e/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
deleted file mode 100644
index 77893d2..0000000
--- a/metron-platform/metron-parsers/src/main/flux/squid/test.yaml
+++ /dev/null
@@ -1,78 +0,0 @@
-# Licensed to the Apache Software Foundation (ASF) under one
-# or more contributor license agreements.  See the NOTICE file
-# distributed with this work for additional information
-# regarding copyright ownership.  The ASF licenses this file
-# to you under the Apache License, Version 2.0 (the
-# "License"); you may not use this file except in compliance
-# with the License.  You may obtain a copy of the License at
-#
-#     http://www.apache.org/licenses/LICENSE-2.0
-#
-# Unless required by applicable law or agreed to in writing, software
-# distributed under the License is distributed on an "AS IS" BASIS,
-# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-# See the License for the specific language governing permissions and
-# limitations under the License.
-
-name: "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/df8d682e/metron-platform/metron-parsers/src/main/flux/yaf/remote.yaml
----------------------------------------------------------------------
diff --git a/metron-platform/metron-parsers/src/main/flux/yaf/remote.yaml b/metron-platform/metron-parsers/src/main/flux/yaf/remote.yaml
deleted file mode 100644
index f50b319..0000000
--- a/metron-platform/metron-parsers/src/main/flux/yaf/remote.yaml
+++ /dev/null
@@ -1,84 +0,0 @@
-# Licensed to the Apache Software Foundation (ASF) under one
-# or more contributor license agreements.  See the NOTICE file
-# distributed with this work for additional information
-# regarding copyright ownership.  The ASF licenses this file
-# to you under the Apache License, Version 2.0 (the
-# "License"); you may not use this file except in compliance
-# with the License.  You may obtain a copy of the License at
-#
-#     http://www.apache.org/licenses/LICENSE-2.0
-#
-# Unless required by applicable law or agreed to in writing, software
-# distributed under the License is distributed on an "AS IS" BASIS,
-# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-# See the License for the specific language governing permissions and
-# limitations under the License.
-
-name: "yaf"
-config:
-    topology.workers: 1
-
-components:
-    -   id: "parser"
-        className: "org.apache.metron.parsers.GrokParser"
-        constructorArgs:
-            - "/patterns/yaf"
-            - "YAF_DELIMITED"
-        configMethods:
-            -   name: "withTimestampField"
-                args:
-                    - "start_time"
-            -   name: "withTimeFields"
-                args:
-                    - ["start_time", "end_time"]
-            -   name: "withDateFormat"
-                args:
-                    - "yyyy-MM-dd HH:mm:ss.S"
-    -   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
-            - "yaf"
-            # zk root
-            - ""
-            # id
-            - "yaf"
-        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}"
-            - "yaf"
-            - 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/df8d682e/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
deleted file mode 100644
index e2985b8..0000000
--- a/metron-platform/metron-parsers/src/main/flux/yaf/test.yaml
+++ /dev/null
@@ -1,85 +0,0 @@
-# Licensed to the Apache Software Foundation (ASF) under one
-# or more contributor license agreements.  See the NOTICE file
-# distributed with this work for additional information
-# regarding copyright ownership.  The ASF licenses this file
-# to you under the Apache License, Version 2.0 (the
-# "License"); you may not use this file except in compliance
-# with the License.  You may obtain a copy of the License at
-#
-#     http://www.apache.org/licenses/LICENSE-2.0
-#
-# Unless required by applicable law or agreed to in writing, software
-# distributed under the License is distributed on an "AS IS" BASIS,
-# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-# See the License for the specific language governing permissions and
-# limitations under the License.
-
-name: "yaf-test"
-config:
-    topology.workers: 1
-
-
-components:
-    -   id: "parser"
-        className: "org.apache.metron.parsers.GrokParser"
-        constructorArgs:
-            - "../metron-parsers/src/main/resources/patterns/yaf"
-            - "YAF_DELIMITED"
-        configMethods:
-            -   name: "withTimestampField"
-                args:
-                    - "start_time"
-            -   name: "withTimeFields"
-                args:
-                    - ["start_time", "end_time"]
-            -   name: "withDateFormat"
-                args:
-                    - "yyyy-MM-dd HH:mm:ss.S"
-    -   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
-            - "yaf"
-            # zk root
-            - ""
-            # id
-            - "yaf"
-        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}"
-            - "yaf"
-            - 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/df8d682e/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 2b92ffb..0379080 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
@@ -39,6 +39,7 @@ import java.text.SimpleDateFormat;
 import java.util.ArrayList;
 import java.util.Date;
 import java.util.List;
+import java.util.Map;
 import java.util.TimeZone;
 
 public class GrokParser implements MessageParser<JSONObject>, Serializable {
@@ -46,61 +47,42 @@ public class GrokParser implements MessageParser<JSONObject>, Serializable {
   protected static final Logger LOG = LoggerFactory.getLogger(GrokParser.class);
 
   protected transient Grok grok;
-  protected String grokHdfsPath;
+  protected String grokPath;
   protected String patternLabel;
-  protected String[] timeFields = new String[0];
+  protected List<String> timeFields = new ArrayList<>();
   protected String timestampField;
   protected SimpleDateFormat dateFormat = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss.S z");
-  protected TimeZone timeZone = TimeZone.getTimeZone("UTC");
   protected String patternsCommonDir = "/patterns/common";
 
-  public GrokParser(String grokHdfsPath, String patternLabel) {
-    this.grokHdfsPath = grokHdfsPath;
-    this.patternLabel = patternLabel;
-  }
-
-  public GrokParser withTimestampField(String timestampField) {
-    this.timestampField = timestampField;
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("Grok parser settting timestamp field: " + timestampField);
-    }
-    return this;
-  }
-
-  public GrokParser withTimeFields(String... timeFields) {
-    this.timeFields = timeFields;
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("Grok parser settting time fields: " + timeFields);
-    }
-    return this;
-  }
-
-  public GrokParser withDateFormat(String dateFormat) {
-    this.dateFormat = new SimpleDateFormat(dateFormat);
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("Grok parser settting date format: " + dateFormat);
-    }
-    return this;
-  }
-
-  public GrokParser withTimeZone(String timeZone) {
-    this.timeZone = TimeZone.getTimeZone(timeZone);
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("Grok parser settting timezone: " + timeZone);
+  @Override
+  public void configure(Map<String, Object> parserConfig) {
+    this.grokPath = (String) parserConfig.get("grokPath");
+    this.patternLabel = (String) parserConfig.get("patternLabel");
+    this.timestampField = (String) parserConfig.get("timestampField");
+    List<String> timeFieldsParam = (List<String>) parserConfig.get("timeFields");
+    if (timeFieldsParam != null) {
+      this.timeFields = timeFieldsParam;
+    }
+    String dateFormatParam = (String) parserConfig.get("dateFormat");
+    if (dateFormatParam != null) {
+      this.dateFormat = new SimpleDateFormat(dateFormatParam);
+    }
+    String timeZoneParam = (String) parserConfig.get("timeZone");
+    if (timeZoneParam != null) {
+      dateFormat.setTimeZone(TimeZone.getTimeZone(timeZoneParam));
+    } else {
+      dateFormat.setTimeZone(TimeZone.getTimeZone("UTC"));
     }
-    return this;
   }
 
   public InputStream openInputStream(String streamName) throws IOException {
-    InputStream is = getClass().getResourceAsStream(streamName);
-    if(is == null) {
-      FileSystem fs = FileSystem.get(new Configuration());
-      Path path = new Path(streamName);
-      if(fs.exists(path)) {
-        return fs.open(path);
-      }
+    FileSystem fs = FileSystem.get(new Configuration());
+    Path path = new Path(streamName);
+    if(fs.exists(path)) {
+      return fs.open(path);
+    } else {
+      return getClass().getResourceAsStream(streamName);
     }
-    return is;
   }
 
   @Override
@@ -119,12 +101,12 @@ public class GrokParser implements MessageParser<JSONObject>, Serializable {
 
       grok.addPatternFromReader(new InputStreamReader(commonInputStream));
       if (LOG.isDebugEnabled()) {
-        LOG.debug("Loading parser-specific patterns from: " + grokHdfsPath);
+        LOG.debug("Loading parser-specific patterns from: " + grokPath);
       }
 
-      InputStream patterInputStream = openInputStream(grokHdfsPath);
+      InputStream patterInputStream = openInputStream(grokPath);
       if (patterInputStream == null) {
-        throw new RuntimeException("Grok parser unable to initialize grok parser: Unable to load " + grokHdfsPath
+        throw new RuntimeException("Grok parser unable to initialize grok parser: Unable to load " + grokPath
                 + " from either classpath or HDFS");
       }
       grok.addPatternFromReader(new InputStreamReader(patterInputStream));
@@ -166,7 +148,7 @@ public class GrokParser implements MessageParser<JSONObject>, Serializable {
       if (message.size() == 0)
         throw new RuntimeException("Grok statement produced a null message. Original message was: "
                 + originalMessage + " and the parsed message was: " + message + " . Check the pattern at: "
-                + grokHdfsPath);
+                + grokPath);
 
       message.put("original_string", originalMessage);
       for (String timeField : timeFields) {
@@ -222,7 +204,6 @@ public class GrokParser implements MessageParser<JSONObject>, Serializable {
       LOG.debug("Grok perser converting timestamp to epoch: " + datetime);
     }
 
-    dateFormat.setTimeZone(timeZone);
     Date date = dateFormat.parse(datetime);
     if (LOG.isDebugEnabled()) {
       LOG.debug("Grok perser converted timestamp to epoch: " + date);

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/df8d682e/metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/asa/GrokAsaParser.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/asa/GrokAsaParser.java b/metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/asa/GrokAsaParser.java
index 0f8a862..4f1c8b0 100644
--- a/metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/asa/GrokAsaParser.java
+++ b/metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/asa/GrokAsaParser.java
@@ -186,7 +186,12 @@ public class GrokAsaParser extends BasicParser {
 
 		return timeInMillisSinceEpoch;
 	}
-	
+
+	@Override
+	public void configure(Map<String, Object> parserConfig) {
+
+	}
+
 	@Override
 	public void init() {
 		// pattern_url = Resources.getResource("patterns/asa");

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/df8d682e/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
index 9d9ab6b..fcefcc2 100644
--- 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
@@ -23,6 +23,7 @@ import java.util.ArrayList;
 import java.util.Calendar;
 import java.util.Date;
 import java.util.List;
+import java.util.Map;
 
 import org.apache.metron.parsers.BasicParser;
 import org.json.simple.JSONObject;
@@ -34,7 +35,12 @@ 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 configure(Map<String, Object> parserConfig) {
+
+	}
+
 	@Override
 	public void init() {
 

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/df8d682e/metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/bolt/ParserBolt.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/bolt/ParserBolt.java b/metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/bolt/ParserBolt.java
index f0f1bd8..39f2641 100644
--- a/metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/bolt/ParserBolt.java
+++ b/metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/bolt/ParserBolt.java
@@ -23,6 +23,7 @@ import backtype.storm.topology.OutputFieldsDeclarer;
 import backtype.storm.tuple.Tuple;
 import org.apache.metron.common.Constants;
 import org.apache.metron.common.bolt.ConfiguredBolt;
+import org.apache.metron.common.bolt.ConfiguredParserBolt;
 import org.apache.metron.parsers.filters.GenericMessageFilter;
 import org.apache.metron.common.utils.ErrorUtils;
 import org.apache.metron.parsers.interfaces.MessageFilter;
@@ -33,7 +34,7 @@ import org.json.simple.JSONObject;
 import java.util.List;
 import java.util.Map;
 
-public class ParserBolt extends ConfiguredBolt {
+public class ParserBolt extends ConfiguredParserBolt {
 
   private OutputCollector collector;
   private MessageParser<JSONObject> parser;

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/df8d682e/metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/bro/BasicBroParser.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/bro/BasicBroParser.java b/metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/bro/BasicBroParser.java
index 74ea082..4052e86 100644
--- a/metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/bro/BasicBroParser.java
+++ b/metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/bro/BasicBroParser.java
@@ -27,6 +27,7 @@ import org.slf4j.LoggerFactory;
 
 import java.util.ArrayList;
 import java.util.List;
+import java.util.Map;
 
 @SuppressWarnings("serial")
 public class BasicBroParser extends BasicParser {
@@ -36,6 +37,11 @@ public class BasicBroParser extends BasicParser {
     private JSONCleaner cleaner = new JSONCleaner();
 
     @Override
+    public void configure(Map<String, Object> parserConfig) {
+
+    }
+
+    @Override
     public void init() {
 
     }

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/df8d682e/metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/fireeye/BasicFireEyeParser.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/fireeye/BasicFireEyeParser.java b/metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/fireeye/BasicFireEyeParser.java
index b90d2b7..498248a 100644
--- a/metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/fireeye/BasicFireEyeParser.java
+++ b/metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/fireeye/BasicFireEyeParser.java
@@ -31,6 +31,7 @@ import java.text.ParseException;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.List;
+import java.util.Map;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 
@@ -59,6 +60,11 @@ public class BasicFireEyeParser extends BasicParser {
 	}
 
 	@Override
+	public void configure(Map<String, Object> parserConfig) {
+
+	}
+
+	@Override
 	public void init() {
 
 	}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/df8d682e/metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/interfaces/MessageParser.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/interfaces/MessageParser.java b/metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/interfaces/MessageParser.java
index 11efa53..81d1b1a 100644
--- a/metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/interfaces/MessageParser.java
+++ b/metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/interfaces/MessageParser.java
@@ -17,10 +17,13 @@
  */
 package org.apache.metron.parsers.interfaces;
 
+import java.io.Serializable;
 import java.util.List;
+import java.util.Map;
 
-public interface MessageParser<T> {
+public interface MessageParser<T> extends Serializable {
 
+	void configure(Map<String, Object> parserConfig);
 	void init();
 	List<T> parse(byte[] rawMessage);
 	boolean validate(T message);

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/df8d682e/metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/ise/BasicIseParser.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/ise/BasicIseParser.java b/metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/ise/BasicIseParser.java
index 19b3ac6..2d559ac 100644
--- a/metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/ise/BasicIseParser.java
+++ b/metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/ise/BasicIseParser.java
@@ -28,6 +28,7 @@ import org.slf4j.LoggerFactory;
 import java.io.StringReader;
 import java.util.ArrayList;
 import java.util.List;
+import java.util.Map;
 
 @SuppressWarnings("serial")
 public class BasicIseParser extends BasicParser {
@@ -37,6 +38,11 @@ public class BasicIseParser extends BasicParser {
 	static final transient ISEParser _parser = new ISEParser("header=");
 
 	@Override
+	public void configure(Map<String, Object> parserConfig) {
+
+	}
+
+	@Override
 	public void init() {
 
 	}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/df8d682e/metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/lancope/BasicLancopeParser.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/lancope/BasicLancopeParser.java b/metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/lancope/BasicLancopeParser.java
index 6c25d67..83eedcc 100644
--- a/metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/lancope/BasicLancopeParser.java
+++ b/metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/lancope/BasicLancopeParser.java
@@ -28,6 +28,7 @@ import java.text.SimpleDateFormat;
 import java.util.ArrayList;
 import java.util.Date;
 import java.util.List;
+import java.util.Map;
 
 @SuppressWarnings("serial")
 public class BasicLancopeParser extends BasicParser {
@@ -38,6 +39,11 @@ public class BasicLancopeParser extends BasicParser {
 					.class);
 
 	@Override
+	public void configure(Map<String, Object> parserConfig) {
+
+	}
+
+	@Override
 	public void init() {
 
 	}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/df8d682e/metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/logstash/BasicLogstashParser.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/logstash/BasicLogstashParser.java b/metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/logstash/BasicLogstashParser.java
index 39177aa..2f5310c 100644
--- a/metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/logstash/BasicLogstashParser.java
+++ b/metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/logstash/BasicLogstashParser.java
@@ -24,10 +24,16 @@ import org.json.simple.parser.JSONParser;
 import java.text.SimpleDateFormat;
 import java.util.ArrayList;
 import java.util.List;
+import java.util.Map;
 
 public class BasicLogstashParser extends BasicParser {
 
 	@Override
+	public void configure(Map<String, Object> parserConfig) {
+
+	}
+
+	@Override
 	public void init() {
 
 	}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/df8d682e/metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/paloalto/BasicPaloAltoFirewallParser.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/paloalto/BasicPaloAltoFirewallParser.java b/metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/paloalto/BasicPaloAltoFirewallParser.java
index c67e2b5..e6b9274 100644
--- a/metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/paloalto/BasicPaloAltoFirewallParser.java
+++ b/metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/paloalto/BasicPaloAltoFirewallParser.java
@@ -27,6 +27,7 @@ import java.net.MalformedURLException;
 import java.net.URL;
 import java.util.ArrayList;
 import java.util.List;
+import java.util.Map;
 
 public class BasicPaloAltoFirewallParser extends BasicParser {
 
@@ -90,6 +91,10 @@ public class BasicPaloAltoFirewallParser extends BasicParser {
   public static final String PktsSent = "pkts_sent";
   public static final String PktsReceived = "pkts_received";
 
+  @Override
+  public void configure(Map<String, Object> parserConfig) {
+
+  }
 
   @Override
   public void init() {

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/df8d682e/metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/snort/BasicSnortParser.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/snort/BasicSnortParser.java b/metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/snort/BasicSnortParser.java
index a82d8d6..1fcb6c4 100644
--- a/metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/snort/BasicSnortParser.java
+++ b/metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/snort/BasicSnortParser.java
@@ -73,6 +73,11 @@ public class BasicSnortParser extends BasicParser {
   private String recordDelimiter = ",";
 
   @Override
+  public void configure(Map<String, Object> parserConfig) {
+
+  }
+
+  @Override
   public void init() {
 
   }

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/df8d682e/metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/sourcefire/BasicSourcefireParser.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/sourcefire/BasicSourcefireParser.java b/metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/sourcefire/BasicSourcefireParser.java
index 40badcd..0bc2671 100644
--- a/metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/sourcefire/BasicSourcefireParser.java
+++ b/metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/sourcefire/BasicSourcefireParser.java
@@ -25,6 +25,7 @@ import org.slf4j.LoggerFactory;
 
 import java.util.ArrayList;
 import java.util.List;
+import java.util.Map;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 
@@ -42,6 +43,11 @@ public class BasicSourcefireParser extends BasicParser {
 	Pattern pattern = Pattern.compile(domain_name_regex);
 
 	@Override
+	public void configure(Map<String, Object> parserConfig) {
+
+	}
+
+	@Override
 	public void init() {
 
 	}



[15/15] incubator-metron git commit: METRON-142 Simplify Parser configuration (merrimanr via cestella) closes apache/incubator-metron#120

Posted by ce...@apache.org.
METRON-142 Simplify Parser configuration (merrimanr via cestella) closes apache/incubator-metron#120


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

Branch: refs/heads/master
Commit: df8d682e8ffe365cd9d03000730f43342cc8dd95
Parents: 00f8588
Author: merrimanr <me...@gmail.com>
Authored: Mon May 16 14:07:25 2016 -0400
Committer: cstella <ce...@gmail.com>
Committed: Mon May 16 14:07:25 2016 -0400

----------------------------------------------------------------------
 metron-deployment/playbooks/metron_install.yml  |     5 +-
 .../files/config/sensors/bro.json               |    20 -
 .../files/config/sensors/pcap.json              |    23 -
 .../files/config/sensors/snort.json             |    28 -
 .../files/config/sensors/yaf.json               |    22 -
 .../roles/metron_streaming/tasks/main.yml       |     6 +
 .../metron_streaming/tasks/source_config.yml    |    12 -
 .../roles/metron_streaming/tasks/topologies.yml |     9 -
 .../monit/templates/scripts/start_topology.sh   |     2 +-
 .../pcapservice/ConfigurationUtilTest.java      |     2 -
 .../org/apache/metron/common/Constants.java     |     4 -
 .../metron/common/bolt/ConfiguredBolt.java      |    35 +-
 .../common/bolt/ConfiguredEnrichmentBolt.java   |    64 +
 .../common/bolt/ConfiguredParserBolt.java       |    64 +
 .../common/configuration/Configuration.java     |     4 -
 .../common/configuration/ConfigurationType.java |    29 +-
 .../common/configuration/Configurations.java    |    46 +-
 .../configuration/ConfigurationsUtils.java      |   151 +-
 .../configuration/EnrichmentConfigurations.java |    49 +
 .../configuration/ParserConfigurations.java     |    48 +
 .../configuration/SensorParserConfig.java       |    89 +
 .../common/interfaces/BulkMessageWriter.java    |     6 +-
 .../metron/common/spout/kafka/SpoutConfig.java  |     6 +-
 .../metron/common/utils/ReflectionUtils.java    |    34 +-
 .../common/bolt/BaseConfiguredBoltTest.java     |    48 +
 .../metron/common/bolt/ConfiguredBoltTest.java  |   162 -
 .../bolt/ConfiguredEnrichmentBoltTest.java      |   137 +
 .../common/bolt/ConfiguredParserBoltTest.java   |   129 +
 .../ConfigurationManagerIntegrationTest.java    |     7 +-
 .../common/cli/ConfigurationsUtilsTest.java     |    36 +-
 .../common/configuration/ConfigurationTest.java |     2 +-
 .../configuration/ConfigurationsTest.java       |     3 +-
 .../SensorEnrichmentConfigTest.java             |     2 +-
 .../SensorEnrichmentUpdateConfigTest.java       |     2 +-
 .../writer/ElasticsearchWriter.java             |     6 +-
 .../ElasticsearchEnrichmentIntegrationTest.java |     2 +-
 .../src/main/assembly/assembly.xml              |    12 +
 .../main/config/zookeeper/enrichments/bro.json  |    20 +
 .../config/zookeeper/enrichments/snort.json     |    28 +
 .../config/zookeeper/enrichments/websphere.json |    20 +
 .../main/config/zookeeper/enrichments/yaf.json  |    22 +
 .../src/main/flux/enrichment/test.yaml          |    10 -
 .../enrichment/bolt/BulkMessageWriterBolt.java  |     3 +-
 .../enrichment/bolt/GenericEnrichmentBolt.java  |    13 +-
 .../apache/metron/enrichment/bolt/JoinBolt.java |     3 +-
 .../metron/enrichment/bolt/SplitBolt.java       |     3 +-
 .../apache/metron/writer/hdfs/HdfsWriter.java   |     8 +-
 .../bolt/BulkMessageWriterBoltTest.java         |    13 +-
 .../enrichment/bolt/EnrichmentJoinBoltTest.java |     2 +-
 .../bolt/GenericEnrichmentBoltTest.java         |     4 +-
 .../metron/enrichment/bolt/SplitBoltTest.java   |    10 +-
 .../bolt/ThreatIntelJoinBoltTest.java           |     6 +-
 .../main/config/zookeeper/enrichments/test.json |    51 +
 .../src/main/config/zookeeper/global.json       |    10 +
 .../metron/integration/BaseIntegrationTest.java |     7 +-
 .../integration/EnrichmentIntegrationTest.java  |    18 +-
 .../components/ConfigUploadComponent.java       |    82 +
 .../components/KafkaWithZKComponent.java        |     2 +
 .../metron/integration/utils/SampleUtil.java    |    21 +-
 .../main/resources/sample/config/global.json    |    10 -
 .../resources/sample/config/sensors/bro.json    |    29 -
 .../resources/sample/config/sensors/pcap.json   |    24 -
 .../resources/sample/config/sensors/snort.json  |    32 -
 .../sample/config/sensors/websphere.json        |    20 -
 .../resources/sample/config/sensors/yaf.json    |    51 -
 .../sample/data/SampleIndexed/YafIndexed        |    10 -
 .../data/SampleInput/.PCAPExampleOutput.crc     |   Bin 44 -> 0 bytes
 .../resources/sample/data/SampleInput/AsaOutput |   100 -
 .../sample/data/SampleInput/BluecoatSyslog.txt  |   144 -
 .../sample/data/SampleInput/BroExampleOutput    | 23411 -----------------
 .../data/SampleInput/FireeyeExampleOutput       |    90 -
 .../sample/data/SampleInput/ISESampleOutput     |   308 -
 .../data/SampleInput/LancopeExampleOutput       |    40 -
 .../sample/data/SampleInput/PCAPExampleOutput   |   Bin 4510 -> 0 bytes
 .../sample/data/SampleInput/PaloaltoOutput      |   100 -
 .../sample/data/SampleInput/SnortOutput         |     3 -
 .../data/SampleInput/SourcefireExampleOutput    |     2 -
 .../sample/data/SampleInput/SquidExampleOutput  |     2 -
 .../sample/data/SampleInput/WebsphereOutput.txt |     5 -
 .../sample/data/SampleInput/YafExampleOutput    |    10 -
 .../sample/data/SampleParsed/BluecoatParsed     |   144 -
 .../sample/data/SampleParsed/SnortParsed        |     3 -
 .../sample/data/SampleParsed/SquidExampleParsed |     2 -
 .../sample/data/SampleParsed/WebsphereParsed    |     5 -
 .../sample/data/SampleParsed/YafExampleParsed   |    10 -
 .../src/main/resources/sample/patterns/test     |     2 -
 .../data/SampleInput/.PCAPExampleOutput.crc     |   Bin 0 -> 44 bytes
 .../src/main/sample/data/SampleInput/AsaOutput  |   100 +
 .../data/SampleInput/FireeyeExampleOutput       |    90 +
 .../sample/data/SampleInput/ISESampleOutput     |   308 +
 .../data/SampleInput/LancopeExampleOutput       |    40 +
 .../sample/data/SampleInput/PCAPExampleOutput   |   Bin 0 -> 4510 bytes
 .../main/sample/data/SampleInput/PaloaltoOutput |   100 +
 .../data/SampleInput/SourcefireExampleOutput    |     2 +
 .../sample/data/bluecoat/parsed/BluecoatParsed  |   144 +
 .../sample/data/bluecoat/raw/BluecoatSyslog.txt |   144 +
 .../sample/data/bro/parsed/BroExampleParsed     |    10 +
 .../main/sample/data/bro/raw/BroExampleOutput   |    10 +
 .../main/sample/data/snort/parsed/SnortParsed   |     3 +
 .../src/main/sample/data/snort/raw/SnortOutput  |     3 +
 .../sample/data/squid/parsed/SquidExampleParsed |     2 +
 .../sample/data/squid/raw/SquidExampleOutput    |     2 +
 .../main/sample/data/test/indexed/TestIndexed   |    10 +
 .../sample/data/test/parsed/TestExampleParsed   |    10 +
 .../data/websphere/parsed/WebsphereParsed       |     5 +
 .../data/websphere/raw/WebsphereOutput.txt      |     5 +
 .../src/main/sample/data/yaf/indexed/YafIndexed |    10 +
 .../sample/data/yaf/parsed/YafExampleParsed     |    10 +
 .../main/sample/data/yaf/raw/YafExampleOutput   |    10 +
 .../src/main/sample/patterns/test               |     2 +
 .../src/main/config/parsers.properties          |    21 -
 .../main/config/zookeeper/parsers/bluecoat.json |     5 +
 .../src/main/config/zookeeper/parsers/bro.json  |     5 +
 .../main/config/zookeeper/parsers/snort.json    |     5 +
 .../main/config/zookeeper/parsers/squid.json    |     9 +
 .../config/zookeeper/parsers/websphere.json     |    11 +
 .../src/main/config/zookeeper/parsers/yaf.json  |    12 +
 .../src/main/flux/bluecoat/remote.yaml          |    71 -
 .../src/main/flux/bluecoat/test.yaml            |    72 -
 .../src/main/flux/bro/remote.yaml               |    71 -
 .../metron-parsers/src/main/flux/bro/test.yaml  |    72 -
 .../src/main/flux/snort/remote.yaml             |    69 -
 .../src/main/flux/snort/test.yaml               |    69 -
 .../src/main/flux/squid/remote.yaml             |    78 -
 .../src/main/flux/squid/test.yaml               |    78 -
 .../src/main/flux/yaf/remote.yaml               |    84 -
 .../metron-parsers/src/main/flux/yaf/test.yaml  |    85 -
 .../org/apache/metron/parsers/GrokParser.java   |    81 +-
 .../metron/parsers/asa/GrokAsaParser.java       |     7 +-
 .../parsers/bluecoat/BasicBluecoatParser.java   |     8 +-
 .../apache/metron/parsers/bolt/ParserBolt.java  |     3 +-
 .../metron/parsers/bro/BasicBroParser.java      |     6 +
 .../parsers/fireeye/BasicFireEyeParser.java     |     6 +
 .../parsers/interfaces/MessageParser.java       |     5 +-
 .../metron/parsers/ise/BasicIseParser.java      |     6 +
 .../parsers/lancope/BasicLancopeParser.java     |     6 +
 .../parsers/logstash/BasicLogstashParser.java   |     6 +
 .../paloalto/BasicPaloAltoFirewallParser.java   |     5 +
 .../metron/parsers/snort/BasicSnortParser.java  |     5 +
 .../sourcefire/BasicSourcefireParser.java       |     6 +
 .../parsers/topology/ParserTopologyBuilder.java |    58 +
 .../parsers/topology/ParserTopologyCLI.java     |   127 +
 .../parsers/websphere/GrokWebSphereParser.java  |     4 -
 .../src/main/scripts/start_parser_topology.sh   |     2 +-
 .../apache/metron/parsers/GrokParserTest.java   |    26 +-
 .../metron/parsers/SampleGrokParserTest.java    |    11 +-
 .../apache/metron/parsers/SquidParserTest.java  |     7 +-
 .../apache/metron/parsers/YafParserTest.java    |    10 +-
 .../integration/BluecoatIntegrationTest.java    |    29 +-
 .../parsers/integration/BroIntegrationTest.java |    37 +
 .../integration/ParserIntegrationTest.java      |   104 +-
 .../parsers/integration/ParserValidation.java   |    26 +
 .../integration/SnortIntegrationTest.java       |    29 +-
 .../integration/SquidIntegrationTest.java       |    30 +-
 .../integration/WebSphereIntegrationTest.java   |    40 +-
 .../parsers/integration/YafIntegrationTest.java |    29 +-
 .../components/ParserTopologyComponent.java     |    84 +
 .../validation/SampleDataValidation.java        |    52 +
 .../websphere/GrokWebSphereParserTest.java      |    54 +-
 .../PcapTopologyIntegrationTest.java            |     2 +-
 .../org/apache/metron/pcap/PcapHelperTest.java  |     2 +-
 .../apache/metron/solr/writer/SolrWriter.java   |     5 +-
 .../SolrEnrichmentIntegrationTest.java          |     4 +-
 .../metron/solr/writer/SolrWriterTest.java      |     3 +-
 .../java/org/apache/metron/TestConstants.java   |    11 +-
 .../org/apache/metron/test/TestDataType.java    |    31 +
 .../test/bolt/BaseEnrichmentBoltTest.java       |    12 +-
 .../metron/test/utils/SampleDataUtils.java      |    41 +
 .../metron/test/utils/ValidationUtils.java      |    49 +
 metron-platform/pom.xml                         |     6 +-
 pom.xml                                         |     4 +-
 171 files changed, 3014 insertions(+), 26121 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/df8d682e/metron-deployment/playbooks/metron_install.yml
----------------------------------------------------------------------
diff --git a/metron-deployment/playbooks/metron_install.yml b/metron-deployment/playbooks/metron_install.yml
index 0118213..7e73cbb 100644
--- a/metron-deployment/playbooks/metron_install.yml
+++ b/metron-deployment/playbooks/metron_install.yml
@@ -26,7 +26,6 @@
   become: true
   roles:
     - role: metron_common
-    - role: monit
   tags:
     - metron-prereqs
 
@@ -106,6 +105,8 @@
 - hosts: metron
   become: true
   roles:
-    - monit-start
+    - role: ambari_gather_facts
+    - role: monit
+    - role: monit-start
   tags:
     - start

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/df8d682e/metron-deployment/roles/metron_streaming/files/config/sensors/bro.json
----------------------------------------------------------------------
diff --git a/metron-deployment/roles/metron_streaming/files/config/sensors/bro.json b/metron-deployment/roles/metron_streaming/files/config/sensors/bro.json
deleted file mode 100644
index 0eb34b3..0000000
--- a/metron-deployment/roles/metron_streaming/files/config/sensors/bro.json
+++ /dev/null
@@ -1,20 +0,0 @@
-{
-  "index": "bro",
-  "batchSize": 5,
-  "enrichment" : {
-    "fieldMap": {
-      "geo": ["ip_dst_addr", "ip_src_addr"],
-      "host": ["host"]
-    }
-  },
-  "threatIntel": {
-    "fieldMap": {
-      "hbaseThreatIntel": ["ip_src_addr", "ip_dst_addr"]
-    },
-    "fieldToTypeMap": {
-      "ip_src_addr" : ["malicious_ip"],
-      "ip_dst_addr" : ["malicious_ip"]
-    }
-  }
-}
-

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/df8d682e/metron-deployment/roles/metron_streaming/files/config/sensors/pcap.json
----------------------------------------------------------------------
diff --git a/metron-deployment/roles/metron_streaming/files/config/sensors/pcap.json b/metron-deployment/roles/metron_streaming/files/config/sensors/pcap.json
deleted file mode 100644
index 8a3eab1..0000000
--- a/metron-deployment/roles/metron_streaming/files/config/sensors/pcap.json
+++ /dev/null
@@ -1,23 +0,0 @@
-{
-  "index": "pcap",
-  "batchSize": 5,
-  "enrichment" : {
-    "fieldMap":
-      {
-      "geo": ["ip_src_addr", "ip_dst_addr"],
-      "host": ["ip_src_addr", "ip_dst_addr"]
-    }
-  },
-  "threatIntel" : {
-    "fieldMap":
-      {
-      "hbaseThreatIntel": ["ip_dst_addr", "ip_src_addr"]
-    },
-    "fieldToTypeMap":
-      {
-      "ip_dst_addr" : [ "malicious_ip" ]
-    ,"ip_src_addr" : [ "malicious_ip" ]
-    }
-  }
-}
-

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/df8d682e/metron-deployment/roles/metron_streaming/files/config/sensors/snort.json
----------------------------------------------------------------------
diff --git a/metron-deployment/roles/metron_streaming/files/config/sensors/snort.json b/metron-deployment/roles/metron_streaming/files/config/sensors/snort.json
deleted file mode 100644
index 9dfc80e..0000000
--- a/metron-deployment/roles/metron_streaming/files/config/sensors/snort.json
+++ /dev/null
@@ -1,28 +0,0 @@
-{
-  "index": "snort",
-  "batchSize": 1,
-  "enrichment" : {
-    "fieldMap":
-      {
-      "geo": ["ip_dst_addr", "ip_src_addr"],
-      "host": ["host"]
-    }
-  },
-  "threatIntel" : {
-    "fieldMap":
-      {
-      "hbaseThreatIntel": ["ip_src_addr", "ip_dst_addr"]
-    },
-    "fieldToTypeMap":
-      {
-      "ip_src_addr" : ["malicious_ip"],
-      "ip_dst_addr" : ["malicious_ip"]
-    },
-    "triageConfig" : {
-      "riskLevelRules" : {
-        "not(IN_SUBNET(ip_dst_addr, '192.168.0.0/24'))" : 10
-      },
-      "aggregator" : "MAX"
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/df8d682e/metron-deployment/roles/metron_streaming/files/config/sensors/yaf.json
----------------------------------------------------------------------
diff --git a/metron-deployment/roles/metron_streaming/files/config/sensors/yaf.json b/metron-deployment/roles/metron_streaming/files/config/sensors/yaf.json
deleted file mode 100644
index 4e67748..0000000
--- a/metron-deployment/roles/metron_streaming/files/config/sensors/yaf.json
+++ /dev/null
@@ -1,22 +0,0 @@
-{
-  "index": "yaf",
-  "batchSize": 5,
-  "enrichment" : {
-    "fieldMap":
-      {
-      "geo": ["ip_dst_addr", "ip_src_addr"],
-      "host": ["host"]
-    }
-  },
-  "threatIntel": {
-    "fieldMap":
-      {
-      "hbaseThreatIntel": ["ip_src_addr", "ip_dst_addr"]
-    },
-    "fieldToTypeMap":
-      {
-      "ip_src_addr" : ["malicious_ip"],
-      "ip_dst_addr" : ["malicious_ip"]
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/df8d682e/metron-deployment/roles/metron_streaming/tasks/main.yml
----------------------------------------------------------------------
diff --git a/metron-deployment/roles/metron_streaming/tasks/main.yml b/metron-deployment/roles/metron_streaming/tasks/main.yml
index d44f2ef..c960dba 100644
--- a/metron-deployment/roles/metron_streaming/tasks/main.yml
+++ b/metron-deployment/roles/metron_streaming/tasks/main.yml
@@ -22,6 +22,12 @@
       - { name: 'bin'}
       - { name: 'config'}
 
+- name: Create Source Config Directory
+  file:
+    path: "{{ zookeeper_config_path }}"
+    state: directory
+    mode: 0755
+
 - include: copy_bundles.yml
 
 - name: Get Default mysql passowrd

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/df8d682e/metron-deployment/roles/metron_streaming/tasks/source_config.yml
----------------------------------------------------------------------
diff --git a/metron-deployment/roles/metron_streaming/tasks/source_config.yml b/metron-deployment/roles/metron_streaming/tasks/source_config.yml
index 21292b1..1c967bd 100644
--- a/metron-deployment/roles/metron_streaming/tasks/source_config.yml
+++ b/metron-deployment/roles/metron_streaming/tasks/source_config.yml
@@ -15,10 +15,6 @@
 #  limitations under the License.
 #
 ---
-- name: Create Source Config Directory
-  file:
-    path: "{{ zookeeper_config_path }}"
-    state: directory
 
 - name: Copy Elasticsearch Global Config File
   template:
@@ -34,14 +30,6 @@
     mode: 0644
   when: install_solr | default(False) == True
 
-- name: Copy Sensor Config Files
-  copy:
-    src: "{{ item }}"
-    dest: "{{ zookeeper_config_path }}"
-    mode: 0644
-  with_items:
-    - ../roles/metron_streaming/files/config/
-
 - name: Load Config
   shell: "{{ metron_directory }}/bin/zk_load_configs.sh --mode PUSH -i {{ zookeeper_config_path }} -z {{ zookeeper_url }} && touch {{ zookeeper_config_path }}/configured"
 

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/df8d682e/metron-deployment/roles/metron_streaming/tasks/topologies.yml
----------------------------------------------------------------------
diff --git a/metron-deployment/roles/metron_streaming/tasks/topologies.yml b/metron-deployment/roles/metron_streaming/tasks/topologies.yml
index d7fdb0c..17bcfbb 100644
--- a/metron-deployment/roles/metron_streaming/tasks/topologies.yml
+++ b/metron-deployment/roles/metron_streaming/tasks/topologies.yml
@@ -25,15 +25,6 @@
     - { regexp: "kafka.pcap.out=", line: "kafka.pcap.out={{ pcap_hdfs_path }}" }
     - { regexp: "spout.kafka.topic.pcap=", line: "spout.kafka.topic.pcap={{ pycapa_topic }}" }
 
-- name: Configure Metron Parser Topologies
-  lineinfile:
-    dest: "{{ metron_parsers_properties_config_path }}"
-    regexp: "{{ item.regexp }}"
-    line: "{{ item.line }}"
-  with_items:
-    - { regexp: "kafka.zk=", line: "kafka.zk={{ zookeeper_url }}" }
-    - { regexp: "kafka.broker=", line: "kafka.broker={{ kafka_broker_url }}" }
-
 - name: Configure Metron Solr topology
   lineinfile: >
     dest={{ metron_solr_properties_config_path }}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/df8d682e/metron-deployment/roles/monit/templates/scripts/start_topology.sh
----------------------------------------------------------------------
diff --git a/metron-deployment/roles/monit/templates/scripts/start_topology.sh b/metron-deployment/roles/monit/templates/scripts/start_topology.sh
index a8a60fa..5e5286d 100644
--- a/metron-deployment/roles/monit/templates/scripts/start_topology.sh
+++ b/metron-deployment/roles/monit/templates/scripts/start_topology.sh
@@ -22,4 +22,4 @@
 #
 export METRON_VERSION={{ metron_version }}
 export METRON_HOME={{ metron_directory }}
-$METRON_HOME/bin/start_parser_topology.sh {{ item }}
+$METRON_HOME/bin/start_parser_topology.sh -k {{ kafka_broker_url }} -z {{ zookeeper_url }} -s {{ item }}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/df8d682e/metron-platform/metron-api/src/test/java/org/apache/metron/pcapservice/ConfigurationUtilTest.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-api/src/test/java/org/apache/metron/pcapservice/ConfigurationUtilTest.java b/metron-platform/metron-api/src/test/java/org/apache/metron/pcapservice/ConfigurationUtilTest.java
index 3c0a77b..4ecb3a4 100644
--- a/metron-platform/metron-api/src/test/java/org/apache/metron/pcapservice/ConfigurationUtilTest.java
+++ b/metron-platform/metron-api/src/test/java/org/apache/metron/pcapservice/ConfigurationUtilTest.java
@@ -27,7 +27,6 @@ public class ConfigurationUtilTest {
 
   @Test
   public void test_getPcapOutputPath() {
-    Assert.assertEquals(ConfigurationUtil.getPcapOutputPath(), null);
     ConfigurationUtil.setPcapOutputPath("/foo");
     Assert.assertEquals(ConfigurationUtil.getPcapOutputPath(), "/foo");
   }
@@ -37,7 +36,6 @@ public class ConfigurationUtilTest {
    */
   @Test
   public void test_getTempQueryDir() {
-    Assert.assertEquals(ConfigurationUtil.getTempQueryOutputPath(), null);
     ConfigurationUtil.setTempQueryOutputPath("/tmp");
     Assert.assertEquals(ConfigurationUtil.getTempQueryOutputPath(), "/tmp");
   }

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/df8d682e/metron-platform/metron-common/src/main/java/org/apache/metron/common/Constants.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-common/src/main/java/org/apache/metron/common/Constants.java b/metron-platform/metron-common/src/main/java/org/apache/metron/common/Constants.java
index 4c7c222..60a5b51 100644
--- a/metron-platform/metron-common/src/main/java/org/apache/metron/common/Constants.java
+++ b/metron-platform/metron-common/src/main/java/org/apache/metron/common/Constants.java
@@ -19,12 +19,8 @@ package org.apache.metron.common;
 
 public class Constants {
 
-  public static final String GLOBAL_CONFIG_NAME = "global";
-  public static final String SENSORS_CONFIG_NAME = "sensors";
   public static final String ZOOKEEPER_ROOT = "/metron";
   public static final String ZOOKEEPER_TOPOLOGY_ROOT = ZOOKEEPER_ROOT + "/topology";
-  public static final String ZOOKEEPER_GLOBAL_ROOT = ZOOKEEPER_TOPOLOGY_ROOT + "/" + GLOBAL_CONFIG_NAME;
-  public static final String ZOOKEEPER_SENSOR_ROOT = ZOOKEEPER_TOPOLOGY_ROOT + "/" + SENSORS_CONFIG_NAME;
   public static final long DEFAULT_CONFIGURED_BOLT_TIMEOUT = 5000;
   public static final String SENSOR_TYPE = "source.type";
   public static final String ENRICHMENT_TOPIC = "enrichments";

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/df8d682e/metron-platform/metron-common/src/main/java/org/apache/metron/common/bolt/ConfiguredBolt.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-common/src/main/java/org/apache/metron/common/bolt/ConfiguredBolt.java b/metron-platform/metron-common/src/main/java/org/apache/metron/common/bolt/ConfiguredBolt.java
index 1364305..2d5e241 100644
--- a/metron-platform/metron-common/src/main/java/org/apache/metron/common/bolt/ConfiguredBolt.java
+++ b/metron-platform/metron-common/src/main/java/org/apache/metron/common/bolt/ConfiguredBolt.java
@@ -29,8 +29,7 @@ import org.apache.curator.framework.recipes.cache.TreeCacheListener;
 import org.apache.curator.retry.ExponentialBackoffRetry;
 import org.apache.log4j.Logger;
 import org.apache.metron.common.Constants;
-import org.apache.metron.common.configuration.Configurations;
-import org.apache.metron.common.configuration.ConfigurationsUtils;
+import org.apache.metron.common.configuration.ConfigurationType;
 
 import java.io.IOException;
 import java.util.Map;
@@ -41,7 +40,6 @@ public abstract class ConfiguredBolt extends BaseRichBolt {
 
   private String zookeeperUrl;
 
-  protected final Configurations configurations = new Configurations();
   protected CuratorFramework client;
   protected TreeCache cache;
 
@@ -49,10 +47,6 @@ public abstract class ConfiguredBolt extends BaseRichBolt {
     this.zookeeperUrl = zookeeperUrl;
   }
 
-  public Configurations getConfigurations() {
-    return configurations;
-  }
-
   public void setCuratorFramework(CuratorFramework client) {
     this.client = client;
   }
@@ -61,7 +55,7 @@ public abstract class ConfiguredBolt extends BaseRichBolt {
     this.cache = cache;
   }
 
-  public void reloadCallback(String name, Configurations.Type type) {
+  public void reloadCallback(String name, ConfigurationType type) {
   }
 
   @Override
@@ -85,11 +79,7 @@ public abstract class ConfiguredBolt extends BaseRichBolt {
           }
         };
         cache.getListenable().addListener(listener);
-        try {
-          ConfigurationsUtils.updateConfigsFromZookeeper(configurations, client);
-        } catch (Exception e) {
-          LOG.warn("Unable to load configs from zookeeper, but the cache should load lazily...");
-        }
+        loadConfig();
       }
       cache.start();
     } catch (Exception e) {
@@ -98,23 +88,8 @@ public abstract class ConfiguredBolt extends BaseRichBolt {
     }
   }
 
-  public void updateConfig(String path, byte[] data) throws IOException {
-    if (data.length != 0) {
-      String name = path.substring(path.lastIndexOf("/") + 1);
-      Configurations.Type type;
-      if (path.startsWith(Constants.ZOOKEEPER_SENSOR_ROOT)) {
-        configurations.updateSensorEnrichmentConfig(name, data);
-        type = Configurations.Type.SENSOR;
-      } else if (Constants.ZOOKEEPER_GLOBAL_ROOT.equals(path)) {
-        configurations.updateGlobalConfig(data);
-        type = Configurations.Type.GLOBAL;
-      } else {
-        configurations.updateConfig(name, data);
-        type = Configurations.Type.OTHER;
-      }
-      reloadCallback(name, type);
-    }
-  }
+  abstract public void loadConfig();
+  abstract public void updateConfig(String path, byte[] data) throws IOException;
 
   @Override
   public void cleanup() {

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/df8d682e/metron-platform/metron-common/src/main/java/org/apache/metron/common/bolt/ConfiguredEnrichmentBolt.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-common/src/main/java/org/apache/metron/common/bolt/ConfiguredEnrichmentBolt.java b/metron-platform/metron-common/src/main/java/org/apache/metron/common/bolt/ConfiguredEnrichmentBolt.java
new file mode 100644
index 0000000..e03e793
--- /dev/null
+++ b/metron-platform/metron-common/src/main/java/org/apache/metron/common/bolt/ConfiguredEnrichmentBolt.java
@@ -0,0 +1,64 @@
+/**
+ * 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.common.bolt;
+
+import org.apache.log4j.Logger;
+import org.apache.metron.common.Constants;
+import org.apache.metron.common.configuration.ConfigurationType;
+import org.apache.metron.common.configuration.ConfigurationsUtils;
+import org.apache.metron.common.configuration.EnrichmentConfigurations;
+
+import java.io.IOException;
+
+public abstract class ConfiguredEnrichmentBolt extends ConfiguredBolt {
+
+  private static final Logger LOG = Logger.getLogger(ConfiguredEnrichmentBolt.class);
+
+  protected final EnrichmentConfigurations configurations = new EnrichmentConfigurations();
+
+  public ConfiguredEnrichmentBolt(String zookeeperUrl) {
+    super(zookeeperUrl);
+  }
+
+  public EnrichmentConfigurations getConfigurations() {
+    return configurations;
+  }
+
+  @Override
+  public void loadConfig() {
+    try {
+      ConfigurationsUtils.updateEnrichmentConfigsFromZookeeper(configurations, client);
+    } catch (Exception e) {
+      LOG.warn("Unable to load configs from zookeeper, but the cache should load lazily...");
+    }
+  }
+
+  @Override
+  public void updateConfig(String path, byte[] data) throws IOException {
+    if (data.length != 0) {
+      String name = path.substring(path.lastIndexOf("/") + 1);
+      if (path.startsWith(ConfigurationType.ENRICHMENT.getZookeeperRoot())) {
+        configurations.updateSensorEnrichmentConfig(name, data);
+        reloadCallback(name, ConfigurationType.ENRICHMENT);
+      } else if (ConfigurationType.GLOBAL.getZookeeperRoot().equals(path)) {
+        configurations.updateGlobalConfig(data);
+        reloadCallback(name, ConfigurationType.GLOBAL);
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/df8d682e/metron-platform/metron-common/src/main/java/org/apache/metron/common/bolt/ConfiguredParserBolt.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-common/src/main/java/org/apache/metron/common/bolt/ConfiguredParserBolt.java b/metron-platform/metron-common/src/main/java/org/apache/metron/common/bolt/ConfiguredParserBolt.java
new file mode 100644
index 0000000..543f87b
--- /dev/null
+++ b/metron-platform/metron-common/src/main/java/org/apache/metron/common/bolt/ConfiguredParserBolt.java
@@ -0,0 +1,64 @@
+/**
+ * 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.common.bolt;
+
+import org.apache.log4j.Logger;
+import org.apache.metron.common.Constants;
+import org.apache.metron.common.configuration.ConfigurationType;
+import org.apache.metron.common.configuration.ConfigurationsUtils;
+import org.apache.metron.common.configuration.ParserConfigurations;
+
+import java.io.IOException;
+
+public abstract class ConfiguredParserBolt extends ConfiguredBolt {
+
+  private static final Logger LOG = Logger.getLogger(ConfiguredEnrichmentBolt.class);
+
+  protected final ParserConfigurations configurations = new ParserConfigurations();
+
+  public ConfiguredParserBolt(String zookeeperUrl) {
+    super(zookeeperUrl);
+  }
+
+  public ParserConfigurations getConfigurations() {
+    return configurations;
+  }
+
+  @Override
+  public void loadConfig() {
+    try {
+      ConfigurationsUtils.updateParserConfigsFromZookeeper(configurations, client);
+    } catch (Exception e) {
+      LOG.warn("Unable to load configs from zookeeper, but the cache should load lazily...");
+    }
+  }
+
+  @Override
+  public void updateConfig(String path, byte[] data) throws IOException {
+    if (data.length != 0) {
+      String name = path.substring(path.lastIndexOf("/") + 1);
+      if (path.startsWith(ConfigurationType.PARSER.getZookeeperRoot())) {
+        configurations.updateSensorParserConfig(name, data);
+        reloadCallback(name, ConfigurationType.PARSER);
+      } else if (ConfigurationType.GLOBAL.getZookeeperRoot().equals(path)) {
+        configurations.updateGlobalConfig(data);
+        reloadCallback(name, ConfigurationType.GLOBAL);
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/df8d682e/metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/Configuration.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/Configuration.java b/metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/Configuration.java
index e526ee4..ebb46e4 100644
--- a/metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/Configuration.java
+++ b/metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/Configuration.java
@@ -48,10 +48,6 @@ public class Configuration extends Configurations {
         } else {
 
             updateGlobalConfig(ConfigurationsUtils.readGlobalConfigFromFile(configFileRoot.toAbsolutePath().toString()));
-            Map<String, byte[]> sensorEnrichmentConfigs = ConfigurationsUtils.readSensorEnrichmentConfigsFromFile(configFileRoot.toAbsolutePath().toString());
-            for(String sensorType: sensorEnrichmentConfigs.keySet()) {
-                updateSensorEnrichmentConfig(sensorType, sensorEnrichmentConfigs.get(sensorType));
-            }
 
         }
 

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/df8d682e/metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/ConfigurationType.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/ConfigurationType.java b/metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/ConfigurationType.java
index 2b9f6cf..9469bf6 100644
--- a/metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/ConfigurationType.java
+++ b/metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/ConfigurationType.java
@@ -28,18 +28,27 @@ import java.io.IOException;
 import java.util.Map;
 
 public enum ConfigurationType implements Function<String, Object> {
-  GLOBAL("."
-        ,Constants.ZOOKEEPER_GLOBAL_ROOT
-        , s -> {
+  GLOBAL("global"
+          ,"."
+          , s -> {
     try {
       return JSONUtils.INSTANCE.load(s, new TypeReference<Map<String, Object>>() {
       });
     } catch (IOException e) {
       throw new RuntimeException("Unable to load " + s, e);
     }
-  })
-  , SENSOR(Constants.SENSORS_CONFIG_NAME
-          ,Constants.ZOOKEEPER_SENSOR_ROOT
+  }),
+  PARSER("parsers"
+          ,"parsers"
+          , s -> {
+    try {
+      return JSONUtils.INSTANCE.load(s, SensorParserConfig.class);
+    } catch (IOException e) {
+      throw new RuntimeException("Unable to load " + s, e);
+    }
+  }),
+  ENRICHMENT("enrichments"
+          ,"enrichments"
           , s -> {
     try {
       return JSONUtils.INSTANCE.load(s, SensorEnrichmentConfig.class);
@@ -47,15 +56,19 @@ public enum ConfigurationType implements Function<String, Object> {
       throw new RuntimeException("Unable to load " + s, e);
     }
   });
+  String name;
   String directory;
   String zookeeperRoot;
   Function<String,?> deserializer;
-  ConfigurationType(String directory, String zookeeperRoot, Function<String, ?> deserializer) {
+  ConfigurationType(String name, String directory, Function<String, ?> deserializer) {
+    this.name = name;
     this.directory = directory;
-    this.zookeeperRoot = zookeeperRoot;
+    this.zookeeperRoot = Constants.ZOOKEEPER_TOPOLOGY_ROOT + "/" + name;
     this.deserializer = deserializer;
   }
 
+  public String getName() { return name; }
+
   public String getDirectory() {
     return directory;
   }

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/df8d682e/metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/Configurations.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/Configurations.java b/metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/Configurations.java
index a152d40..f33ebd7 100644
--- a/metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/Configurations.java
+++ b/metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/Configurations.java
@@ -19,7 +19,6 @@ package org.apache.metron.common.configuration;
 
 import com.fasterxml.jackson.core.type.TypeReference;
 import org.apache.log4j.Logger;
-import org.apache.metron.common.configuration.enrichment.SensorEnrichmentConfig;
 import org.apache.metron.common.utils.JSONUtils;
 
 import java.io.ByteArrayInputStream;
@@ -34,20 +33,15 @@ public class Configurations implements Serializable {
 
   private static final Logger LOG = Logger.getLogger(Configurations.class);
 
-  public enum Type {
-    GLOBAL, SENSOR, OTHER
-  }
-
-  public static final String GLOBAL_CONFIG_NAME = "global";
-
-  private ConcurrentMap<String, Object> configurations = new ConcurrentHashMap<>();
+  protected ConcurrentMap<String, Object> configurations = new ConcurrentHashMap<>();
 
   @SuppressWarnings("unchecked")
   public Map<String, Object> getGlobalConfig() {
-    return (Map<String, Object>) configurations.get(GLOBAL_CONFIG_NAME);
+    return (Map<String, Object>) configurations.get(ConfigurationType.GLOBAL.getName());
   }
 
   public void updateGlobalConfig(byte[] data) throws IOException {
+    if (data == null) throw new IllegalStateException("global config data cannot be null");
     updateGlobalConfig(new ByteArrayInputStream(data));
   }
 
@@ -58,39 +52,7 @@ public class Configurations implements Serializable {
   }
 
   public void updateGlobalConfig(Map<String, Object> globalConfig) {
-    configurations.put(GLOBAL_CONFIG_NAME, globalConfig);
-  }
-
-  public SensorEnrichmentConfig getSensorEnrichmentConfig(String sensorType) {
-    return (SensorEnrichmentConfig) configurations.get(sensorType);
-  }
-
-  public void updateSensorEnrichmentConfig(String sensorType, byte[] data) throws IOException {
-    updateSensorEnrichmentConfig(sensorType, new ByteArrayInputStream(data));
-  }
-
-  public void updateSensorEnrichmentConfig(String sensorType, InputStream io) throws IOException {
-    SensorEnrichmentConfig sensorEnrichmentConfig = JSONUtils.INSTANCE.load(io, SensorEnrichmentConfig.class);
-    updateSensorEnrichmentConfig(sensorType, sensorEnrichmentConfig);
-  }
-
-  public void updateSensorEnrichmentConfig(String sensorType, SensorEnrichmentConfig sensorEnrichmentConfig) {
-    configurations.put(sensorType, sensorEnrichmentConfig);
-  }
-
-  @SuppressWarnings("unchecked")
-  public Map<String, Object> getConfig(String name) {
-    return (Map<String, Object>) configurations.get(name);
-  }
-
-  public void updateConfig(String name, byte[] data) throws IOException {
-    if (data == null) throw new IllegalStateException("config data cannot be null");
-    Map<String, Object> config = JSONUtils.INSTANCE.load(new ByteArrayInputStream(data), new TypeReference<Map<String, Object>>() {});
-    updateConfig(name, config);
-  }
-
-  public void updateConfig(String name, Map<String, Object> config) {
-    configurations.put(name, config);
+    configurations.put(ConfigurationType.GLOBAL.getName(), globalConfig);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/df8d682e/metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/ConfigurationsUtils.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/ConfigurationsUtils.java b/metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/ConfigurationsUtils.java
index 1aa2ca8..56ac9d9 100644
--- a/metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/ConfigurationsUtils.java
+++ b/metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/ConfigurationsUtils.java
@@ -27,11 +27,11 @@ import org.apache.metron.common.configuration.enrichment.SensorEnrichmentConfig;
 import org.apache.metron.common.utils.JSONUtils;
 import org.apache.zookeeper.KeeperException;
 
+import java.io.ByteArrayInputStream;
 import java.io.File;
 import java.io.IOException;
 import java.io.PrintStream;
 import java.nio.file.Files;
-import java.nio.file.Paths;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
@@ -69,6 +69,25 @@ public class ConfigurationsUtils {
     writeToZookeeper(ConfigurationType.GLOBAL.getZookeeperRoot(), globalConfig, client);
   }
 
+  public static void writeSensorParserConfigToZookeeper(String sensorType, SensorParserConfig sensorParserConfig, String zookeeperUrl) throws Exception {
+    writeSensorParserConfigToZookeeper(sensorType, JSONUtils.INSTANCE.toJSON(sensorParserConfig), zookeeperUrl);
+  }
+
+  public static void writeSensorParserConfigToZookeeper(String sensorType, byte[] configData, String zookeeperUrl) throws Exception {
+    CuratorFramework client = getClient(zookeeperUrl);
+    client.start();
+    try {
+      writeSensorParserConfigToZookeeper(sensorType, configData, client);
+    }
+    finally {
+      client.close();
+    }
+  }
+
+  public static void writeSensorParserConfigToZookeeper(String sensorType, byte[] configData, CuratorFramework client) throws Exception {
+    writeToZookeeper(ConfigurationType.PARSER.getZookeeperRoot() + "/" + sensorType, configData, client);
+  }
+
   public static void writeSensorEnrichmentConfigToZookeeper(String sensorType, SensorEnrichmentConfig sensorEnrichmentConfig, String zookeeperUrl) throws Exception {
     writeSensorEnrichmentConfigToZookeeper(sensorType, JSONUtils.INSTANCE.toJSON(sensorEnrichmentConfig), zookeeperUrl);
   }
@@ -85,8 +104,7 @@ public class ConfigurationsUtils {
   }
 
   public static void writeSensorEnrichmentConfigToZookeeper(String sensorType, byte[] configData, CuratorFramework client) throws Exception {
-    ConfigurationType.SENSOR.deserialize(new String(configData));
-    writeToZookeeper(ConfigurationType.SENSOR.getZookeeperRoot()+ "/" + sensorType, configData, client);
+    writeToZookeeper(ConfigurationType.ENRICHMENT.getZookeeperRoot() + "/" + sensorType, configData, client);
   }
 
   public static void writeConfigToZookeeper(String name, Map<String, Object> config, String zookeeperUrl) throws Exception {
@@ -114,18 +132,42 @@ public class ConfigurationsUtils {
 
   public static void updateConfigsFromZookeeper(Configurations configurations, CuratorFramework client) throws Exception {
     configurations.updateGlobalConfig(readGlobalConfigBytesFromZookeeper(client));
-    List<String> sensorTypes = client.getChildren().forPath(Constants.ZOOKEEPER_SENSOR_ROOT);
+  }
+
+  public static void updateParserConfigsFromZookeeper(ParserConfigurations configurations, CuratorFramework client) throws Exception {
+    updateConfigsFromZookeeper(configurations, client);
+    List<String> sensorTypes = client.getChildren().forPath(ConfigurationType.PARSER.getZookeeperRoot());
+    for(String sensorType: sensorTypes) {
+      configurations.updateSensorParserConfig(sensorType, readSensorParserConfigBytesFromZookeeper(sensorType, client));
+    }
+  }
+
+  public static void updateEnrichmentConfigsFromZookeeper(EnrichmentConfigurations configurations, CuratorFramework client) throws Exception {
+    updateConfigsFromZookeeper(configurations, client);
+    List<String> sensorTypes = client.getChildren().forPath(ConfigurationType.ENRICHMENT.getZookeeperRoot());
     for(String sensorType: sensorTypes) {
       configurations.updateSensorEnrichmentConfig(sensorType, readSensorEnrichmentConfigBytesFromZookeeper(sensorType, client));
     }
   }
 
+  public static SensorEnrichmentConfig readSensorEnrichmentConfigFromZookeeper(String sensorType, CuratorFramework client) throws Exception {
+    return JSONUtils.INSTANCE.load(new ByteArrayInputStream(readFromZookeeper(ConfigurationType.ENRICHMENT.getZookeeperRoot() + "/" + sensorType, client)), SensorEnrichmentConfig.class);
+  }
+
+  public static SensorParserConfig readSensorParserConfigFromZookeeper(String sensorType, CuratorFramework client) throws Exception {
+    return JSONUtils.INSTANCE.load(new ByteArrayInputStream(readFromZookeeper(ConfigurationType.PARSER.getZookeeperRoot() + "/" + sensorType, client)), SensorParserConfig.class);
+  }
+
   public static byte[] readGlobalConfigBytesFromZookeeper(CuratorFramework client) throws Exception {
-    return readFromZookeeper(Constants.ZOOKEEPER_GLOBAL_ROOT, client);
+    return readFromZookeeper(ConfigurationType.GLOBAL.getZookeeperRoot(), client);
+  }
+
+  public static byte[] readSensorParserConfigBytesFromZookeeper(String sensorType, CuratorFramework client) throws Exception {
+    return readFromZookeeper(ConfigurationType.PARSER.getZookeeperRoot() + "/" + sensorType, client);
   }
 
   public static byte[] readSensorEnrichmentConfigBytesFromZookeeper(String sensorType, CuratorFramework client) throws Exception {
-    return readFromZookeeper(Constants.ZOOKEEPER_SENSOR_ROOT + "/" + sensorType, client);
+    return readFromZookeeper(ConfigurationType.ENRICHMENT.getZookeeperRoot() + "/" + sensorType, client);
   }
 
   public static byte[] readConfigBytesFromZookeeper(String name, CuratorFramework client) throws Exception {
@@ -136,60 +178,99 @@ public class ConfigurationsUtils {
     return client.getData().forPath(path);
   }
 
+  public static void uploadConfigsToZookeeper(String globalConfigPath, String parsersConfigPath, String enrichmentsConfigPath, String zookeeperUrl) throws Exception {
+    try(CuratorFramework client = getClient(zookeeperUrl)) {
+      client.start();
+      uploadConfigsToZookeeper(globalConfigPath, parsersConfigPath, enrichmentsConfigPath, client);
+    }
+  }
+
   public static void uploadConfigsToZookeeper(String rootFilePath, CuratorFramework client) throws Exception {
-    ConfigurationsUtils.writeGlobalConfigToZookeeper(readGlobalConfigFromFile(rootFilePath), client);
-    Map<String, byte[]> sensorEnrichmentConfigs = readSensorEnrichmentConfigsFromFile(rootFilePath);
-    for(String sensorType: sensorEnrichmentConfigs.keySet()) {
-      ConfigurationsUtils.writeSensorEnrichmentConfigToZookeeper(sensorType, sensorEnrichmentConfigs.get(sensorType), client);
+    uploadConfigsToZookeeper(rootFilePath, rootFilePath, rootFilePath, client);
+  }
+
+  public static void uploadConfigsToZookeeper(String globalConfigPath, String parsersConfigPath, String enrichmentsConfigPath, CuratorFramework client) throws Exception {
+    if (globalConfigPath != null) {
+      byte[] globalConfig = readGlobalConfigFromFile(globalConfigPath);
+      if (globalConfig.length > 0) {
+        ConfigurationsUtils.writeGlobalConfigToZookeeper(readGlobalConfigFromFile(globalConfigPath), client);
+      }
+    }
+    if (parsersConfigPath != null) {
+      Map<String, byte[]> sensorParserConfigs = readSensorParserConfigsFromFile(parsersConfigPath);
+      for (String sensorType : sensorParserConfigs.keySet()) {
+        ConfigurationsUtils.writeSensorParserConfigToZookeeper(sensorType, sensorParserConfigs.get(sensorType), client);
+      }
+    }
+    if (enrichmentsConfigPath != null) {
+      Map<String, byte[]> sensorEnrichmentConfigs = readSensorEnrichmentConfigsFromFile(enrichmentsConfigPath);
+      for (String sensorType : sensorEnrichmentConfigs.keySet()) {
+        ConfigurationsUtils.writeSensorEnrichmentConfigToZookeeper(sensorType, sensorEnrichmentConfigs.get(sensorType), client);
+      }
     }
   }
 
-  public static void uploadConfigsToZookeeper(String rootFilePath, String zookeeperUrl) throws Exception {
-    try(CuratorFramework client = getClient(zookeeperUrl)) {
-      client.start();
-      uploadConfigsToZookeeper(rootFilePath, client);
+  public static byte[] readGlobalConfigFromFile(String rootPath) throws IOException {
+    byte[] globalConfig = new byte[0];
+    File configPath = new File(rootPath, ConfigurationType.GLOBAL.getName() + ".json");
+    if (configPath.exists()) {
+      globalConfig = Files.readAllBytes(configPath.toPath());
     }
+    return globalConfig;
   }
 
-  public static byte[] readGlobalConfigFromFile(String rootFilePath) throws IOException {
-    return Files.readAllBytes(Paths.get(rootFilePath, Constants.GLOBAL_CONFIG_NAME + ".json"));
+  public static Map<String, byte[]> readSensorParserConfigsFromFile(String rootPath) throws IOException {
+    return readSensorConfigsFromFile(rootPath, ConfigurationType.PARSER);
   }
 
   public static Map<String, byte[]> readSensorEnrichmentConfigsFromFile(String rootPath) throws IOException {
-    Map<String, byte[]> sensorEnrichmentConfigs = new HashMap<>();
-    for(File file: new File(rootPath, Constants.SENSORS_CONFIG_NAME).listFiles()) {
-      if(file.getName().endsWith(".json")) {
-        sensorEnrichmentConfigs.put(FilenameUtils.removeExtension(file.getName()), Files.readAllBytes(file.toPath()));
+    return readSensorConfigsFromFile(rootPath, ConfigurationType.ENRICHMENT);
+  }
+
+  public static Map<String, byte[]> readSensorConfigsFromFile(String rootPath, ConfigurationType configType) throws IOException {
+    Map<String, byte[]> sensorConfigs = new HashMap<>();
+    File configPath = new File(rootPath, configType.getDirectory());
+    if (configPath.exists()) {
+      File[] children = configPath.listFiles();
+      if (children != null) {
+        for (File file : children) {
+          sensorConfigs.put(FilenameUtils.removeExtension(file.getName()), Files.readAllBytes(file.toPath()));
+        }
       }
     }
-    return sensorEnrichmentConfigs;
+    return sensorConfigs;
   }
 
+
   public interface ConfigurationVisitor{
     void visit(ConfigurationType configurationType, String name, String data);
   }
+
   public static void visitConfigs(CuratorFramework client, ConfigurationVisitor callback) throws Exception {
-    //Output global configs
-    {
-      ConfigurationType configType = ConfigurationType.GLOBAL;
-      byte[] globalConfigData = client.getData().forPath(configType.getZookeeperRoot());
-      callback.visit(configType, "global", new String(globalConfigData));
-    }
-    //Output sensor specific configs
-    {
-      ConfigurationType configType = ConfigurationType.SENSOR;
-      List<String> children = client.getChildren().forPath(configType.getZookeeperRoot());
-      for (String child : children) {
-        byte[] data = client.getData().forPath(configType.getZookeeperRoot() + "/" + child);
-        callback.visit(configType, child, new String(data));
+    visitConfigs(client, callback, ConfigurationType.GLOBAL);
+    visitConfigs(client, callback, ConfigurationType.PARSER);
+    visitConfigs(client, callback, ConfigurationType.ENRICHMENT);
+  }
+
+  public static void visitConfigs(CuratorFramework client, ConfigurationVisitor callback, ConfigurationType configType) throws Exception {
+    if (client.checkExists().forPath(configType.getZookeeperRoot()) != null) {
+      if (configType.equals(ConfigurationType.GLOBAL)) {
+        byte[] globalConfigData = client.getData().forPath(configType.getZookeeperRoot());
+        callback.visit(configType, "global", new String(globalConfigData));
+      } else if (configType.equals(ConfigurationType.PARSER) || configType.equals(ConfigurationType.ENRICHMENT)) {
+        List<String> children = client.getChildren().forPath(configType.getZookeeperRoot());
+        for (String child : children) {
+          byte[] data = client.getData().forPath(configType.getZookeeperRoot() + "/" + child);
+          callback.visit(configType, child, new String(data));
+        }
       }
     }
   }
+
   public static void dumpConfigs(PrintStream out, CuratorFramework client) throws Exception {
     ConfigurationsUtils.visitConfigs(client, (type, name, data) -> {
       type.deserialize(data);
       out.println(type + " Config: " + name + "\n" + data);
     });
   }
-
 }

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/df8d682e/metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/EnrichmentConfigurations.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/EnrichmentConfigurations.java b/metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/EnrichmentConfigurations.java
new file mode 100644
index 0000000..bf5b856
--- /dev/null
+++ b/metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/EnrichmentConfigurations.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.common.configuration;
+
+import org.apache.metron.common.configuration.enrichment.SensorEnrichmentConfig;
+import org.apache.metron.common.utils.JSONUtils;
+
+import java.io.ByteArrayInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+
+public class EnrichmentConfigurations extends Configurations {
+
+    public SensorEnrichmentConfig getSensorEnrichmentConfig(String sensorType) {
+        return (SensorEnrichmentConfig) configurations.get(getKey(sensorType));
+    }
+
+    public void updateSensorEnrichmentConfig(String sensorType, byte[] data) throws IOException {
+        updateSensorEnrichmentConfig(sensorType, new ByteArrayInputStream(data));
+    }
+
+    public void updateSensorEnrichmentConfig(String sensorType, InputStream io) throws IOException {
+        SensorEnrichmentConfig sensorEnrichmentConfig = JSONUtils.INSTANCE.load(io, SensorEnrichmentConfig.class);
+        updateSensorEnrichmentConfig(sensorType, sensorEnrichmentConfig);
+    }
+
+    public void updateSensorEnrichmentConfig(String sensorType, SensorEnrichmentConfig sensorEnrichmentConfig) {
+        configurations.put(getKey(sensorType), sensorEnrichmentConfig);
+    }
+
+    private String getKey(String sensorType) {
+        return ConfigurationType.ENRICHMENT.getName() + "." + sensorType;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/df8d682e/metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/ParserConfigurations.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/ParserConfigurations.java b/metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/ParserConfigurations.java
new file mode 100644
index 0000000..72630d9
--- /dev/null
+++ b/metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/ParserConfigurations.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.common.configuration;
+
+import org.apache.metron.common.utils.JSONUtils;
+
+import java.io.ByteArrayInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+
+public class ParserConfigurations extends Configurations {
+
+  public SensorParserConfig getSensorParserConfig(String sensorType) {
+    return (SensorParserConfig) configurations.get(getKey(sensorType));
+  }
+
+  public void updateSensorParserConfig(String sensorType, byte[] data) throws IOException {
+    updateSensorParserConfig(sensorType, new ByteArrayInputStream(data));
+  }
+
+  public void updateSensorParserConfig(String sensorType, InputStream io) throws IOException {
+    SensorParserConfig sensorParserConfig = JSONUtils.INSTANCE.load(io, SensorParserConfig.class);
+    updateSensorParserConfig(sensorType, sensorParserConfig);
+  }
+
+  public void updateSensorParserConfig(String sensorType, SensorParserConfig sensorParserConfig) {
+    configurations.put(getKey(sensorType), sensorParserConfig);
+  }
+
+  private String getKey(String sensorType) {
+    return ConfigurationType.PARSER.getName() + "." + sensorType;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/df8d682e/metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/SensorParserConfig.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/SensorParserConfig.java b/metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/SensorParserConfig.java
new file mode 100644
index 0000000..8cf1901
--- /dev/null
+++ b/metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/SensorParserConfig.java
@@ -0,0 +1,89 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.metron.common.configuration;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import org.apache.metron.common.utils.JSONUtils;
+
+import java.io.IOException;
+import java.util.Map;
+
+public class SensorParserConfig {
+
+  private String parserClassName;
+  private String sensorTopic;
+  private Map<String, Object> parserConfig;
+
+  public String getParserClassName() {
+    return parserClassName;
+  }
+
+  public void setParserClassName(String parserClassName) {
+    this.parserClassName = parserClassName;
+  }
+
+  public String getSensorTopic() {
+    return sensorTopic;
+  }
+
+  public void setSensorTopic(String sensorTopic) {
+    this.sensorTopic = sensorTopic;
+  }
+
+  public Map<String, Object> getParserConfig() {
+    return parserConfig;
+  }
+
+  public void setParserConfig(Map<String, Object> parserConfig) {
+    this.parserConfig = parserConfig;
+  }
+
+  public static SensorParserConfig fromBytes(byte[] config) throws IOException {
+    return JSONUtils.INSTANCE.load(new String(config), SensorParserConfig.class);
+  }
+
+  public String toJSON() throws JsonProcessingException {
+    return JSONUtils.INSTANCE.toJSON(this, true);
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) return true;
+    if (o == null || getClass() != o.getClass()) return false;
+
+    SensorParserConfig that = (SensorParserConfig) o;
+
+    if (getParserClassName() != null ? !getParserClassName().equals(that.getParserClassName()) : that.getParserClassName() != null) return false;
+    if (getSensorTopic() != null ? !getSensorTopic().equals(that.getSensorTopic()) : that.getSensorTopic() != null) return false;
+    return getParserConfig() != null ? getParserConfig().equals(that.getParserConfig()) : that.getParserConfig() == null;
+  }
+
+  @Override
+  public String toString() {
+    return "{parserClassName=" + parserClassName + ", sensorTopic=" + sensorTopic +
+            ", parserConfig=" + parserConfig + "}";
+  }
+
+  @Override
+  public int hashCode() {
+    int result = getParserClassName() != null ? getParserClassName().hashCode() : 0;
+    result = 31 * result + (getSensorTopic() != null ? getSensorTopic().hashCode() : 0);
+    result = 31 * result + (getParserConfig() != null ? getParserConfig().hashCode() : 0);
+    return result;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/df8d682e/metron-platform/metron-common/src/main/java/org/apache/metron/common/interfaces/BulkMessageWriter.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-common/src/main/java/org/apache/metron/common/interfaces/BulkMessageWriter.java b/metron-platform/metron-common/src/main/java/org/apache/metron/common/interfaces/BulkMessageWriter.java
index 6fb3d78..aaa6c51 100644
--- a/metron-platform/metron-common/src/main/java/org/apache/metron/common/interfaces/BulkMessageWriter.java
+++ b/metron-platform/metron-common/src/main/java/org/apache/metron/common/interfaces/BulkMessageWriter.java
@@ -18,14 +18,14 @@
 package org.apache.metron.common.interfaces;
 
 import backtype.storm.tuple.Tuple;
-import org.apache.metron.common.configuration.Configurations;
+import org.apache.metron.common.configuration.EnrichmentConfigurations;
 
 import java.util.List;
 import java.util.Map;
 
 public interface BulkMessageWriter<T> extends AutoCloseable {
 
-  void init(Map stormConf, Configurations configuration) throws Exception;
-  void write(String sensorType, Configurations configurations, List<Tuple> tuples, List<T> messages) throws Exception;
+  void init(Map stormConf, EnrichmentConfigurations configuration) throws Exception;
+  void write(String sensorType, EnrichmentConfigurations configurations, List<Tuple> tuples, List<T> messages) throws Exception;
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/df8d682e/metron-platform/metron-common/src/main/java/org/apache/metron/common/spout/kafka/SpoutConfig.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-common/src/main/java/org/apache/metron/common/spout/kafka/SpoutConfig.java b/metron-platform/metron-common/src/main/java/org/apache/metron/common/spout/kafka/SpoutConfig.java
index 97085b6..8f0a6af 100644
--- a/metron-platform/metron-common/src/main/java/org/apache/metron/common/spout/kafka/SpoutConfig.java
+++ b/metron-platform/metron-common/src/main/java/org/apache/metron/common/spout/kafka/SpoutConfig.java
@@ -21,7 +21,7 @@ package org.apache.metron.common.spout.kafka;
 import storm.kafka.BrokerHosts;
 
 public class SpoutConfig extends storm.kafka.SpoutConfig {
-  private static enum Offset {
+  public static enum Offset {
     BEGINNING, END, WHERE_I_LEFT_OFF;
   }
   public SpoutConfig(BrokerHosts hosts, String topic, String zkRoot, String id) {
@@ -40,7 +40,8 @@ public class SpoutConfig extends storm.kafka.SpoutConfig {
     startOffsetTime = kafka.api.OffsetRequest.EarliestTime();
     return this;
   }
-  private void from(Offset offset) {
+
+  public SpoutConfig from(Offset offset) {
     if(offset == Offset.BEGINNING) {
       ignoreZkOffsets = true;
       startOffsetTime = kafka.api.OffsetRequest.EarliestTime();
@@ -53,5 +54,6 @@ public class SpoutConfig extends storm.kafka.SpoutConfig {
       ignoreZkOffsets = false;
       startOffsetTime = kafka.api.OffsetRequest.LatestTime();
     }
+    return this;
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/df8d682e/metron-platform/metron-common/src/main/java/org/apache/metron/common/utils/ReflectionUtils.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-common/src/main/java/org/apache/metron/common/utils/ReflectionUtils.java b/metron-platform/metron-common/src/main/java/org/apache/metron/common/utils/ReflectionUtils.java
index 2afa097..141221d 100644
--- a/metron-platform/metron-common/src/main/java/org/apache/metron/common/utils/ReflectionUtils.java
+++ b/metron-platform/metron-common/src/main/java/org/apache/metron/common/utils/ReflectionUtils.java
@@ -27,20 +27,26 @@ public class ReflectionUtils<T> {
       return defaultClass;
     }
     else {
-      try {
-        Class<? extends T> clazz = (Class<? extends T>) Class.forName(className);
-        instance = clazz.getConstructor().newInstance();
-      } catch (InstantiationException e) {
-        throw new IllegalStateException("Unable to instantiate connector.", e);
-      } catch (IllegalAccessException e) {
-        throw new IllegalStateException("Unable to instantiate connector: illegal access", e);
-      } catch (InvocationTargetException e) {
-        throw new IllegalStateException("Unable to instantiate connector", e);
-      } catch (NoSuchMethodException e) {
-        throw new IllegalStateException("Unable to instantiate connector: no such method", e);
-      } catch (ClassNotFoundException e) {
-        throw new IllegalStateException("Unable to instantiate connector: class not found", e);
-      }
+      instance = createInstance(className);
+    }
+    return instance;
+  }
+
+  public static <T> T createInstance(String className) {
+    T instance;
+    try {
+      Class<? extends T> clazz = (Class<? extends T>) Class.forName(className);
+      instance = clazz.getConstructor().newInstance();
+    } catch (InstantiationException e) {
+      throw new IllegalStateException("Unable to instantiate connector.", e);
+    } catch (IllegalAccessException e) {
+      throw new IllegalStateException("Unable to instantiate connector: illegal access", e);
+    } catch (InvocationTargetException e) {
+      throw new IllegalStateException("Unable to instantiate connector", e);
+    } catch (NoSuchMethodException e) {
+      throw new IllegalStateException("Unable to instantiate connector: no such method", e);
+    } catch (ClassNotFoundException e) {
+      throw new IllegalStateException("Unable to instantiate connector: class not found", e);
     }
     return instance;
   }

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/df8d682e/metron-platform/metron-common/src/test/java/org/apache/metron/common/bolt/BaseConfiguredBoltTest.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-common/src/test/java/org/apache/metron/common/bolt/BaseConfiguredBoltTest.java b/metron-platform/metron-common/src/test/java/org/apache/metron/common/bolt/BaseConfiguredBoltTest.java
new file mode 100644
index 0000000..f9901cd
--- /dev/null
+++ b/metron-platform/metron-common/src/test/java/org/apache/metron/common/bolt/BaseConfiguredBoltTest.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.common.bolt;
+
+import org.apache.metron.test.bolt.BaseBoltTest;
+import org.junit.Assert;
+
+import java.util.HashSet;
+import java.util.Set;
+
+public class BaseConfiguredBoltTest extends BaseBoltTest {
+
+  protected static Set<String> configsUpdated = new HashSet<>();
+
+  protected void waitForConfigUpdate(final String expectedConfigUpdate) {
+    waitForConfigUpdate(new HashSet<String>() {{ add(expectedConfigUpdate); }});
+  }
+
+  protected void waitForConfigUpdate(Set<String> expectedConfigUpdates) {
+    int count = 0;
+    while (!configsUpdated.equals(expectedConfigUpdates)) {
+      if (count++ > 5) {
+        Assert.fail("ConfiguredBolt was not updated in time");
+        return;
+      }
+      try {
+        Thread.sleep(500);
+      } catch (InterruptedException e) {
+        e.printStackTrace();
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/df8d682e/metron-platform/metron-common/src/test/java/org/apache/metron/common/bolt/ConfiguredBoltTest.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-common/src/test/java/org/apache/metron/common/bolt/ConfiguredBoltTest.java b/metron-platform/metron-common/src/test/java/org/apache/metron/common/bolt/ConfiguredBoltTest.java
deleted file mode 100644
index 6df930b..0000000
--- a/metron-platform/metron-common/src/test/java/org/apache/metron/common/bolt/ConfiguredBoltTest.java
+++ /dev/null
@@ -1,162 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.metron.common.bolt;
-
-import backtype.storm.topology.OutputFieldsDeclarer;
-import backtype.storm.tuple.Tuple;
-import org.apache.curator.test.TestingServer;
-import org.apache.metron.common.Constants;
-import org.apache.metron.TestConstants;
-import org.apache.metron.test.bolt.BaseEnrichmentBoltTest;
-import org.apache.metron.common.configuration.Configurations;
-import org.apache.metron.common.configuration.enrichment.SensorEnrichmentConfig;
-import org.apache.metron.common.configuration.ConfigurationsUtils;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Test;
-
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-
-public class ConfiguredBoltTest extends BaseEnrichmentBoltTest {
-  private static Set<String> configsUpdated = new HashSet<>();
-  private Set<String> allConfigurationTypes = new HashSet<>();
-  private String zookeeperUrl;
-
-  public static class StandAloneConfiguredBolt extends ConfiguredBolt {
-
-    public StandAloneConfiguredBolt(String zookeeperUrl) {
-      super(zookeeperUrl);
-    }
-
-    @Override
-    public void execute(Tuple input) {
-    }
-
-    @Override
-    public void declareOutputFields(OutputFieldsDeclarer declarer) {
-    }
-
-    @Override
-    public void reloadCallback(String name, Configurations.Type type) {
-      configsUpdated.add(name);
-    }
-  }
-
-  @Before
-  public void setupConfiguration() throws Exception {
-    TestingServer testZkServer = new TestingServer(true);
-    this.zookeeperUrl = testZkServer.getConnectString();
-    byte[] globalConfig = ConfigurationsUtils.readGlobalConfigFromFile(TestConstants.SAMPLE_CONFIG_PATH);
-    ConfigurationsUtils.writeGlobalConfigToZookeeper(globalConfig, zookeeperUrl);
-    allConfigurationTypes.add(Constants.GLOBAL_CONFIG_NAME);
-    Map<String, byte[]> sensorEnrichmentConfigs = ConfigurationsUtils.readSensorEnrichmentConfigsFromFile(TestConstants.SAMPLE_CONFIG_PATH);
-    for (String sensorType : sensorEnrichmentConfigs.keySet()) {
-      ConfigurationsUtils.writeSensorEnrichmentConfigToZookeeper(sensorType, sensorEnrichmentConfigs.get(sensorType), zookeeperUrl);
-      allConfigurationTypes.add(sensorType);
-    }
-  }
-
-  @Test
-  public void test() throws Exception {
-    Configurations sampleConfigurations = new Configurations();
-    try {
-      StandAloneConfiguredBolt configuredBolt = new StandAloneConfiguredBolt(null);
-      configuredBolt.prepare(new HashMap(), topologyContext, outputCollector);
-      Assert.fail("A valid zookeeper url must be supplied");
-    } catch (RuntimeException e){}
-
-    configsUpdated = new HashSet<>();
-    sampleConfigurations.updateGlobalConfig(ConfigurationsUtils.readGlobalConfigFromFile(TestConstants.SAMPLE_CONFIG_PATH));
-    Map<String, byte[]> sensorEnrichmentConfigs = ConfigurationsUtils.readSensorEnrichmentConfigsFromFile(TestConstants.SAMPLE_CONFIG_PATH);
-    for (String sensorType : sensorEnrichmentConfigs.keySet()) {
-      sampleConfigurations.updateSensorEnrichmentConfig(sensorType, sensorEnrichmentConfigs.get(sensorType));
-    }
-
-    StandAloneConfiguredBolt configuredBolt = new StandAloneConfiguredBolt(zookeeperUrl);
-    configuredBolt.prepare(new HashMap(), topologyContext, outputCollector);
-    waitForConfigUpdate(allConfigurationTypes);
-    Assert.assertEquals(sampleConfigurations, configuredBolt.configurations);
-
-    configsUpdated = new HashSet<>();
-    Map<String, Object> sampleGlobalConfig = sampleConfigurations.getGlobalConfig();
-    sampleGlobalConfig.put("newGlobalField", "newGlobalValue");
-    ConfigurationsUtils.writeGlobalConfigToZookeeper(sampleGlobalConfig, zookeeperUrl);
-    waitForConfigUpdate(Constants.GLOBAL_CONFIG_NAME);
-    Assert.assertEquals("Add global config field", sampleConfigurations.getGlobalConfig(), configuredBolt.configurations.getGlobalConfig());
-
-    configsUpdated = new HashSet<>();
-    sampleGlobalConfig.remove("newGlobalField");
-    ConfigurationsUtils.writeGlobalConfigToZookeeper(sampleGlobalConfig, zookeeperUrl);
-    waitForConfigUpdate(Constants.GLOBAL_CONFIG_NAME);
-    Assert.assertEquals("Remove global config field", sampleConfigurations, configuredBolt.configurations);
-
-    configsUpdated = new HashSet<>();
-    String sensorType = "testSensorConfig";
-    SensorEnrichmentConfig testSensorConfig = new SensorEnrichmentConfig();
-    testSensorConfig.setBatchSize(50);
-    testSensorConfig.setIndex("test");
-    Map<String, List<String>> enrichmentFieldMap = new HashMap<>();
-    enrichmentFieldMap.put("enrichmentTest", new ArrayList<String>() {{
-      add("enrichmentField");
-    }});
-    testSensorConfig.getEnrichment().setFieldMap(enrichmentFieldMap);
-    Map<String, List<String>> threatIntelFieldMap = new HashMap<>();
-    threatIntelFieldMap.put("threatIntelTest", new ArrayList<String>() {{
-      add("threatIntelField");
-    }});
-    testSensorConfig.getThreatIntel().setFieldMap(threatIntelFieldMap);
-    sampleConfigurations.updateSensorEnrichmentConfig(sensorType, testSensorConfig);
-    ConfigurationsUtils.writeSensorEnrichmentConfigToZookeeper(sensorType, testSensorConfig, zookeeperUrl);
-    waitForConfigUpdate(sensorType);
-    Assert.assertEquals("Add new sensor config", sampleConfigurations, configuredBolt.configurations);
-
-    configsUpdated = new HashSet<>();
-    String someConfigType = "someConfig";
-    Map<String, Object> someConfig = new HashMap<>();
-    someConfig.put("someField", "someValue");
-    sampleConfigurations.updateConfig(someConfigType, someConfig);
-    ConfigurationsUtils.writeConfigToZookeeper(someConfigType, someConfig, zookeeperUrl);
-    waitForConfigUpdate(someConfigType);
-    Assert.assertEquals("Add new misc config", sampleConfigurations, configuredBolt.configurations);
-    configuredBolt.cleanup();
-  }
-
-  private void waitForConfigUpdate(final String expectedConfigUpdate) {
-    waitForConfigUpdate(new HashSet<String>() {{ add(expectedConfigUpdate); }});
-  }
-
-  private void waitForConfigUpdate(Set<String> expectedConfigUpdates) {
-    int count = 0;
-    while (!configsUpdated.equals(expectedConfigUpdates)) {
-      if (count++ > 5) {
-        Assert.fail("ConfiguredBolt was not updated in time");
-        return;
-      }
-      try {
-        Thread.sleep(500);
-      } catch (InterruptedException e) {
-        e.printStackTrace();
-      }
-    }
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/df8d682e/metron-platform/metron-common/src/test/java/org/apache/metron/common/bolt/ConfiguredEnrichmentBoltTest.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-common/src/test/java/org/apache/metron/common/bolt/ConfiguredEnrichmentBoltTest.java b/metron-platform/metron-common/src/test/java/org/apache/metron/common/bolt/ConfiguredEnrichmentBoltTest.java
new file mode 100644
index 0000000..c5f2304
--- /dev/null
+++ b/metron-platform/metron-common/src/test/java/org/apache/metron/common/bolt/ConfiguredEnrichmentBoltTest.java
@@ -0,0 +1,137 @@
+/**
+ * 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.common.bolt;
+
+import backtype.storm.topology.OutputFieldsDeclarer;
+import backtype.storm.tuple.Tuple;
+import org.apache.curator.test.TestingServer;
+import org.apache.metron.common.Constants;
+import org.apache.metron.TestConstants;
+import org.apache.metron.common.configuration.*;
+import org.apache.metron.test.bolt.BaseEnrichmentBoltTest;
+import org.apache.metron.common.configuration.enrichment.SensorEnrichmentConfig;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+public class ConfiguredEnrichmentBoltTest extends BaseConfiguredBoltTest {
+
+  private Set<String> enrichmentConfigurationTypes = new HashSet<>();
+  private String zookeeperUrl;
+
+  public static class StandAloneConfiguredEnrichmentBolt extends ConfiguredEnrichmentBolt {
+
+    public StandAloneConfiguredEnrichmentBolt(String zookeeperUrl) {
+      super(zookeeperUrl);
+    }
+
+    @Override
+    public void execute(Tuple input) {
+    }
+
+    @Override
+    public void declareOutputFields(OutputFieldsDeclarer declarer) {
+    }
+
+    @Override
+    public void reloadCallback(String name, ConfigurationType type) {
+      configsUpdated.add(name);
+    }
+  }
+
+  @Before
+  public void setupConfiguration() throws Exception {
+    TestingServer testZkServer = new TestingServer(true);
+    this.zookeeperUrl = testZkServer.getConnectString();
+    byte[] globalConfig = ConfigurationsUtils.readGlobalConfigFromFile(TestConstants.SAMPLE_CONFIG_PATH);
+    ConfigurationsUtils.writeGlobalConfigToZookeeper(globalConfig, zookeeperUrl);
+    enrichmentConfigurationTypes.add(ConfigurationType.GLOBAL.getName());
+    Map<String, byte[]> sensorEnrichmentConfigs = ConfigurationsUtils.readSensorEnrichmentConfigsFromFile(TestConstants.ENRICHMENTS_CONFIGS_PATH);
+    for (String sensorType : sensorEnrichmentConfigs.keySet()) {
+      ConfigurationsUtils.writeSensorEnrichmentConfigToZookeeper(sensorType, sensorEnrichmentConfigs.get(sensorType), zookeeperUrl);
+      enrichmentConfigurationTypes.add(sensorType);
+    }
+    Map<String, byte[]> sensorParserConfigs = ConfigurationsUtils.readSensorParserConfigsFromFile(TestConstants.PARSER_CONFIGS_PATH);
+    for (String sensorType : sensorParserConfigs.keySet()) {
+      ConfigurationsUtils.writeSensorParserConfigToZookeeper(sensorType, sensorParserConfigs.get(sensorType), zookeeperUrl);
+    }
+  }
+
+  @Test
+  public void test() throws Exception {
+    EnrichmentConfigurations sampleConfigurations = new EnrichmentConfigurations();
+    try {
+      StandAloneConfiguredEnrichmentBolt configuredBolt = new StandAloneConfiguredEnrichmentBolt(null);
+      configuredBolt.prepare(new HashMap(), topologyContext, outputCollector);
+      Assert.fail("A valid zookeeper url must be supplied");
+    } catch (RuntimeException e){}
+
+    configsUpdated = new HashSet<>();
+    sampleConfigurations.updateGlobalConfig(ConfigurationsUtils.readGlobalConfigFromFile(TestConstants.SAMPLE_CONFIG_PATH));
+    Map<String, byte[]> sensorEnrichmentConfigs = ConfigurationsUtils.readSensorEnrichmentConfigsFromFile(TestConstants.ENRICHMENTS_CONFIGS_PATH);
+    for (String sensorType : sensorEnrichmentConfigs.keySet()) {
+      sampleConfigurations.updateSensorEnrichmentConfig(sensorType, sensorEnrichmentConfigs.get(sensorType));
+    }
+
+    StandAloneConfiguredEnrichmentBolt configuredBolt = new StandAloneConfiguredEnrichmentBolt(zookeeperUrl);
+    configuredBolt.prepare(new HashMap(), topologyContext, outputCollector);
+    waitForConfigUpdate(enrichmentConfigurationTypes);
+    Assert.assertEquals(sampleConfigurations, configuredBolt.configurations);
+
+    configsUpdated = new HashSet<>();
+    Map<String, Object> sampleGlobalConfig = sampleConfigurations.getGlobalConfig();
+    sampleGlobalConfig.put("newGlobalField", "newGlobalValue");
+    ConfigurationsUtils.writeGlobalConfigToZookeeper(sampleGlobalConfig, zookeeperUrl);
+    waitForConfigUpdate(ConfigurationType.GLOBAL.getName());
+    Assert.assertEquals("Add global config field", sampleConfigurations.getGlobalConfig(), configuredBolt.configurations.getGlobalConfig());
+
+    configsUpdated = new HashSet<>();
+    sampleGlobalConfig.remove("newGlobalField");
+    ConfigurationsUtils.writeGlobalConfigToZookeeper(sampleGlobalConfig, zookeeperUrl);
+    waitForConfigUpdate(ConfigurationType.GLOBAL.getName());
+    Assert.assertEquals("Remove global config field", sampleConfigurations, configuredBolt.configurations);
+
+    configsUpdated = new HashSet<>();
+    String sensorType = "testSensorConfig";
+    SensorEnrichmentConfig testSensorConfig = new SensorEnrichmentConfig();
+    testSensorConfig.setBatchSize(50);
+    testSensorConfig.setIndex("test");
+    Map<String, List<String>> enrichmentFieldMap = new HashMap<>();
+    enrichmentFieldMap.put("enrichmentTest", new ArrayList<String>() {{
+      add("enrichmentField");
+    }});
+    testSensorConfig.getEnrichment().setFieldMap(enrichmentFieldMap);
+    Map<String, List<String>> threatIntelFieldMap = new HashMap<>();
+    threatIntelFieldMap.put("threatIntelTest", new ArrayList<String>() {{
+      add("threatIntelField");
+    }});
+    testSensorConfig.getThreatIntel().setFieldMap(threatIntelFieldMap);
+    sampleConfigurations.updateSensorEnrichmentConfig(sensorType, testSensorConfig);
+    ConfigurationsUtils.writeSensorEnrichmentConfigToZookeeper(sensorType, testSensorConfig, zookeeperUrl);
+    waitForConfigUpdate(sensorType);
+    Assert.assertEquals("Add new sensor config", sampleConfigurations, configuredBolt.configurations);
+    configuredBolt.cleanup();
+  }
+}
\ No newline at end of file