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/04/05 21:41:57 UTC

[01/15] incubator-metron git commit: METRON 86: Adding Solr indexing support (merrimanr via cestella) closes apache/incubator-metron#67

Repository: incubator-metron
Updated Branches:
  refs/heads/master edeec014c -> e59b1a31d


http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/e59b1a31/metron-streaming/Metron-Topologies/src/test/java/org/apache/metron/integration/EnrichmentIntegrationTest.java
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-Topologies/src/test/java/org/apache/metron/integration/EnrichmentIntegrationTest.java b/metron-streaming/Metron-Topologies/src/test/java/org/apache/metron/integration/EnrichmentIntegrationTest.java
deleted file mode 100644
index 1aa729c..0000000
--- a/metron-streaming/Metron-Topologies/src/test/java/org/apache/metron/integration/EnrichmentIntegrationTest.java
+++ /dev/null
@@ -1,469 +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
- * <p/>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p/>
- * 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;
-
-import com.fasterxml.jackson.core.type.TypeReference;
-import com.google.common.base.*;
-import com.google.common.collect.Iterables;
-import com.google.common.io.Files;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.client.HTableInterface;
-import org.apache.metron.Constants;
-import org.apache.metron.hbase.TableProvider;
-import org.apache.metron.hbase.converters.threatintel.ThreatIntelKey;
-import org.apache.metron.hbase.converters.threatintel.ThreatIntelValue;
-import org.apache.metron.integration.util.TestUtils;
-import org.apache.metron.integration.util.UnitTestHelper;
-import org.apache.metron.integration.util.integration.ComponentRunner;
-import org.apache.metron.integration.util.integration.Processor;
-import org.apache.metron.integration.util.integration.ReadinessState;
-import org.apache.metron.integration.util.integration.components.ElasticSearchComponent;
-import org.apache.metron.integration.util.integration.components.FluxTopologyComponent;
-import org.apache.metron.integration.util.integration.components.KafkaWithZKComponent;
-import org.apache.metron.integration.util.mock.MockGeoAdapter;
-import org.apache.metron.integration.util.mock.MockHTable;
-import org.apache.metron.integration.util.threatintel.ThreatIntelHelper;
-import org.apache.metron.reference.lookup.LookupKV;
-import org.apache.metron.utils.SourceConfigUtils;
-import org.junit.Assert;
-import org.junit.Test;
-import org.apache.metron.utils.JSONUtils;
-
-import javax.annotation.Nullable;
-import java.io.*;
-import java.text.SimpleDateFormat;
-import java.util.*;
-
-public class EnrichmentIntegrationTest {
-  private static final String SRC_IP = "ip_src_addr";
-  private static final String DST_IP = "ip_dst_addr";
-  private String fluxPath = "src/main/resources/Metron_Configs/topologies/enrichment/test.yaml";
-  private String indexDir = "target/elasticsearch";
-  private String hdfsDir = "target/enrichmentIntegrationTest/hdfs";
-  private String sampleParsedPath = "src/main/resources/SampleParsed/YafExampleParsed";
-  private String sampleIndexedPath = "src/main/resources/SampleIndexed/YafIndexed";
-  private Map<String, String> sourceConfigs = new HashMap<>();
-
-  public static class Provider implements TableProvider, Serializable {
-    MockHTable.Provider  provider = new MockHTable.Provider();
-    @Override
-    public HTableInterface getTable(Configuration config, String tableName) throws IOException {
-      return provider.getTable(config, tableName);
-    }
-  }
-
-  public static void cleanHdfsDir(String hdfsDirStr) {
-    File hdfsDir = new File(hdfsDirStr);
-    Stack<File> fs = new Stack<>();
-    if(hdfsDir.exists()) {
-      fs.push(hdfsDir);
-      while(!fs.empty()) {
-        File f = fs.pop();
-        if (f.isDirectory()) {
-          for(File child : f.listFiles()) {
-            fs.push(child);
-          }
-        }
-        else {
-          if (f.getName().startsWith("enrichment") || f.getName().endsWith(".json")) {
-            f.delete();
-          }
-        }
-      }
-    }
-  }
-
-  public static List<Map<String, Object> > readDocsFromDisk(String hdfsDirStr) throws IOException {
-    List<Map<String, Object>> ret = new ArrayList<>();
-    File hdfsDir = new File(hdfsDirStr);
-    Stack<File> fs = new Stack<>();
-    if(hdfsDir.exists()) {
-      fs.push(hdfsDir);
-      while(!fs.empty()) {
-        File f = fs.pop();
-        if(f.isDirectory()) {
-          for (File child : f.listFiles()) {
-            fs.push(child);
-          }
-        }
-        else {
-          System.out.println("Processed " + f);
-          if (f.getName().startsWith("enrichment") || f.getName().endsWith(".json")) {
-            List<byte[]> data = TestUtils.readSampleData(f.getPath());
-            Iterables.addAll(ret, Iterables.transform(data, new Function<byte[], Map<String, Object>>() {
-              @Nullable
-              @Override
-              public Map<String, Object> apply(@Nullable byte[] bytes) {
-                String s = new String(bytes);
-                try {
-                  return JSONUtils.INSTANCE.load(s, new TypeReference<Map<String, Object>>() {
-                  });
-                } catch (IOException e) {
-                  throw new RuntimeException(e);
-                }
-              }
-            }));
-          }
-        }
-      }
-    }
-    return ret;
-  }
-
-
-  @Test
-  public void test() throws Exception {
-    cleanHdfsDir(hdfsDir);
-    final String dateFormat = "yyyy.MM.dd.hh";
-    final String index = "test_index_" + new SimpleDateFormat(dateFormat).format(new Date());
-    String yafConfig = "{\n" +
-            "  \"index\": \"test\",\n" +
-            "  \"batchSize\": 5,\n" +
-            "  \"enrichmentFieldMap\":\n" +
-            "  {\n" +
-            "    \"geo\": [\"" + SRC_IP + "\", \"" + DST_IP + "\"],\n" +
-            "    \"host\": [\"" + SRC_IP + "\", \"" + DST_IP + "\"]\n" +
-            "  },\n" +
-            "  \"threatIntelFieldMap\":\n" +
-            "  {\n" +
-            "    \"ip\": [\"" + SRC_IP + "\", \"" + DST_IP + "\"]\n" +
-            "  }\n" +
-            "}";
-    sourceConfigs.put("test", yafConfig);
-    final List<byte[]> inputMessages = TestUtils.readSampleData(sampleParsedPath);
-    final String cf = "cf";
-    final String trackerHBaseTable = "tracker";
-    final String ipThreatIntelTable = "ip_threat_intel";
-    final Properties topologyProperties = new Properties() {{
-      setProperty("org.apache.metron.enrichment.host.known_hosts", "[{\"ip\":\"10.1.128.236\", \"local\":\"YES\", \"type\":\"webserver\", \"asset_value\" : \"important\"},\n" +
-              "{\"ip\":\"10.1.128.237\", \"local\":\"UNKNOWN\", \"type\":\"unknown\", \"asset_value\" : \"important\"},\n" +
-              "{\"ip\":\"10.60.10.254\", \"local\":\"YES\", \"type\":\"printer\", \"asset_value\" : \"important\"},\n" +
-              "{\"ip\":\"10.0.2.15\", \"local\":\"YES\", \"type\":\"printer\", \"asset_value\" : \"important\"}]");
-      setProperty("hbase.provider.impl","" + Provider.class.getName());
-      setProperty("threat.intel.tracker.table", trackerHBaseTable);
-      setProperty("threat.intel.tracker.cf", cf);
-      setProperty("threat.intel.ip.table", ipThreatIntelTable);
-      setProperty("threat.intel.ip.cf", cf);
-      setProperty("es.clustername", "metron");
-      setProperty("es.port", "9300");
-      setProperty("es.ip", "localhost");
-      setProperty("index.date.format", dateFormat);
-      setProperty("index.hdfs.output", hdfsDir);
-    }};
-    final KafkaWithZKComponent kafkaComponent = new KafkaWithZKComponent().withTopics(new ArrayList<KafkaWithZKComponent.Topic>() {{
-      add(new KafkaWithZKComponent.Topic(Constants.ENRICHMENT_TOPIC, 1));
-    }})
-            .withPostStartCallback(new Function<KafkaWithZKComponent, Void>() {
-              @Nullable
-              @Override
-              public Void apply(@Nullable KafkaWithZKComponent kafkaWithZKComponent) {
-                topologyProperties.setProperty("kafka.zk", kafkaWithZKComponent.getZookeeperConnect());
-                try {
-                  for(String sourceType: sourceConfigs.keySet()) {
-                    SourceConfigUtils.writeToZookeeper(sourceType, sourceConfigs.get(sourceType).getBytes(), kafkaWithZKComponent.getZookeeperConnect());
-                  }
-                } catch (Exception e) {
-                  e.printStackTrace();
-                }
-                return null;
-              }
-            });
-
-    ElasticSearchComponent esComponent = new ElasticSearchComponent.Builder()
-            .withHttpPort(9211)
-            .withIndexDir(new File(indexDir))
-            .build();
-
-    //create MockHBaseTables
-    final MockHTable trackerTable = (MockHTable)MockHTable.Provider.addToCache(trackerHBaseTable, cf);
-    final MockHTable ipTable = (MockHTable)MockHTable.Provider.addToCache(ipThreatIntelTable, cf);
-    ThreatIntelHelper.INSTANCE.load(ipTable, cf, new ArrayList<LookupKV<ThreatIntelKey, ThreatIntelValue>>(){{
-      add(new LookupKV<>(new ThreatIntelKey("10.0.2.3"), new ThreatIntelValue(new HashMap<String, String>())));
-    }});
-
-    FluxTopologyComponent fluxComponent = new FluxTopologyComponent.Builder()
-            .withTopologyLocation(new File(fluxPath))
-            .withTopologyName("test")
-            .withTopologyProperties(topologyProperties)
-            .build();
-
-    UnitTestHelper.verboseLogging();
-    ComponentRunner runner = new ComponentRunner.Builder()
-            .withComponent("kafka", kafkaComponent)
-            .withComponent("elasticsearch", esComponent)
-            .withComponent("storm", fluxComponent)
-            .withMillisecondsBetweenAttempts(10000)
-            .withNumRetries(10)
-            .build();
-    runner.start();
-    try {
-      fluxComponent.submitTopology();
-      kafkaComponent.writeMessages(Constants.ENRICHMENT_TOPIC, inputMessages);
-      List<Map<String, Object>> docs =
-              runner.process(new Processor<List<Map<String, Object>>>() {
-                List<Map<String, Object>> docs = null;
-
-                public ReadinessState process(ComponentRunner runner) {
-                  ElasticSearchComponent elasticSearchComponent = runner.getComponent("elasticsearch", ElasticSearchComponent.class);
-                  if (elasticSearchComponent.hasIndex(index)) {
-                    List<Map<String, Object>> docsFromDisk;
-                    try {
-                      docs = elasticSearchComponent.getAllIndexedDocs(index, "test_doc");
-                      docsFromDisk = readDocsFromDisk(hdfsDir);
-                      System.out.println(docs.size() + " vs " + inputMessages.size() + " vs " + docsFromDisk.size());
-                    } catch (IOException e) {
-                      throw new IllegalStateException("Unable to retrieve indexed documents.", e);
-                    }
-                    if (docs.size() < inputMessages.size() || docs.size() != docsFromDisk.size()) {
-                      return ReadinessState.NOT_READY;
-                    } else {
-                      return ReadinessState.READY;
-                    }
-                  } else {
-                    return ReadinessState.NOT_READY;
-                  }
-                }
-
-                public List<Map<String, Object>> getResult() {
-                  return docs;
-                }
-              });
-
-
-      Assert.assertEquals(inputMessages.size(), docs.size());
-
-      for (Map<String, Object> doc : docs) {
-        baseValidation(doc);
-        hostEnrichmentValidation(doc);
-        geoEnrichmentValidation(doc);
-        threatIntelValidation(doc);
-
-      }
-      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], "test_doc");
-      for (Map<String, Object> doc : docsFromDisk) {
-        baseValidation(doc);
-        hostEnrichmentValidation(doc);
-        geoEnrichmentValidation(doc);
-        threatIntelValidation(doc);
-
-      }
-    }
-    finally {
-      cleanHdfsDir(hdfsDir);
-      runner.stop();
-    }
-  }
-
-  public static void baseValidation(Map<String, Object> jsonDoc) {
-    assertEnrichmentsExists("threatintels.", setOf("ip"), jsonDoc.keySet());
-    assertEnrichmentsExists("enrichments.", setOf("geo", "host"), jsonDoc.keySet());
-    for(Map.Entry<String, Object> kv : jsonDoc.entrySet()) {
-      //ensure no values are empty.
-      Assert.assertTrue(kv.getValue().toString().length() > 0);
-    }
-    //ensure we always have a source ip and destination ip
-    Assert.assertNotNull(jsonDoc.get(SRC_IP));
-    Assert.assertNotNull(jsonDoc.get(DST_IP));
-  }
-
-  private static class EvaluationPayload {
-    Map<String, Object> indexedDoc;
-    String key;
-    public EvaluationPayload(Map<String, Object> indexedDoc, String key) {
-      this.indexedDoc = indexedDoc;
-      this.key = key;
-    }
-  }
-
-  private static enum HostEnrichments implements Predicate<EvaluationPayload>{
-    LOCAL_LOCATION(new Predicate<EvaluationPayload>() {
-
-      @Override
-      public boolean apply(@Nullable EvaluationPayload evaluationPayload) {
-        return evaluationPayload.indexedDoc.get("enrichments.host." + evaluationPayload.key + ".known_info.local").equals("YES");
-      }
-    })
-    ,UNKNOWN_LOCATION(new Predicate<EvaluationPayload>() {
-
-      @Override
-      public boolean apply(@Nullable EvaluationPayload evaluationPayload) {
-        return evaluationPayload.indexedDoc.get("enrichments.host." + evaluationPayload.key + ".known_info.local").equals("UNKNOWN");
-      }
-    })
-    ,IMPORTANT(new Predicate<EvaluationPayload>() {
-      @Override
-      public boolean apply(@Nullable EvaluationPayload evaluationPayload) {
-        return evaluationPayload.indexedDoc.get("enrichments.host." + evaluationPayload.key + ".known_info.asset_value").equals("important");
-      }
-    })
-    ,PRINTER_TYPE(new Predicate<EvaluationPayload>() {
-      @Override
-      public boolean apply(@Nullable EvaluationPayload evaluationPayload) {
-        return evaluationPayload.indexedDoc.get("enrichments.host." + evaluationPayload.key + ".known_info.type").equals("printer");
-      }
-    })
-    ,WEBSERVER_TYPE(new Predicate<EvaluationPayload>() {
-      @Override
-      public boolean apply(@Nullable EvaluationPayload evaluationPayload) {
-        return evaluationPayload.indexedDoc.get("enrichments.host." + evaluationPayload.key + ".known_info.type").equals("webserver");
-      }
-    })
-    ,UNKNOWN_TYPE(new Predicate<EvaluationPayload>() {
-      @Override
-      public boolean apply(@Nullable EvaluationPayload evaluationPayload) {
-        return evaluationPayload.indexedDoc.get("enrichments.host." + evaluationPayload.key + ".known_info.type").equals("unknown");
-      }
-    })
-    ;
-
-    Predicate<EvaluationPayload> _predicate;
-    HostEnrichments(Predicate<EvaluationPayload> predicate) {
-      this._predicate = predicate;
-    }
-
-    public boolean apply(EvaluationPayload payload) {
-      return _predicate.apply(payload);
-    }
-
-  }
-
-  private static void assertEnrichmentsExists(String topLevel, Set<String> expectedEnrichments, Set<String> keys) {
-    for(String key : keys) {
-      if(key.startsWith(topLevel)) {
-        String secondLevel = Iterables.get(Splitter.on(".").split(key), 1);
-        String message = "Found an enrichment/threat intel (" + secondLevel + ") that I didn't expect (expected enrichments :"
-                       + Joiner.on(",").join(expectedEnrichments) + "), but it was not there.  If you've created a new"
-                       + " enrichment, then please add a validation method to this unit test.  Otherwise, it's a solid error"
-                       + " and should be investigated.";
-        Assert.assertTrue( message, expectedEnrichments.contains(secondLevel));
-      }
-    }
-  }
-  private static void threatIntelValidation(Map<String, Object> indexedDoc) {
-    if(keyPatternExists("threatintels.", indexedDoc)) {
-      //if we have any threat intel messages, we want to tag is_alert to true
-      Assert.assertEquals(indexedDoc.get("is_alert"), "true");
-    }
-    else {
-      //For YAF this is the case, but if we do snort later on, this will be invalid.
-      Assert.assertNull(indexedDoc.get("is_alert"));
-    }
-    //ip threat intels
-    if(keyPatternExists("threatintels.ip.", indexedDoc)) {
-      if(indexedDoc.get(SRC_IP).equals("10.0.2.3")) {
-        Assert.assertEquals(indexedDoc.get("threatintels.ip." + SRC_IP + ".ip_threat_intel"), "alert");
-      }
-      else if(indexedDoc.get(DST_IP).equals("10.0.2.3")) {
-        Assert.assertEquals(indexedDoc.get("threatintels.ip." + DST_IP + ".ip_threat_intel"), "alert");
-      }
-      else {
-        Assert.fail("There was a threat intels that I did not expect.");
-      }
-    }
-
-  }
-
-  private static void geoEnrichmentValidation(Map<String, Object> indexedDoc) {
-    //should have geo enrichment on every message due to mock geo adapter
-    Assert.assertEquals(indexedDoc.get("enrichments.geo." + DST_IP + ".location_point"), MockGeoAdapter.DEFAULT_LOCATION_POINT);
-    Assert.assertEquals(indexedDoc.get("enrichments.geo." + SRC_IP +".location_point"), MockGeoAdapter.DEFAULT_LOCATION_POINT);
-    Assert.assertEquals(indexedDoc.get("enrichments.geo." + DST_IP + ".longitude"), MockGeoAdapter.DEFAULT_LONGITUDE);
-    Assert.assertEquals(indexedDoc.get("enrichments.geo." + SRC_IP + ".longitude"), MockGeoAdapter.DEFAULT_LONGITUDE);
-    Assert.assertEquals(indexedDoc.get("enrichments.geo." + DST_IP + ".city"), MockGeoAdapter.DEFAULT_CITY);
-    Assert.assertEquals(indexedDoc.get("enrichments.geo." + SRC_IP + ".city"), MockGeoAdapter.DEFAULT_CITY);
-    Assert.assertEquals(indexedDoc.get("enrichments.geo." + DST_IP + ".latitude"), MockGeoAdapter.DEFAULT_LATITUDE);
-    Assert.assertEquals(indexedDoc.get("enrichments.geo." + SRC_IP + ".latitude"), MockGeoAdapter.DEFAULT_LATITUDE);
-    Assert.assertEquals(indexedDoc.get("enrichments.geo." + DST_IP + ".country"), MockGeoAdapter.DEFAULT_COUNTRY);
-    Assert.assertEquals(indexedDoc.get("enrichments.geo." + SRC_IP + ".country"), MockGeoAdapter.DEFAULT_COUNTRY);
-    Assert.assertEquals(indexedDoc.get("enrichments.geo." + DST_IP + ".dmaCode"), MockGeoAdapter.DEFAULT_DMACODE);
-    Assert.assertEquals(indexedDoc.get("enrichments.geo." + SRC_IP + ".dmaCode"), MockGeoAdapter.DEFAULT_DMACODE);
-    Assert.assertEquals(indexedDoc.get("enrichments.geo." + DST_IP + ".postalCode"), MockGeoAdapter.DEFAULT_POSTAL_CODE);
-    Assert.assertEquals(indexedDoc.get("enrichments.geo." + SRC_IP + ".postalCode"), MockGeoAdapter.DEFAULT_POSTAL_CODE);
-  }
-
-  private static void hostEnrichmentValidation(Map<String, Object> indexedDoc) {
-    boolean enriched = false;
-    //important local printers
-    {
-      Set<String> ips = setOf("10.0.2.15", "10.60.10.254");
-      if (ips.contains(indexedDoc.get(SRC_IP))) {
-        //this is a local, important, printer
-        Assert.assertTrue(Predicates.and(HostEnrichments.LOCAL_LOCATION
-                ,HostEnrichments.IMPORTANT
-                ,HostEnrichments.PRINTER_TYPE
-                ).apply(new EvaluationPayload(indexedDoc, SRC_IP))
-        );
-        enriched = true;
-      }
-      if (ips.contains(indexedDoc.get(DST_IP))) {
-        Assert.assertTrue(Predicates.and(HostEnrichments.LOCAL_LOCATION
-                ,HostEnrichments.IMPORTANT
-                ,HostEnrichments.PRINTER_TYPE
-                ).apply(new EvaluationPayload(indexedDoc, DST_IP))
-        );
-        enriched = true;
-      }
-    }
-    //important local webservers
-    {
-      Set<String> ips = setOf("10.1.128.236");
-      if (ips.contains(indexedDoc.get(SRC_IP))) {
-        //this is a local, important, printer
-        Assert.assertTrue(Predicates.and(HostEnrichments.LOCAL_LOCATION
-                ,HostEnrichments.IMPORTANT
-                ,HostEnrichments.WEBSERVER_TYPE
-                ).apply(new EvaluationPayload(indexedDoc, SRC_IP))
-        );
-        enriched = true;
-      }
-      if (ips.contains(indexedDoc.get(DST_IP))) {
-        Assert.assertTrue(Predicates.and(HostEnrichments.LOCAL_LOCATION
-                ,HostEnrichments.IMPORTANT
-                ,HostEnrichments.WEBSERVER_TYPE
-                ).apply(new EvaluationPayload(indexedDoc, DST_IP))
-        );
-        enriched = true;
-      }
-    }
-    if(!enriched) {
-      Assert.assertFalse(keyPatternExists("enrichments.host", indexedDoc));
-    }
-  }
-
-
-  private static boolean keyPatternExists(String pattern, Map<String, Object> indexedObj) {
-    for(String k : indexedObj.keySet()) {
-      if(k.startsWith(pattern)) {
-        return true;
-      }
-    }
-    return false;
-  }
-  private static Set<String> setOf(String... items) {
-    Set<String> ret = new HashSet<>();
-    for(String item : items) {
-      ret.add(item);
-    }
-    return ret;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/e59b1a31/metron-streaming/Metron-Topologies/src/test/java/org/apache/metron/integration/ParserIntegrationTest.java
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-Topologies/src/test/java/org/apache/metron/integration/ParserIntegrationTest.java b/metron-streaming/Metron-Topologies/src/test/java/org/apache/metron/integration/ParserIntegrationTest.java
index 3c6972a..10e13da 100644
--- a/metron-streaming/Metron-Topologies/src/test/java/org/apache/metron/integration/ParserIntegrationTest.java
+++ b/metron-streaming/Metron-Topologies/src/test/java/org/apache/metron/integration/ParserIntegrationTest.java
@@ -17,7 +17,6 @@
  */
 package org.apache.metron.integration;
 
-import com.google.common.base.Function;
 import org.apache.metron.Constants;
 import org.apache.metron.integration.util.TestUtils;
 import org.apache.metron.integration.util.UnitTestHelper;
@@ -26,50 +25,38 @@ import org.apache.metron.integration.util.integration.Processor;
 import org.apache.metron.integration.util.integration.ReadinessState;
 import org.apache.metron.integration.util.integration.components.FluxTopologyComponent;
 import org.apache.metron.integration.util.integration.components.KafkaWithZKComponent;
-import org.apache.metron.utils.SourceConfigUtils;
 import org.codehaus.jackson.map.ObjectMapper;
 import org.junit.Assert;
 import org.junit.Test;
 
-import javax.annotation.Nullable;
-import java.io.*;
-import java.util.*;
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
 
-public abstract class ParserIntegrationTest {
+public abstract class ParserIntegrationTest extends BaseIntegrationTest {
 
   public abstract String getFluxPath();
   public abstract String getSampleInputPath();
   public abstract String getSampleParsedPath();
-  public abstract String getSourceType();
-  public abstract String getSourceConfig();
+  public abstract String getSensorType();
   public abstract String getFluxTopicProperty();
 
   @Test
   public void test() throws Exception {
 
-    final String kafkaTopic = "test";
+    final String kafkaTopic = getSensorType();
 
     final List<byte[]> inputMessages = TestUtils.readSampleData(getSampleInputPath());
 
     final Properties topologyProperties = new Properties() {{
       setProperty(getFluxTopicProperty(), kafkaTopic);
     }};
-    final KafkaWithZKComponent kafkaComponent = new KafkaWithZKComponent().withTopics(new ArrayList<KafkaWithZKComponent.Topic>() {{
+    final KafkaWithZKComponent kafkaComponent = getKafkaComponent(topologyProperties, new ArrayList<KafkaWithZKComponent.Topic>() {{
       add(new KafkaWithZKComponent.Topic(kafkaTopic, 1));
-    }})
-            .withPostStartCallback(new Function<KafkaWithZKComponent, Void>() {
-              @Nullable
-              @Override
-              public Void apply(@Nullable KafkaWithZKComponent kafkaWithZKComponent) {
-                topologyProperties.setProperty("kafka.zk", kafkaWithZKComponent.getZookeeperConnect());
-                try {
-                  SourceConfigUtils.writeToZookeeper(getSourceType(), getSourceConfig().getBytes(), kafkaWithZKComponent.getZookeeperConnect());
-                } catch (Exception e) {
-                  e.printStackTrace();
-                }
-                return null;
-              }
-            });
+    }});
 
     topologyProperties.setProperty("kafka.broker", kafkaComponent.getBrokerList());
     FluxTopologyComponent fluxComponent = new FluxTopologyComponent.Builder()

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/e59b1a31/metron-streaming/Metron-Topologies/src/test/java/org/apache/metron/integration/PcapParserIntegrationTest.java
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-Topologies/src/test/java/org/apache/metron/integration/PcapParserIntegrationTest.java b/metron-streaming/Metron-Topologies/src/test/java/org/apache/metron/integration/PcapParserIntegrationTest.java
index 284e3c0..6a220a4 100644
--- a/metron-streaming/Metron-Topologies/src/test/java/org/apache/metron/integration/PcapParserIntegrationTest.java
+++ b/metron-streaming/Metron-Topologies/src/test/java/org/apache/metron/integration/PcapParserIntegrationTest.java
@@ -6,9 +6,9 @@
  * 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
- *
+ * <p/>
  * http://www.apache.org/licenses/LICENSE-2.0
- *
+ * <p/>
  * 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.
@@ -17,7 +17,6 @@
  */
 package org.apache.metron.integration;
 
-import com.google.common.base.Function;
 import com.google.common.collect.Iterables;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
@@ -28,6 +27,7 @@ import org.apache.hadoop.io.BytesWritable;
 import org.apache.hadoop.io.IntWritable;
 import org.apache.hadoop.io.SequenceFile;
 import org.apache.hadoop.io.SequenceFile.Reader;
+import org.apache.metron.Constants;
 import org.apache.metron.hbase.TableProvider;
 import org.apache.metron.integration.util.UnitTestHelper;
 import org.apache.metron.integration.util.integration.ComponentRunner;
@@ -42,13 +42,16 @@ import org.json.simple.JSONObject;
 import org.junit.Assert;
 import org.junit.Test;
 
-import javax.annotation.Nullable;
 import java.io.File;
 import java.io.IOException;
 import java.io.Serializable;
-import java.util.*;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
 
-public class PcapParserIntegrationTest {
+public class PcapParserIntegrationTest extends BaseIntegrationTest {
 
   private static String BASE_DIR = "pcap";
   private static String DATA_DIR = BASE_DIR + "/data_dir";
@@ -57,7 +60,8 @@ public class PcapParserIntegrationTest {
   private String targetDir = "target";
 
   public static class Provider implements TableProvider, Serializable {
-    MockHTable.Provider  provider = new MockHTable.Provider();
+    MockHTable.Provider provider = new MockHTable.Provider();
+
     @Override
     public HTableInterface getTable(Configuration config, String tableName) throws IOException {
       return provider.getTable(config, tableName);
@@ -80,8 +84,9 @@ public class PcapParserIntegrationTest {
     }
     return outDir;
   }
+
   private static void clearOutDir(File outDir) {
-    for(File f : outDir.listFiles()) {
+    for (File f : outDir.listFiles()) {
       f.delete();
     }
   }
@@ -89,13 +94,13 @@ public class PcapParserIntegrationTest {
   private static Map<String, byte[]> readPcaps(Path pcapFile) throws IOException {
     SequenceFile.Reader reader = new SequenceFile.Reader(new Configuration(),
             Reader.file(pcapFile)
-            );
+    );
     Map<String, byte[]> ret = new HashMap<>();
     IntWritable key = new IntWritable();
     BytesWritable value = new BytesWritable();
     PcapParser parser = new PcapParser();
     parser.init();
-    while(reader.next(key, value)) {
+    while (reader.next(key, value)) {
       int keyInt = key.get();
       byte[] valueBytes = value.copyBytes();
       JSONObject message = parser.parse(valueBytes).get(0);
@@ -124,29 +129,19 @@ public class PcapParserIntegrationTest {
     File baseDir = new File(new File(targetDir), BASE_DIR);
     Assert.assertNotNull(topologiesDir);
     Assert.assertNotNull(targetDir);
-    Path pcapFile = new Path(topologiesDir + "/../../SampleInput/PCAPExampleOutput");
+    Path pcapFile = new Path("../Metron-Testing/src/main/resources/sample/data/SampleInput/PCAPExampleOutput");
     final Map<String, byte[]> pcapEntries = readPcaps(pcapFile);
     Assert.assertTrue(Iterables.size(pcapEntries.keySet()) > 0);
     final Properties topologyProperties = new Properties() {{
-      setProperty("hbase.provider.impl","" + Provider.class.getName());
+      setProperty("hbase.provider.impl", "" + Provider.class.getName());
       setProperty("spout.kafka.topic.pcap", kafkaTopic);
-      setProperty("bolt.hbase.table.name",tableName);
+      setProperty("bolt.hbase.table.name", tableName);
       setProperty("bolt.hbase.table.fields", columnFamily + ":" + columnIdentifier);
     }};
-    final KafkaWithZKComponent kafkaComponent = new KafkaWithZKComponent().withTopics(new ArrayList<KafkaWithZKComponent.Topic>() {{
+    final KafkaWithZKComponent kafkaComponent = getKafkaComponent(topologyProperties, new ArrayList<KafkaWithZKComponent.Topic>() {{
       add(new KafkaWithZKComponent.Topic(kafkaTopic, 1));
-    }})
-            .withPostStartCallback(new Function<KafkaWithZKComponent, Void>() {
-                                     @Nullable
-                                     @Override
-                                     public Void apply(@Nullable KafkaWithZKComponent kafkaWithZKComponent) {
-
-                                       topologyProperties.setProperty("kafka.zk", kafkaWithZKComponent.getZookeeperConnect());
-                                       return null;
-                                     }
-                                   }
-            );
-    //.withExistingZookeeper("localhost:2000");
+      add(new KafkaWithZKComponent.Topic(Constants.ENRICHMENT_TOPIC, 1));
+    }});
 
     FluxTopologyComponent fluxComponent = new FluxTopologyComponent.Builder()
             .withTopologyLocation(new File(topologiesDir + "/pcap/test.yaml"))
@@ -154,7 +149,7 @@ public class PcapParserIntegrationTest {
             .withTopologyProperties(topologyProperties)
             .build();
 
-    final MockHTable pcapTable = (MockHTable)MockHTable.Provider.addToCache(tableName, columnFamily);
+    final MockHTable pcapTable = (MockHTable) MockHTable.Provider.addToCache(tableName, columnFamily);
 
     UnitTestHelper.verboseLogging();
     ComponentRunner runner = new ComponentRunner.Builder()
@@ -166,7 +161,6 @@ public class PcapParserIntegrationTest {
             .build();
     try {
       runner.start();
-      System.out.println("Components started...");
       fluxComponent.submitTopology();
       kafkaComponent.writeMessages(kafkaTopic, pcapEntries.values());
       System.out.println("Sent pcap data: " + pcapEntries.size());
@@ -178,8 +172,9 @@ public class PcapParserIntegrationTest {
         public ReadinessState process(ComponentRunner runner) {
           int hbaseCount = 0;
           try {
+            System.out.println("Waiting...");
             ResultScanner resultScanner = pcapTable.getScanner(columnFamily.getBytes(), columnIdentifier.getBytes());
-            while(resultScanner.next() != null) hbaseCount++;
+            while (resultScanner.next() != null) hbaseCount++;
           } catch (IOException e) {
             e.printStackTrace();
           }
@@ -198,7 +193,7 @@ public class PcapParserIntegrationTest {
       ResultScanner resultScanner = pcapTable.getScanner(columnFamily.getBytes(), columnIdentifier.getBytes());
       Result result;
       int rowCount = 0;
-      while((result = resultScanner.next()) != null) {
+      while ((result = resultScanner.next()) != null) {
         String rowKey = new String(result.getRow());
         byte[] hbaseValue = result.getValue(columnFamily.getBytes(), columnIdentifier.getBytes());
         byte[] originalValue = pcapEntries.get(rowKey);
@@ -208,8 +203,7 @@ public class PcapParserIntegrationTest {
       }
       Assert.assertEquals(pcapEntries.size(), rowCount);
       System.out.println("Ended");
-    }
-    finally {
+    } finally {
       runner.stop();
       clearOutDir(outDir);
       clearOutDir(queryDir);

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/e59b1a31/metron-streaming/Metron-Topologies/src/test/java/org/apache/metron/integration/SnortIntegrationTest.java
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-Topologies/src/test/java/org/apache/metron/integration/SnortIntegrationTest.java b/metron-streaming/Metron-Topologies/src/test/java/org/apache/metron/integration/SnortIntegrationTest.java
index 7508ad7..752c622 100644
--- a/metron-streaming/Metron-Topologies/src/test/java/org/apache/metron/integration/SnortIntegrationTest.java
+++ b/metron-streaming/Metron-Topologies/src/test/java/org/apache/metron/integration/SnortIntegrationTest.java
@@ -21,41 +21,25 @@ public class SnortIntegrationTest extends ParserIntegrationTest {
 
   @Override
   public String getFluxPath() {
-    return "src/main/resources/Metron_Configs/topologies/snort/test.yaml";
+    return "../Metron-Topologies/src/main/resources/Metron_Configs/topologies/snort/test.yaml";
   }
 
   @Override
   public String getSampleInputPath() {
-    return "src/main/resources/SampleInput/SnortOutput";
+    return "../Metron-Testing/src/main/resources/sample/data/SampleInput/SnortOutput";
   }
 
   @Override
   public String getSampleParsedPath() {
-    return "src/main/resources/SampleParsed/SnortParsed";
+    return "../Metron-Testing/src/main/resources/sample/data/SampleParsed/SnortParsed";
   }
 
   @Override
-  public String getSourceType() {
+  public String getSensorType() {
     return "snort";
   }
 
   @Override
-  public String getSourceConfig() {
-    return "{\"index\": \"snort\"," +
-            " \"batchSize\": 1," +
-            " \"enrichmentFieldMap\":" +
-            "  {" +
-            "    \"geo\": [\"src\", \"dst\"]," +
-            "    \"host\": [\"src\", \"dst\"]" +
-            "  }," +
-            "  \"threatIntelFieldMap\":" +
-            "  {" +
-            "    \"ip\": [\"src\", \"dst\"]" +
-            "  }" +
-            "}";
-  }
-
-  @Override
   public String getFluxTopicProperty() {
     return "spout.kafka.topic.snort";
   }

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/e59b1a31/metron-streaming/Metron-Topologies/src/test/java/org/apache/metron/integration/YafIntegrationTest.java
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-Topologies/src/test/java/org/apache/metron/integration/YafIntegrationTest.java b/metron-streaming/Metron-Topologies/src/test/java/org/apache/metron/integration/YafIntegrationTest.java
index f255a0a..114fa98 100644
--- a/metron-streaming/Metron-Topologies/src/test/java/org/apache/metron/integration/YafIntegrationTest.java
+++ b/metron-streaming/Metron-Topologies/src/test/java/org/apache/metron/integration/YafIntegrationTest.java
@@ -21,41 +21,25 @@ public class YafIntegrationTest extends ParserIntegrationTest {
 
   @Override
   public String getFluxPath() {
-    return "src/main/resources/Metron_Configs/topologies/yaf/test.yaml";
+    return "../Metron-Topologies/src/main/resources/Metron_Configs/topologies/yaf/test.yaml";
   }
 
   @Override
   public String getSampleInputPath() {
-    return "src/main/resources/SampleInput/YafExampleOutput";
+    return "../Metron-Testing/src/main/resources/sample/data/SampleInput/YafExampleOutput";
   }
 
   @Override
   public String getSampleParsedPath() {
-    return "src/main/resources/SampleParsed/YafExampleParsed";
+    return "../Metron-Testing/src/main/resources/sample/data/SampleParsed/YafExampleParsed";
   }
 
   @Override
-  public String getSourceType() {
+  public String getSensorType() {
     return "yaf";
   }
 
   @Override
-  public String getSourceConfig() {
-    return "{\"index\": \"yaf\"," +
-            " \"batchSize\": 5," +
-            " \"enrichmentFieldMap\":" +
-            "  {" +
-            "    \"geo\": [\"ip_src_addr\", \"ip_dst_addr\"]," +
-            "    \"host\": [\"ip_src_addr\", \"ip_dst_addr\"]" +
-            "  }," +
-            "  \"threatIntelFieldMap\":" +
-            "  {" +
-            "    \"ip\": [\"ip_src_addr\", \"ip_dst_addr\"]" +
-            "  }" +
-            "}";
-  }
-
-  @Override
   public String getFluxTopicProperty() {
     return "spout.kafka.topic.yaf";
   }

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/e59b1a31/metron-streaming/Metron-Topologies/src/test/java/org/apache/metron/integration/util/TestUtils.java
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-Topologies/src/test/java/org/apache/metron/integration/util/TestUtils.java b/metron-streaming/Metron-Topologies/src/test/java/org/apache/metron/integration/util/TestUtils.java
deleted file mode 100644
index a3db041..0000000
--- a/metron-streaming/Metron-Topologies/src/test/java/org/apache/metron/integration/util/TestUtils.java
+++ /dev/null
@@ -1,37 +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
- * <p/>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p/>
- * 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.util;
-
-import java.io.BufferedReader;
-import java.io.FileReader;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
-
-public class TestUtils {
-
-  public static List<byte[]> readSampleData(String samplePath) throws IOException {
-    BufferedReader br = new BufferedReader(new FileReader(samplePath));
-    List<byte[]> ret = new ArrayList<>();
-    for (String line = null; (line = br.readLine()) != null; ) {
-      ret.add(line.getBytes());
-    }
-    br.close();
-    return ret;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/e59b1a31/metron-streaming/Metron-Topologies/src/test/java/org/apache/metron/integration/util/mock/MockGeoAdapter.java
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-Topologies/src/test/java/org/apache/metron/integration/util/mock/MockGeoAdapter.java b/metron-streaming/Metron-Topologies/src/test/java/org/apache/metron/integration/util/mock/MockGeoAdapter.java
deleted file mode 100644
index ee71cda..0000000
--- a/metron-streaming/Metron-Topologies/src/test/java/org/apache/metron/integration/util/mock/MockGeoAdapter.java
+++ /dev/null
@@ -1,63 +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.integration.util.mock;
-
-import com.google.common.base.Joiner;
-import org.apache.metron.enrichment.interfaces.EnrichmentAdapter;
-import org.json.simple.JSONObject;
-
-import java.io.Serializable;
-
-public class MockGeoAdapter implements EnrichmentAdapter<String>,
-        Serializable {
-
-  public static final String DEFAULT_LOC_ID = "1";
-  public static final String DEFAULT_COUNTRY = "test country";
-  public static final String DEFAULT_CITY = "test city";
-  public static final String DEFAULT_POSTAL_CODE = "test postalCode";
-  public static final String DEFAULT_LATITUDE = "test latitude";
-  public static final String DEFAULT_LONGITUDE = "test longitude";
-  public static final String DEFAULT_DMACODE= "test dmaCode";
-  public static final String DEFAULT_LOCATION_POINT= Joiner.on(',').join(DEFAULT_LONGITUDE, DEFAULT_LATITUDE);
-
-  @Override
-  public void logAccess(String value) {
-
-  }
-
-  public JSONObject enrich(String metadata) {
-    JSONObject enriched = new JSONObject();
-    enriched.put("locID", DEFAULT_LOC_ID);
-    enriched.put("country", DEFAULT_COUNTRY);
-    enriched.put("city", DEFAULT_CITY);
-    enriched.put("postalCode", DEFAULT_POSTAL_CODE);
-    enriched.put("latitude", DEFAULT_LATITUDE);
-    enriched.put("longitude", DEFAULT_LONGITUDE);
-    enriched.put("dmaCode", DEFAULT_DMACODE);
-    enriched.put("location_point", DEFAULT_LOCATION_POINT);
-    return enriched;
-  }
-
-  public boolean initializeAdapter() {
-    return true;
-  }
-
-  public void cleanup() {
-
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/e59b1a31/metron-streaming/Metron-Topologies/src/test/java/org/apache/metron/integration/util/mock/MockHBaseConnector.java
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-Topologies/src/test/java/org/apache/metron/integration/util/mock/MockHBaseConnector.java b/metron-streaming/Metron-Topologies/src/test/java/org/apache/metron/integration/util/mock/MockHBaseConnector.java
deleted file mode 100644
index b6b4a9d..0000000
--- a/metron-streaming/Metron-Topologies/src/test/java/org/apache/metron/integration/util/mock/MockHBaseConnector.java
+++ /dev/null
@@ -1,52 +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.integration.util.mock;
-
-import org.apache.hadoop.hbase.client.Put;
-import org.apache.hadoop.hbase.client.RetriesExhaustedWithDetailsException;
-import org.apache.metron.hbase.Connector;
-import org.apache.metron.hbase.TupleTableConfig;
-
-import java.io.IOException;
-import java.io.InterruptedIOException;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
-
-public class MockHBaseConnector extends Connector {
-    static List<Put> puts = Collections.synchronizedList(new ArrayList<Put>());
-    public MockHBaseConnector(TupleTableConfig conf, String _quorum, String _port) throws IOException {
-        super(conf, _quorum, _port);
-    }
-
-    @Override
-    public void put(Put put) throws InterruptedIOException, RetriesExhaustedWithDetailsException {
-        puts.add(put);
-    }
-
-    @Override
-    public void close() {
-
-    }
-    public static void clear() {
-        puts.clear();
-    }
-    public static List<Put> getPuts() {
-        return puts;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/e59b1a31/metron-streaming/Metron-Topologies/src/test/java/org/apache/metron/integration/util/threatintel/ThreatIntelHelper.java
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-Topologies/src/test/java/org/apache/metron/integration/util/threatintel/ThreatIntelHelper.java b/metron-streaming/Metron-Topologies/src/test/java/org/apache/metron/integration/util/threatintel/ThreatIntelHelper.java
deleted file mode 100644
index 1e64362..0000000
--- a/metron-streaming/Metron-Topologies/src/test/java/org/apache/metron/integration/util/threatintel/ThreatIntelHelper.java
+++ /dev/null
@@ -1,40 +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.integration.util.threatintel;
-
-import org.apache.hadoop.hbase.client.HTableInterface;
-import org.apache.hadoop.hbase.client.Put;
-import org.apache.metron.hbase.converters.threatintel.ThreatIntelKey;
-import org.apache.metron.hbase.converters.threatintel.ThreatIntelValue;
-import org.apache.metron.reference.lookup.LookupKV;
-import org.apache.metron.threatintel.ThreatIntelResults;
-import org.apache.metron.hbase.converters.threatintel.ThreatIntelConverter;
-
-import java.io.IOException;
-
-public enum ThreatIntelHelper {
-    INSTANCE;
-    ThreatIntelConverter converter = new ThreatIntelConverter();
-
-    public void load(HTableInterface table, String cf, Iterable<LookupKV<ThreatIntelKey, ThreatIntelValue>> results) throws IOException {
-        for(LookupKV<ThreatIntelKey, ThreatIntelValue> result : results) {
-            Put put = converter.toPut(cf, result.getKey(), result.getValue());
-            table.put(put);
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/e59b1a31/metron-streaming/pom.xml
----------------------------------------------------------------------
diff --git a/metron-streaming/pom.xml b/metron-streaming/pom.xml
index 86486ab..c4222e8 100644
--- a/metron-streaming/pom.xml
+++ b/metron-streaming/pom.xml
@@ -47,6 +47,8 @@
 		<global_json_schema_validator_version>2.2.5</global_json_schema_validator_version>
 		<global_slf4j_version>1.7.7</global_slf4j_version>
 		<global_opencsv_version>3.7</global_opencsv_version>
+		<global_solr_version>5.2.1</global_solr_version>
+		<global_mockito_version>1.9.5</global_mockito_version>
 	</properties>
 	<licenses>
 		<license>
@@ -75,6 +77,8 @@
 		<module>Metron-DataLoads</module>
 		<module>Metron-Topologies</module>
 		<module>Metron-Pcap_Service</module>
+		<module>Metron-Elasticsearch</module>
+		<module>Metron-Solr</module>
 		<module>Metron-Testing</module>
 	</modules>
 	<dependencies>
@@ -137,7 +141,9 @@
 						<exclude>**/*.json</exclude>
 						<exclude>**/*.log</exclude>
 						<exclude>**/src/main/resources/patterns/**</exclude>
-						<exclude>**/src/main/resources/SampleInput/**</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>**/dependency-reduced-pom.xml</exclude>
 					</excludes>
 				</configuration>

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/e59b1a31/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 9474cf0..8554419 100644
--- a/pom.xml
+++ b/pom.xml
@@ -19,70 +19,72 @@
 	code base of Metron.
   -->
 <project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
-		 xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
-	<modelVersion>4.0.0</modelVersion>
-	<groupId>org.apache.metron</groupId>
-	<artifactId>Metron</artifactId>
-	<version>0.1BETA</version>
-	<packaging>pom</packaging>
-	<name>Metron</name>
-	<description>Performs release auditing for Metron.</description>
-	<url>https://metron.incubator.apache.org/</url>
-	<build>
-		<plugins>
-			<plugin>
-				<groupId>org.apache.rat</groupId>
-				<artifactId>apache-rat-plugin</artifactId>
-				<version>0.11</version>
-				<configuration>
-				<excludes>
-					<exclude>**/README.md</exclude>
-					<exclude>**/VERSION</exclude>
-					<exclude>**/*.json</exclude>
-					<exclude>**/*.log</exclude>
-					<exclude>**/*.template</exclude>
-					<exclude>**/.*</exclude>
-					<exclude>**/.*/**</exclude>
-					<exclude>**/*.seed</exclude>
-					<exclude>**/*.iml</exclude>
-					<exclude>**/ansible.cfg</exclude>
-					<exclude>site/**</exclude>
-          <exclude>metron-ui/lib/public/css/normalize.min.css</exclude>
-          <exclude>metron-ui/lib/public/app/panels/pcap/lib/showdown.js</exclude>
-          <!-- 3rd party bundled javascript dependencies -->
-          <exclude>metron-ui/lib/public/vendor/**</exclude>
-          <!-- Kibana panels copied from kibana and bundled -->
-          <exclude>metron-ui/lib/public/app/panels/dashcontrol/**</exclude>
-          <exclude>metron-ui/lib/public/app/panels/filtering/**</exclude>
-          <exclude>metron-ui/lib/public/app/panels/histogram/**</exclude>
-          <exclude>metron-ui/lib/public/app/panels/hits/**</exclude>
-          <exclude>metron-ui/lib/public/app/panels/map/**</exclude>
-          <exclude>metron-ui/lib/public/app/panels/query/**</exclude>
-          <exclude>metron-ui/lib/public/app/panels/sparklines/**</exclude>
-          <exclude>metron-ui/lib/public/app/panels/table/**</exclude>
-          <exclude>metron-ui/lib/public/app/panels/terms/**</exclude>
-          <exclude>metron-ui/lib/public/app/panels/text/**</exclude>
-          <exclude>metron-ui/lib/public/app/panels/timepicker/**</exclude>
-          <exclude>metron-ui/lib/public/app/panels/trends/**</exclude>
-          <exclude>metron-ui/lib/public/app/panels/bettermap/**</exclude>
-          <exclude>metron-ui/lib/public/app/panels/column/**</exclude>
-          <exclude>metron-ui/lib/public/app/panels/derivequeries/**</exclude>
-          <exclude>metron-ui/lib/public/app/panels/stats/**</exclude>
-          <exclude>metron-ui/lib/public/app/partials/**</exclude>
-          <exclude>metron-ui/lib/public/app/services/**</exclude>
-          <!-- fontawesome fonts are declared in the license, so we can exclude here -->
-          <exclude>metron-ui/lib/public/css/font-awesome.min.css</exclude>
-          <exclude>metron-ui/lib/public/font/**</exclude>
-					<exclude>**/src/main/resources/patterns/**</exclude>
-					<exclude>**/src/test/resources/**</exclude>
-					<exclude>**/src/main/resources/Sample*/**</exclude>
-					<exclude>**/dependency-reduced-pom.xml</exclude>
-				        <exclude>**/files/opensoc-ui</exclude>
-					<exclude>**/target/**</exclude>
-					<exclude>**/bro-plugin-kafka/build/**</exclude>
-				</excludes>
-				</configuration>
-			</plugin>
-		</plugins>
-	</build>
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <modelVersion>4.0.0</modelVersion>
+    <groupId>org.apache.metron</groupId>
+    <artifactId>Metron</artifactId>
+    <version>0.1BETA</version>
+    <packaging>pom</packaging>
+    <name>Metron</name>
+    <description>Performs release auditing for Metron.</description>
+    <url>https://metron.incubator.apache.org/</url>
+    <build>
+        <plugins>
+            <plugin>
+                <groupId>org.apache.rat</groupId>
+                <artifactId>apache-rat-plugin</artifactId>
+                <version>0.11</version>
+                <configuration>
+                    <excludes>
+                        <exclude>**/README.md</exclude>
+                        <exclude>**/VERSION</exclude>
+                        <exclude>**/*.json</exclude>
+                        <exclude>**/*.log</exclude>
+                        <exclude>**/*.template</exclude>
+                        <exclude>**/.*</exclude>
+                        <exclude>**/.*/**</exclude>
+                        <exclude>**/*.seed</exclude>
+                        <exclude>**/*.iml</exclude>
+                        <exclude>**/ansible.cfg</exclude>
+                        <exclude>site/**</exclude>
+                        <exclude>metron-ui/lib/public/css/normalize.min.css</exclude>
+                        <exclude>metron-ui/lib/public/app/panels/pcap/lib/showdown.js</exclude>
+                        <!-- 3rd party bundled javascript dependencies -->
+                        <exclude>metron-ui/lib/public/vendor/**</exclude>
+                        <!-- Kibana panels copied from kibana and bundled -->
+                        <exclude>metron-ui/lib/public/app/panels/dashcontrol/**</exclude>
+                        <exclude>metron-ui/lib/public/app/panels/filtering/**</exclude>
+                        <exclude>metron-ui/lib/public/app/panels/histogram/**</exclude>
+                        <exclude>metron-ui/lib/public/app/panels/hits/**</exclude>
+                        <exclude>metron-ui/lib/public/app/panels/map/**</exclude>
+                        <exclude>metron-ui/lib/public/app/panels/query/**</exclude>
+                        <exclude>metron-ui/lib/public/app/panels/sparklines/**</exclude>
+                        <exclude>metron-ui/lib/public/app/panels/table/**</exclude>
+                        <exclude>metron-ui/lib/public/app/panels/terms/**</exclude>
+                        <exclude>metron-ui/lib/public/app/panels/text/**</exclude>
+                        <exclude>metron-ui/lib/public/app/panels/timepicker/**</exclude>
+                        <exclude>metron-ui/lib/public/app/panels/trends/**</exclude>
+                        <exclude>metron-ui/lib/public/app/panels/bettermap/**</exclude>
+                        <exclude>metron-ui/lib/public/app/panels/column/**</exclude>
+                        <exclude>metron-ui/lib/public/app/panels/derivequeries/**</exclude>
+                        <exclude>metron-ui/lib/public/app/panels/stats/**</exclude>
+                        <exclude>metron-ui/lib/public/app/partials/**</exclude>
+                        <exclude>metron-ui/lib/public/app/services/**</exclude>
+                        <exclude>metron-ui/lib/public/app/services/**</exclude>
+                        <!-- fontawesome fonts are declared in the license, so we can exclude here -->
+                        <exclude>metron-ui/lib/public/css/font-awesome.min.css</exclude>
+                        <exclude>metron-ui/lib/public/font/**</exclude>
+                        <exclude>metron-ui/node_modules/**</exclude>
+                        <exclude>**/src/main/resources/patterns/**</exclude>
+                        <exclude>**/src/test/resources/**</exclude>
+                        <exclude>**/src/main/resources/sample/data/Sample*/**</exclude>
+                        <exclude>**/dependency-reduced-pom.xml</exclude>
+                        <exclude>**/files/opensoc-ui</exclude>
+                        <exclude>**/target/**</exclude>
+                        <exclude>**/bro-plugin-kafka/build/**</exclude>
+                    </excludes>
+                </configuration>
+            </plugin>
+        </plugins>
+    </build>
 </project>


[12/15] incubator-metron git commit: METRON 86: Adding Solr indexing support (merrimanr via cestella) closes apache/incubator-metron#67

Posted by ce...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/e59b1a31/metron-streaming/Metron-Solr/src/test/resources/solr/conf/currency.xml
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-Solr/src/test/resources/solr/conf/currency.xml b/metron-streaming/Metron-Solr/src/test/resources/solr/conf/currency.xml
new file mode 100644
index 0000000..3a9c58a
--- /dev/null
+++ b/metron-streaming/Metron-Solr/src/test/resources/solr/conf/currency.xml
@@ -0,0 +1,67 @@
+<?xml version="1.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.
+-->
+
+<!-- Example exchange rates file for CurrencyField type named "currency" in example schema -->
+
+<currencyConfig version="1.0">
+  <rates>
+    <!-- Updated from http://www.exchangerate.com/ at 2011-09-27 -->
+    <rate from="USD" to="ARS" rate="4.333871" comment="ARGENTINA Peso" />
+    <rate from="USD" to="AUD" rate="1.025768" comment="AUSTRALIA Dollar" />
+    <rate from="USD" to="EUR" rate="0.743676" comment="European Euro" />
+    <rate from="USD" to="BRL" rate="1.881093" comment="BRAZIL Real" />
+    <rate from="USD" to="CAD" rate="1.030815" comment="CANADA Dollar" />
+    <rate from="USD" to="CLP" rate="519.0996" comment="CHILE Peso" />
+    <rate from="USD" to="CNY" rate="6.387310" comment="CHINA Yuan" />
+    <rate from="USD" to="CZK" rate="18.47134" comment="CZECH REP. Koruna" />
+    <rate from="USD" to="DKK" rate="5.515436" comment="DENMARK Krone" />
+    <rate from="USD" to="HKD" rate="7.801922" comment="HONG KONG Dollar" />
+    <rate from="USD" to="HUF" rate="215.6169" comment="HUNGARY Forint" />
+    <rate from="USD" to="ISK" rate="118.1280" comment="ICELAND Krona" />
+    <rate from="USD" to="INR" rate="49.49088" comment="INDIA Rupee" />
+    <rate from="USD" to="XDR" rate="0.641358" comment="INTNL MON. FUND SDR" />
+    <rate from="USD" to="ILS" rate="3.709739" comment="ISRAEL Sheqel" />
+    <rate from="USD" to="JPY" rate="76.32419" comment="JAPAN Yen" />
+    <rate from="USD" to="KRW" rate="1169.173" comment="KOREA (SOUTH) Won" />
+    <rate from="USD" to="KWD" rate="0.275142" comment="KUWAIT Dinar" />
+    <rate from="USD" to="MXN" rate="13.85895" comment="MEXICO Peso" />
+    <rate from="USD" to="NZD" rate="1.285159" comment="NEW ZEALAND Dollar" />
+    <rate from="USD" to="NOK" rate="5.859035" comment="NORWAY Krone" />
+    <rate from="USD" to="PKR" rate="87.57007" comment="PAKISTAN Rupee" />
+    <rate from="USD" to="PEN" rate="2.730683" comment="PERU Sol" />
+    <rate from="USD" to="PHP" rate="43.62039" comment="PHILIPPINES Peso" />
+    <rate from="USD" to="PLN" rate="3.310139" comment="POLAND Zloty" />
+    <rate from="USD" to="RON" rate="3.100932" comment="ROMANIA Leu" />
+    <rate from="USD" to="RUB" rate="32.14663" comment="RUSSIA Ruble" />
+    <rate from="USD" to="SAR" rate="3.750465" comment="SAUDI ARABIA Riyal" />
+    <rate from="USD" to="SGD" rate="1.299352" comment="SINGAPORE Dollar" />
+    <rate from="USD" to="ZAR" rate="8.329761" comment="SOUTH AFRICA Rand" />
+    <rate from="USD" to="SEK" rate="6.883442" comment="SWEDEN Krona" />
+    <rate from="USD" to="CHF" rate="0.906035" comment="SWITZERLAND Franc" />
+    <rate from="USD" to="TWD" rate="30.40283" comment="TAIWAN Dollar" />
+    <rate from="USD" to="THB" rate="30.89487" comment="THAILAND Baht" />
+    <rate from="USD" to="AED" rate="3.672955" comment="U.A.E. Dirham" />
+    <rate from="USD" to="UAH" rate="7.988582" comment="UKRAINE Hryvnia" />
+    <rate from="USD" to="GBP" rate="0.647910" comment="UNITED KINGDOM Pound" />
+    
+    <!-- Cross-rates for some common currencies -->
+    <rate from="EUR" to="GBP" rate="0.869914" />  
+    <rate from="EUR" to="NOK" rate="7.800095" />  
+    <rate from="GBP" to="NOK" rate="8.966508" />  
+  </rates>
+</currencyConfig>

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/e59b1a31/metron-streaming/Metron-Solr/src/test/resources/solr/conf/lang/stopwords_en.txt
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-Solr/src/test/resources/solr/conf/lang/stopwords_en.txt b/metron-streaming/Metron-Solr/src/test/resources/solr/conf/lang/stopwords_en.txt
new file mode 100644
index 0000000..2c164c0
--- /dev/null
+++ b/metron-streaming/Metron-Solr/src/test/resources/solr/conf/lang/stopwords_en.txt
@@ -0,0 +1,54 @@
+# 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.
+
+# a couple of test stopwords to test that the words are really being
+# configured from this file:
+stopworda
+stopwordb
+
+# Standard english stop words taken from Lucene's StopAnalyzer
+a
+an
+and
+are
+as
+at
+be
+but
+by
+for
+if
+in
+into
+is
+it
+no
+not
+of
+on
+or
+such
+that
+the
+their
+then
+there
+these
+they
+this
+to
+was
+will
+with

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/e59b1a31/metron-streaming/Metron-Solr/src/test/resources/solr/conf/protwords.txt
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-Solr/src/test/resources/solr/conf/protwords.txt b/metron-streaming/Metron-Solr/src/test/resources/solr/conf/protwords.txt
new file mode 100644
index 0000000..1dfc0ab
--- /dev/null
+++ b/metron-streaming/Metron-Solr/src/test/resources/solr/conf/protwords.txt
@@ -0,0 +1,21 @@
+# 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.
+
+#-----------------------------------------------------------------------
+# Use a protected word file to protect against the stemmer reducing two
+# unrelated words to the same base word.
+
+# Some non-words that normally won't be encountered,
+# just to test that they won't be stemmed.
+dontstems
+zwhacky
+

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/e59b1a31/metron-streaming/Metron-Solr/src/test/resources/solr/conf/schema.xml
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-Solr/src/test/resources/solr/conf/schema.xml b/metron-streaming/Metron-Solr/src/test/resources/solr/conf/schema.xml
new file mode 100644
index 0000000..a689e1c
--- /dev/null
+++ b/metron-streaming/Metron-Solr/src/test/resources/solr/conf/schema.xml
@@ -0,0 +1,191 @@
+<?xml version="1.0" encoding="UTF-8" ?>
+<!--
+ 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.
+-->
+
+<schema name="metron" version="1.5">
+
+    <field name="_version_" type="long" indexed="true" stored="true"/>
+    <field name="id" type="string" indexed="true" stored="true" required="true" multiValued="false"/>
+    <field name="sensorType" type="string" indexed="true" stored="true" required="true"/>
+
+    <dynamicField name="*_i" type="int" indexed="true" stored="true"/>
+    <dynamicField name="*_is" type="int" indexed="true" stored="true" multiValued="true"/>
+    <dynamicField name="*_s" type="string" indexed="true" stored="true"/>
+    <dynamicField name="*_ss" type="string" indexed="true" stored="true" multiValued="true"/>
+    <dynamicField name="*_l" type="long" indexed="true" stored="true"/>
+    <dynamicField name="*_ls" type="long" indexed="true" stored="true" multiValued="true"/>
+    <dynamicField name="*_t" type="text_general" indexed="true" stored="true"/>
+    <dynamicField name="*_txt" type="text_general" indexed="true" stored="true" multiValued="true"/>
+    <dynamicField name="*_en" type="text_en" indexed="true" stored="true" multiValued="true"/>
+    <dynamicField name="*_b" type="boolean" indexed="true" stored="true"/>
+    <dynamicField name="*_bs" type="boolean" indexed="true" stored="true" multiValued="true"/>
+    <dynamicField name="*_f" type="float" indexed="true" stored="true"/>
+    <dynamicField name="*_fs" type="float" indexed="true" stored="true" multiValued="true"/>
+    <dynamicField name="*_d" type="double" indexed="true" stored="true"/>
+    <dynamicField name="*_ds" type="double" indexed="true" stored="true" multiValued="true"/>
+    <dynamicField name="*_coordinate" type="tdouble" indexed="true" stored="false"/>
+    <dynamicField name="*_dt" type="date" indexed="true" stored="true"/>
+    <dynamicField name="*_dts" type="date" indexed="true" stored="true" multiValued="true"/>
+    <dynamicField name="*_p" type="location" indexed="true" stored="true"/>
+    <dynamicField name="*_ti" type="tint" indexed="true" stored="true"/>
+    <dynamicField name="*_tl" type="tlong" indexed="true" stored="true"/>
+    <dynamicField name="*_tf" type="tfloat" indexed="true" stored="true"/>
+    <dynamicField name="*_td" type="tdouble" indexed="true" stored="true"/>
+    <dynamicField name="*_tdt" type="tdate" indexed="true" stored="true"/>
+    <dynamicField name="*_c" type="currency" indexed="true" stored="true"/>
+    <dynamicField name="ignored_*" type="ignored" multiValued="true"/>
+    <dynamicField name="attr_*" type="text_general" indexed="true" stored="true" multiValued="true"/>
+    <dynamicField name="random_*" type="random"/>
+
+    <uniqueKey>id</uniqueKey>
+
+    <fieldType name="string" class="solr.StrField" sortMissingLast="true"/>
+    <fieldType name="boolean" class="solr.BoolField" sortMissingLast="true"/>
+    <fieldType name="int" class="solr.TrieIntField" precisionStep="0" positionIncrementGap="0"/>
+    <fieldType name="float" class="solr.TrieFloatField" precisionStep="0" positionIncrementGap="0"/>
+    <fieldType name="long" class="solr.TrieLongField" precisionStep="0" positionIncrementGap="0"/>
+    <fieldType name="double" class="solr.TrieDoubleField" precisionStep="0" positionIncrementGap="0"/>
+    <fieldType name="tint" class="solr.TrieIntField" precisionStep="8" positionIncrementGap="0"/>
+    <fieldType name="tfloat" class="solr.TrieFloatField" precisionStep="8" positionIncrementGap="0"/>
+    <fieldType name="tlong" class="solr.TrieLongField" precisionStep="8" positionIncrementGap="0"/>
+    <fieldType name="tdouble" class="solr.TrieDoubleField" precisionStep="8" positionIncrementGap="0"/>
+    <fieldType name="date" class="solr.TrieDateField" precisionStep="0" positionIncrementGap="0"/>
+    <fieldType name="tdate" class="solr.TrieDateField" precisionStep="6" positionIncrementGap="0"/>
+    <fieldType name="binary" class="solr.BinaryField"/>
+    <fieldType name="random" class="solr.RandomSortField" indexed="true"/>
+    <fieldType name="text_ws" class="solr.TextField" positionIncrementGap="100">
+        <analyzer>
+            <tokenizer class="solr.WhitespaceTokenizerFactory"/>
+        </analyzer>
+    </fieldType>
+    <fieldType name="text_general" class="solr.TextField" positionIncrementGap="100">
+        <analyzer type="index">
+            <tokenizer class="solr.StandardTokenizerFactory"/>
+            <filter class="solr.StopFilterFactory" ignoreCase="true" words="stopwords.txt"/>
+            <filter class="solr.LowerCaseFilterFactory"/>
+        </analyzer>
+        <analyzer type="query">
+            <tokenizer class="solr.StandardTokenizerFactory"/>
+            <filter class="solr.StopFilterFactory" ignoreCase="true" words="stopwords.txt"/>
+            <filter class="solr.SynonymFilterFactory" synonyms="synonyms.txt" ignoreCase="true" expand="true"/>
+            <filter class="solr.LowerCaseFilterFactory"/>
+        </analyzer>
+    </fieldType>
+    <fieldType name="text_en" class="solr.TextField" positionIncrementGap="100">
+        <analyzer type="index">
+            <tokenizer class="solr.StandardTokenizerFactory"/>
+            <filter class="solr.StopFilterFactory"
+                    ignoreCase="true"
+                    words="lang/stopwords_en.txt"
+            />
+            <filter class="solr.LowerCaseFilterFactory"/>
+            <filter class="solr.EnglishPossessiveFilterFactory"/>
+            <filter class="solr.KeywordMarkerFilterFactory" protected="protwords.txt"/>
+            <filter class="solr.PorterStemFilterFactory"/>
+        </analyzer>
+        <analyzer type="query">
+            <tokenizer class="solr.StandardTokenizerFactory"/>
+            <filter class="solr.SynonymFilterFactory" synonyms="synonyms.txt" ignoreCase="true" expand="true"/>
+            <filter class="solr.StopFilterFactory"
+                    ignoreCase="true"
+                    words="lang/stopwords_en.txt"
+            />
+            <filter class="solr.LowerCaseFilterFactory"/>
+            <filter class="solr.EnglishPossessiveFilterFactory"/>
+            <filter class="solr.KeywordMarkerFilterFactory" protected="protwords.txt"/>
+            <filter class="solr.PorterStemFilterFactory"/>
+        </analyzer>
+    </fieldType>
+    <fieldType name="text_en_splitting" class="solr.TextField" positionIncrementGap="100" autoGeneratePhraseQueries="true">
+        <analyzer type="index">
+            <tokenizer class="solr.WhitespaceTokenizerFactory"/>
+            <filter class="solr.StopFilterFactory"
+                    ignoreCase="true"
+                    words="lang/stopwords_en.txt"
+            />
+            <filter class="solr.WordDelimiterFilterFactory" generateWordParts="1" generateNumberParts="1" catenateWords="1" catenateNumbers="1" catenateAll="0" splitOnCaseChange="1"/>
+            <filter class="solr.LowerCaseFilterFactory"/>
+            <filter class="solr.KeywordMarkerFilterFactory" protected="protwords.txt"/>
+            <filter class="solr.PorterStemFilterFactory"/>
+        </analyzer>
+        <analyzer type="query">
+            <tokenizer class="solr.WhitespaceTokenizerFactory"/>
+            <filter class="solr.SynonymFilterFactory" synonyms="synonyms.txt" ignoreCase="true" expand="true"/>
+            <filter class="solr.StopFilterFactory"
+                    ignoreCase="true"
+                    words="lang/stopwords_en.txt"
+            />
+            <filter class="solr.WordDelimiterFilterFactory" generateWordParts="1" generateNumberParts="1" catenateWords="0" catenateNumbers="0" catenateAll="0" splitOnCaseChange="1"/>
+            <filter class="solr.LowerCaseFilterFactory"/>
+            <filter class="solr.KeywordMarkerFilterFactory" protected="protwords.txt"/>
+            <filter class="solr.PorterStemFilterFactory"/>
+        </analyzer>
+    </fieldType>
+
+    <fieldType name="text_en_splitting_tight" class="solr.TextField" positionIncrementGap="100" autoGeneratePhraseQueries="true">
+        <analyzer>
+            <tokenizer class="solr.WhitespaceTokenizerFactory"/>
+            <filter class="solr.SynonymFilterFactory" synonyms="synonyms.txt" ignoreCase="true" expand="false"/>
+            <filter class="solr.StopFilterFactory" ignoreCase="true" words="lang/stopwords_en.txt"/>
+            <filter class="solr.WordDelimiterFilterFactory" generateWordParts="0" generateNumberParts="0" catenateWords="1" catenateNumbers="1" catenateAll="0"/>
+            <filter class="solr.LowerCaseFilterFactory"/>
+            <filter class="solr.KeywordMarkerFilterFactory" protected="protwords.txt"/>
+            <filter class="solr.EnglishMinimalStemFilterFactory"/>
+            <filter class="solr.RemoveDuplicatesTokenFilterFactory"/>
+        </analyzer>
+    </fieldType>
+    <fieldType name="text_general_rev" class="solr.TextField" positionIncrementGap="100">
+        <analyzer type="index">
+            <tokenizer class="solr.StandardTokenizerFactory"/>
+            <filter class="solr.StopFilterFactory" ignoreCase="true" words="stopwords.txt"/>
+            <filter class="solr.LowerCaseFilterFactory"/>
+            <filter class="solr.ReversedWildcardFilterFactory" withOriginal="true"
+                    maxPosAsterisk="3" maxPosQuestion="2" maxFractionAsterisk="0.33"/>
+        </analyzer>
+        <analyzer type="query">
+            <tokenizer class="solr.StandardTokenizerFactory"/>
+            <filter class="solr.SynonymFilterFactory" synonyms="synonyms.txt" ignoreCase="true" expand="true"/>
+            <filter class="solr.StopFilterFactory" ignoreCase="true" words="stopwords.txt"/>
+            <filter class="solr.LowerCaseFilterFactory"/>
+        </analyzer>
+    </fieldType>
+    <fieldType name="alphaOnlySort" class="solr.TextField" sortMissingLast="true" omitNorms="true">
+        <analyzer>
+            <tokenizer class="solr.KeywordTokenizerFactory"/>
+            <filter class="solr.LowerCaseFilterFactory"/>
+            <filter class="solr.TrimFilterFactory"/>
+            <filter class="solr.PatternReplaceFilterFactory"
+                    pattern="([^a-z])" replacement="" replace="all"
+            />
+        </analyzer>
+    </fieldType>
+    <fieldType name="lowercase" class="solr.TextField" positionIncrementGap="100">
+        <analyzer>
+            <tokenizer class="solr.KeywordTokenizerFactory"/>
+            <filter class="solr.LowerCaseFilterFactory"/>
+        </analyzer>
+    </fieldType>
+    <fieldType name="ignored" stored="false" indexed="false" multiValued="true" class="solr.StrField"/>
+    <fieldType name="point" class="solr.PointType" dimension="2" subFieldSuffix="_d"/>
+    <fieldType name="location" class="solr.LatLonType" subFieldSuffix="_coordinate"/>
+    <fieldType name="location_rpt" class="solr.SpatialRecursivePrefixTreeFieldType"
+               geo="true" distErrPct="0.025" maxDistErr="0.001" distanceUnits="kilometers"/>
+    <fieldType name="bbox" class="solr.BBoxField"
+               geo="true" distanceUnits="kilometers" numberType="_bbox_coord"/>
+    <fieldType name="_bbox_coord" class="solr.TrieDoubleField" precisionStep="8" docValues="true" stored="false"/>
+    <fieldType name="currency" class="solr.CurrencyField" precisionStep="8" defaultCurrency="USD" currencyConfig="currency.xml"/>
+</schema>

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/e59b1a31/metron-streaming/Metron-Solr/src/test/resources/solr/conf/solrconfig.xml
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-Solr/src/test/resources/solr/conf/solrconfig.xml b/metron-streaming/Metron-Solr/src/test/resources/solr/conf/solrconfig.xml
new file mode 100644
index 0000000..7d30fea
--- /dev/null
+++ b/metron-streaming/Metron-Solr/src/test/resources/solr/conf/solrconfig.xml
@@ -0,0 +1,583 @@
+<?xml version="1.0" encoding="UTF-8" ?>
+<!--
+ 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.
+-->
+
+<!-- 
+     For more details about configurations options that may appear in
+     this file, see http://wiki.apache.org/solr/SolrConfigXml. 
+-->
+<config>
+  <!-- In all configuration below, a prefix of "solr." for class names
+       is an alias that causes solr to search appropriate packages,
+       including org.apache.solr.(search|update|request|core|analysis)
+
+       You may also specify a fully qualified Java classname if you
+       have your own custom plugins.
+    -->
+
+  <!-- Controls what version of Lucene various components of Solr
+       adhere to.  Generally, you want to use the latest version to
+       get all bug fixes and improvements. It is highly recommended
+       that you fully re-index after changing this setting as it can
+       affect both how text is indexed and queried.
+  -->
+  <luceneMatchVersion>5.2.1</luceneMatchVersion>
+
+  <!-- Data Directory
+
+       Used to specify an alternate directory to hold all index data
+       other than the default ./data under the Solr home.  If
+       replication is in use, this should match the replication
+       configuration.
+    -->
+  <dataDir>${solr.data.dir:}</dataDir>
+
+
+  <!-- The DirectoryFactory to use for indexes.
+       
+       solr.StandardDirectoryFactory is filesystem
+       based and tries to pick the best implementation for the current
+       JVM and platform.  solr.NRTCachingDirectoryFactory, the default,
+       wraps solr.StandardDirectoryFactory and caches small files in memory
+       for better NRT performance.
+
+       One can force a particular implementation via solr.MMapDirectoryFactory,
+       solr.NIOFSDirectoryFactory, or solr.SimpleFSDirectoryFactory.
+
+       solr.RAMDirectoryFactory is memory based, not
+       persistent, and doesn't work with replication.
+    -->
+  <directoryFactory name="DirectoryFactory" 
+                    class="${solr.directoryFactory:solr.NRTCachingDirectoryFactory}">
+  </directoryFactory> 
+
+  <!-- The CodecFactory for defining the format of the inverted index.
+       The default implementation is SchemaCodecFactory, which is the official Lucene
+       index format, but hooks into the schema to provide per-field customization of
+       the postings lists and per-document values in the fieldType element
+       (postingsFormat/docValuesFormat). Note that most of the alternative implementations
+       are experimental, so if you choose to customize the index format, it's a good
+       idea to convert back to the official format e.g. via IndexWriter.addIndexes(IndexReader)
+       before upgrading to a newer version to avoid unnecessary reindexing.
+  -->
+  <codecFactory class="solr.SchemaCodecFactory"/>
+
+  <schemaFactory class="ClassicIndexSchemaFactory"/>
+
+  <!-- ~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
+       Index Config - These settings control low-level behavior of indexing
+       Most example settings here show the default value, but are commented
+       out, to more easily see where customizations have been made.
+       
+       Note: This replaces <indexDefaults> and <mainIndex> from older versions
+       ~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~ -->
+  <indexConfig>
+
+    <!-- LockFactory 
+
+         This option specifies which Lucene LockFactory implementation
+         to use.
+      
+         single = SingleInstanceLockFactory - suggested for a
+                  read-only index or when there is no possibility of
+                  another process trying to modify the index.
+         native = NativeFSLockFactory - uses OS native file locking.
+                  Do not use when multiple solr webapps in the same
+                  JVM are attempting to share a single index.
+         simple = SimpleFSLockFactory  - uses a plain file for locking
+
+         Defaults: 'native' is default for Solr3.6 and later, otherwise
+                   'simple' is the default
+
+         More details on the nuances of each LockFactory...
+         http://wiki.apache.org/lucene-java/AvailableLockFactories
+    -->
+    <lockType>${solr.lock.type:native}</lockType>
+
+    <!-- Lucene Infostream
+       
+         To aid in advanced debugging, Lucene provides an "InfoStream"
+         of detailed information when indexing.
+
+         Setting the value to true will instruct the underlying Lucene
+         IndexWriter to write its info stream to solr's log. By default,
+         this is enabled here, and controlled through log4j.properties.
+      -->
+     <infoStream>true</infoStream>
+  </indexConfig>
+
+
+  <!-- JMX
+       
+       This example enables JMX if and only if an existing MBeanServer
+       is found, use this if you want to configure JMX through JVM
+       parameters. Remove this to disable exposing Solr configuration
+       and statistics to JMX.
+
+       For more details see http://wiki.apache.org/solr/SolrJmx
+    -->
+  <jmx />
+  <!-- If you want to connect to a particular server, specify the
+       agentId 
+    -->
+  <!-- <jmx agentId="myAgent" /> -->
+  <!-- If you want to start a new MBeanServer, specify the serviceUrl -->
+  <!-- <jmx serviceUrl="service:jmx:rmi:///jndi/rmi://localhost:9999/solr"/>
+    -->
+
+  <!-- The default high-performance update handler -->
+  <updateHandler class="solr.DirectUpdateHandler2">
+
+    <!-- Enables a transaction log, used for real-time get, durability, and
+         and solr cloud replica recovery.  The log can grow as big as
+         uncommitted changes to the index, so use of a hard autoCommit
+         is recommended (see below).
+         "dir" - the target directory for transaction logs, defaults to the
+                solr data directory.
+         "numVersionBuckets" - sets the number of buckets used to keep
+                track of max version values when checking for re-ordered
+                updates; increase this value to reduce the cost of
+                synchronizing access to version buckets during high-volume
+                indexing, this requires 8 bytes (long) * numVersionBuckets
+                of heap space per Solr core.
+    -->
+    <updateLog>
+      <str name="dir">${solr.ulog.dir:}</str>
+      <int name="numVersionBuckets">${solr.ulog.numVersionBuckets:65536}</int>
+    </updateLog>
+ 
+    <!-- AutoCommit
+
+         Perform a hard commit automatically under certain conditions.
+         Instead of enabling autoCommit, consider using "commitWithin"
+         when adding documents. 
+
+         http://wiki.apache.org/solr/UpdateXmlMessages
+
+         maxDocs - Maximum number of documents to add since the last
+                   commit before automatically triggering a new commit.
+
+         maxTime - Maximum amount of time in ms that is allowed to pass
+                   since a document was added before automatically
+                   triggering a new commit. 
+         openSearcher - if false, the commit causes recent index changes
+           to be flushed to stable storage, but does not cause a new
+           searcher to be opened to make those changes visible.
+
+         If the updateLog is enabled, then it's highly recommended to
+         have some sort of hard autoCommit to limit the log size.
+      -->
+     <autoCommit> 
+       <maxTime>${solr.autoCommit.maxTime:15000}</maxTime> 
+       <openSearcher>false</openSearcher> 
+     </autoCommit>
+
+    <!-- softAutoCommit is like autoCommit except it causes a
+         'soft' commit which only ensures that changes are visible
+         but does not ensure that data is synced to disk.  This is
+         faster and more near-realtime friendly than a hard commit.
+      -->
+     <autoSoftCommit> 
+       <maxTime>${solr.autoSoftCommit.maxTime:-1}</maxTime> 
+     </autoSoftCommit>
+
+  </updateHandler>
+  
+  <!-- ~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
+       Query section - these settings control query time things like caches
+       ~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~ -->
+  <query>
+    <!-- Max Boolean Clauses
+
+         Maximum number of clauses in each BooleanQuery,  an exception
+         is thrown if exceeded.
+
+         ** WARNING **
+         
+         This option actually modifies a global Lucene property that
+         will affect all SolrCores.  If multiple solrconfig.xml files
+         disagree on this property, the value at any given moment will
+         be based on the last SolrCore to be initialized.
+         
+      -->
+    <maxBooleanClauses>1024</maxBooleanClauses>
+
+
+    <!-- Solr Internal Query Caches
+
+         There are two implementations of cache available for Solr,
+         LRUCache, based on a synchronized LinkedHashMap, and
+         FastLRUCache, based on a ConcurrentHashMap.  
+
+         FastLRUCache has faster gets and slower puts in single
+         threaded operation and thus is generally faster than LRUCache
+         when the hit ratio of the cache is high (> 75%), and may be
+         faster under other scenarios on multi-cpu systems.
+    -->
+
+    <!-- Filter Cache
+
+         Cache used by SolrIndexSearcher for filters (DocSets),
+         unordered sets of *all* documents that match a query.  When a
+         new searcher is opened, its caches may be prepopulated or
+         "autowarmed" using data from caches in the old searcher.
+         autowarmCount is the number of items to prepopulate.  For
+         LRUCache, the autowarmed items will be the most recently
+         accessed items.
+
+         Parameters:
+           class - the SolrCache implementation LRUCache or
+               (LRUCache or FastLRUCache)
+           size - the maximum number of entries in the cache
+           initialSize - the initial capacity (number of entries) of
+               the cache.  (see java.util.HashMap)
+           autowarmCount - the number of entries to prepopulate from
+               and old cache.  
+      -->
+    <filterCache class="solr.FastLRUCache"
+                 size="512"
+                 initialSize="512"
+                 autowarmCount="0"/>
+
+    <!-- Query Result Cache
+
+        Caches results of searches - ordered lists of document ids
+        (DocList) based on a query, a sort, and the range of documents requested.
+        Additional supported parameter by LRUCache:
+           maxRamMB - the maximum amount of RAM (in MB) that this cache is allowed
+                      to occupy
+     -->
+    <queryResultCache class="solr.LRUCache"
+                     size="512"
+                     initialSize="512"
+                     autowarmCount="0"/>
+   
+    <!-- Document Cache
+
+         Caches Lucene Document objects (the stored fields for each
+         document).  Since Lucene internal document ids are transient,
+         this cache will not be autowarmed.  
+      -->
+    <documentCache class="solr.LRUCache"
+                   size="512"
+                   initialSize="512"
+                   autowarmCount="0"/>
+    
+    <!-- custom cache currently used by block join --> 
+    <cache name="perSegFilter"
+      class="solr.search.LRUCache"
+      size="10"
+      initialSize="0"
+      autowarmCount="10"
+      regenerator="solr.NoOpRegenerator" />
+
+    <!-- Lazy Field Loading
+
+         If true, stored fields that are not requested will be loaded
+         lazily.  This can result in a significant speed improvement
+         if the usual case is to not load all stored fields,
+         especially if the skipped fields are large compressed text
+         fields.
+    -->
+    <enableLazyFieldLoading>true</enableLazyFieldLoading>
+
+   <!-- Result Window Size
+
+        An optimization for use with the queryResultCache.  When a search
+        is requested, a superset of the requested number of document ids
+        are collected.  For example, if a search for a particular query
+        requests matching documents 10 through 19, and queryWindowSize is 50,
+        then documents 0 through 49 will be collected and cached.  Any further
+        requests in that range can be satisfied via the cache.  
+     -->
+   <queryResultWindowSize>20</queryResultWindowSize>
+
+   <!-- Maximum number of documents to cache for any entry in the
+        queryResultCache. 
+     -->
+   <queryResultMaxDocsCached>200</queryResultMaxDocsCached>
+
+    <!-- Use Cold Searcher
+
+         If a search request comes in and there is no current
+         registered searcher, then immediately register the still
+         warming searcher and use it.  If "false" then all requests
+         will block until the first searcher is done warming.
+      -->
+    <useColdSearcher>false</useColdSearcher>
+
+    <!-- Max Warming Searchers
+         
+         Maximum number of searchers that may be warming in the
+         background concurrently.  An error is returned if this limit
+         is exceeded.
+
+         Recommend values of 1-2 for read-only slaves, higher for
+         masters w/o cache warming.
+      -->
+    <maxWarmingSearchers>2</maxWarmingSearchers>
+
+  </query>
+
+
+  <!-- Request Dispatcher
+
+       This section contains instructions for how the SolrDispatchFilter
+       should behave when processing requests for this SolrCore.
+
+       handleSelect is a legacy option that affects the behavior of requests
+       such as /select?qt=XXX
+
+       handleSelect="true" will cause the SolrDispatchFilter to process
+       the request and dispatch the query to a handler specified by the 
+       "qt" param, assuming "/select" isn't already registered.
+
+       handleSelect="false" will cause the SolrDispatchFilter to
+       ignore "/select" requests, resulting in a 404 unless a handler
+       is explicitly registered with the name "/select"
+
+       handleSelect="true" is not recommended for new users, but is the default
+       for backwards compatibility
+    -->
+  <requestDispatcher handleSelect="false" >
+    <!-- Request Parsing
+
+         These settings indicate how Solr Requests may be parsed, and
+         what restrictions may be placed on the ContentStreams from
+         those requests
+
+         enableRemoteStreaming - enables use of the stream.file
+         and stream.url parameters for specifying remote streams.
+
+         multipartUploadLimitInKB - specifies the max size (in KiB) of
+         Multipart File Uploads that Solr will allow in a Request.
+         
+         formdataUploadLimitInKB - specifies the max size (in KiB) of
+         form data (application/x-www-form-urlencoded) sent via
+         POST. You can use POST to pass request parameters not
+         fitting into the URL.
+         
+         addHttpRequestToContext - if set to true, it will instruct
+         the requestParsers to include the original HttpServletRequest
+         object in the context map of the SolrQueryRequest under the 
+         key "httpRequest". It will not be used by any of the existing
+         Solr components, but may be useful when developing custom 
+         plugins.
+         
+         *** WARNING ***
+         The settings below authorize Solr to fetch remote files, You
+         should make sure your system has some authentication before
+         using enableRemoteStreaming="true"
+
+      --> 
+    <requestParsers enableRemoteStreaming="true" 
+                    multipartUploadLimitInKB="2048000"
+                    formdataUploadLimitInKB="2048"
+                    addHttpRequestToContext="false"/>
+
+    <!-- HTTP Caching
+
+         Set HTTP caching related parameters (for proxy caches and clients).
+
+         The options below instruct Solr not to output any HTTP Caching
+         related headers
+      -->
+    <httpCaching never304="true" />
+
+  </requestDispatcher>
+
+  <!-- Request Handlers 
+
+       http://wiki.apache.org/solr/SolrRequestHandler
+
+       Incoming queries will be dispatched to a specific handler by name
+       based on the path specified in the request.
+
+       Legacy behavior: If the request path uses "/select" but no Request
+       Handler has that name, and if handleSelect="true" has been specified in
+       the requestDispatcher, then the Request Handler is dispatched based on
+       the qt parameter.  Handlers without a leading '/' are accessed this way
+       like so: http://host/app/[core/]select?qt=name  If no qt is
+       given, then the requestHandler that declares default="true" will be
+       used or the one named "standard".
+
+       If a Request Handler is declared with startup="lazy", then it will
+       not be initialized until the first request that uses it.
+
+    -->
+  <!-- SearchHandler
+
+       http://wiki.apache.org/solr/SearchHandler
+
+       For processing Search Queries, the primary Request Handler
+       provided with Solr is "SearchHandler" It delegates to a sequent
+       of SearchComponents (see below) and supports distributed
+       queries across multiple shards
+    -->
+  <requestHandler name="/select" class="solr.SearchHandler">
+    <!-- default values for query parameters can be specified, these
+         will be overridden by parameters in the request
+      -->
+     <lst name="defaults">
+       <str name="echoParams">explicit</str>
+       <int name="rows">10</int>
+     </lst>
+
+    </requestHandler>
+
+  <!-- A request handler that returns indented JSON by default -->
+  <requestHandler name="/query" class="solr.SearchHandler">
+     <lst name="defaults">
+       <str name="echoParams">explicit</str>
+       <str name="wt">json</str>
+       <str name="indent">true</str>
+       <str name="df">text</str>
+     </lst>
+  </requestHandler>
+
+  <!--
+    The export request handler is used to export full sorted result sets.
+    Do not change these defaults.
+  -->
+  <requestHandler name="/export" class="solr.SearchHandler">
+    <lst name="invariants">
+      <str name="rq">{!xport}</str>
+      <str name="wt">xsort</str>
+      <str name="distrib">false</str>
+    </lst>
+
+    <arr name="components">
+      <str>query</str>
+    </arr>
+  </requestHandler>
+
+
+  <initParams path="/update/**,/query,/select,/tvrh,/elevate,/spell">
+    <lst name="defaults">
+      <str name="df">text</str>
+    </lst>
+  </initParams>
+
+  <!-- Field Analysis Request Handler
+
+       RequestHandler that provides much the same functionality as
+       analysis.jsp. Provides the ability to specify multiple field
+       types and field names in the same request and outputs
+       index-time and query-time analysis for each of them.
+
+       Request parameters are:
+       analysis.fieldname - field name whose analyzers are to be used
+
+       analysis.fieldtype - field type whose analyzers are to be used
+       analysis.fieldvalue - text for index-time analysis
+       q (or analysis.q) - text for query time analysis
+       analysis.showmatch (true|false) - When set to true and when
+           query analysis is performed, the produced tokens of the
+           field value analysis will be marked as "matched" for every
+           token that is produces by the query analysis
+   -->
+  <requestHandler name="/analysis/field" 
+                  startup="lazy"
+                  class="solr.FieldAnalysisRequestHandler" />
+
+
+  <!-- Document Analysis Handler
+
+       http://wiki.apache.org/solr/AnalysisRequestHandler
+
+       An analysis handler that provides a breakdown of the analysis
+       process of provided documents. This handler expects a (single)
+       content stream with the following format:
+
+       <docs>
+         <doc>
+           <field name="id">1</field>
+           <field name="name">The Name</field>
+           <field name="text">The Text Value</field>
+         </doc>
+         <doc>...</doc>
+         <doc>...</doc>
+         ...
+       </docs>
+
+    Note: Each document must contain a field which serves as the
+    unique key. This key is used in the returned response to associate
+    an analysis breakdown to the analyzed document.
+
+    Like the FieldAnalysisRequestHandler, this handler also supports
+    query analysis by sending either an "analysis.query" or "q"
+    request parameter that holds the query text to be analyzed. It
+    also supports the "analysis.showmatch" parameter which when set to
+    true, all field tokens that match the query tokens will be marked
+    as a "match". 
+  -->
+  <requestHandler name="/analysis/document" 
+                  class="solr.DocumentAnalysisRequestHandler" 
+                  startup="lazy" />
+
+  <!-- Echo the request contents back to the client -->
+  <requestHandler name="/debug/dump" class="solr.DumpRequestHandler" >
+    <lst name="defaults">
+     <str name="echoParams">explicit</str> 
+     <str name="echoHandler">true</str>
+    </lst>
+  </requestHandler>
+  
+
+
+  <!-- Search Components
+
+       Search components are registered to SolrCore and used by 
+       instances of SearchHandler (which can access them by name)
+       
+       By default, the following components are available:
+       
+       <searchComponent name="query"     class="solr.QueryComponent" />
+       <searchComponent name="facet"     class="solr.FacetComponent" />
+       <searchComponent name="mlt"       class="solr.MoreLikeThisComponent" />
+       <searchComponent name="highlight" class="solr.HighlightComponent" />
+       <searchComponent name="stats"     class="solr.StatsComponent" />
+       <searchComponent name="debug"     class="solr.DebugComponent" />
+       
+     -->
+
+  <!-- Terms Component
+
+       http://wiki.apache.org/solr/TermsComponent
+
+       A component to return terms and document frequency of those
+       terms
+    -->
+  <searchComponent name="terms" class="solr.TermsComponent"/>
+
+  <!-- A request handler for demonstrating the terms component -->
+  <requestHandler name="/terms" class="solr.SearchHandler" startup="lazy">
+     <lst name="defaults">
+      <bool name="terms">true</bool>
+      <bool name="distrib">false</bool>
+    </lst>     
+    <arr name="components">
+      <str>terms</str>
+    </arr>
+  </requestHandler>
+
+  <!-- Legacy config for the admin interface -->
+  <admin>
+    <defaultQuery>*:*</defaultQuery>
+  </admin>
+
+</config>

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/e59b1a31/metron-streaming/Metron-Solr/src/test/resources/solr/conf/stopwords.txt
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-Solr/src/test/resources/solr/conf/stopwords.txt b/metron-streaming/Metron-Solr/src/test/resources/solr/conf/stopwords.txt
new file mode 100644
index 0000000..ae1e83e
--- /dev/null
+++ b/metron-streaming/Metron-Solr/src/test/resources/solr/conf/stopwords.txt
@@ -0,0 +1,14 @@
+# 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.

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/e59b1a31/metron-streaming/Metron-Solr/src/test/resources/solr/conf/synonyms.txt
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-Solr/src/test/resources/solr/conf/synonyms.txt b/metron-streaming/Metron-Solr/src/test/resources/solr/conf/synonyms.txt
new file mode 100644
index 0000000..7f72128
--- /dev/null
+++ b/metron-streaming/Metron-Solr/src/test/resources/solr/conf/synonyms.txt
@@ -0,0 +1,29 @@
+# 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.
+
+#-----------------------------------------------------------------------
+#some test synonym mappings unlikely to appear in real input text
+aaafoo => aaabar
+bbbfoo => bbbfoo bbbbar
+cccfoo => cccbar cccbaz
+fooaaa,baraaa,bazaaa
+
+# Some synonym groups specific to this example
+GB,gib,gigabyte,gigabytes
+MB,mib,megabyte,megabytes
+Television, Televisions, TV, TVs
+#notice we use "gib" instead of "GiB" so any WordDelimiterFilter coming
+#after us won't split it into two words.
+
+# Synonym mappings can be used for spelling correction too
+pixima => pixma
+

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/e59b1a31/metron-streaming/Metron-Solr/src/test/resources/solr/solr.xml
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-Solr/src/test/resources/solr/solr.xml b/metron-streaming/Metron-Solr/src/test/resources/solr/solr.xml
new file mode 100644
index 0000000..318ad09
--- /dev/null
+++ b/metron-streaming/Metron-Solr/src/test/resources/solr/solr.xml
@@ -0,0 +1,14 @@
+<solr>
+    <solrcloud>
+        <str name="host">${host:}</str>
+        <int name="hostPort">${jetty.port:8983}</int>
+        <str name="hostContext">${hostContext:solr}</str>
+        <int name="zkClientTimeout">${zkClientTimeout:15000}</int>
+        <bool name="genericCoreNodeNames">${genericCoreNodeNames:true}</bool>
+    </solrcloud>
+    <shardHandlerFactory name="shardHandlerFactory"
+                         class="HttpShardHandlerFactory">
+        <int name="socketTimeout">${socketTimeout:0}</int>
+        <int name="connTimeout">${connTimeout:0}</int>
+    </shardHandlerFactory>
+</solr>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/e59b1a31/metron-streaming/Metron-Testing/pom.xml
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-Testing/pom.xml b/metron-streaming/Metron-Testing/pom.xml
index 7498375..a842322 100644
--- a/metron-streaming/Metron-Testing/pom.xml
+++ b/metron-streaming/Metron-Testing/pom.xml
@@ -35,11 +35,7 @@
       <artifactId>junit</artifactId>
       <version>${global_junit_version}</version>
     </dependency>
-    <dependency>
-      <groupId>org.elasticsearch</groupId>
-      <artifactId>elasticsearch</artifactId>
-      <version>${global_elasticsearch_version}</version>
-    </dependency>
+
     <dependency>
       <groupId>org.apache.storm</groupId>
       <artifactId>flux-core</artifactId>
@@ -110,11 +106,6 @@
       <artifactId>Metron-MessageParsers</artifactId>
       <version>0.1BETA</version>
     </dependency>
-    <dependency>
-      <groupId>org.apache.metron</groupId>
-      <artifactId>Metron-MessageParsers</artifactId>
-      <version>0.1BETA</version>
-    </dependency>
   </dependencies>
 
   <build>

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/e59b1a31/metron-streaming/Metron-Testing/src/main/java/org/apache/metron/integration/BaseIntegrationTest.java
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-Testing/src/main/java/org/apache/metron/integration/BaseIntegrationTest.java b/metron-streaming/Metron-Testing/src/main/java/org/apache/metron/integration/BaseIntegrationTest.java
new file mode 100644
index 0000000..084d8c5
--- /dev/null
+++ b/metron-streaming/Metron-Testing/src/main/java/org/apache/metron/integration/BaseIntegrationTest.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.integration;
+
+import com.google.common.base.Function;
+import org.apache.metron.integration.util.integration.components.KafkaWithZKComponent;
+import org.apache.metron.util.SampleUtil;
+import org.apache.metron.utils.ConfigurationsUtils;
+
+import javax.annotation.Nullable;
+import java.util.List;
+import java.util.Properties;
+
+public abstract class BaseIntegrationTest {
+
+  protected KafkaWithZKComponent getKafkaComponent(final Properties topologyProperties, List<KafkaWithZKComponent.Topic> topics) {
+    return new KafkaWithZKComponent().withTopics(topics)
+            .withPostStartCallback(new Function<KafkaWithZKComponent, Void>() {
+              @Nullable
+              @Override
+              public Void apply(@Nullable KafkaWithZKComponent kafkaWithZKComponent) {
+                topologyProperties.setProperty("kafka.zk", kafkaWithZKComponent.getZookeeperConnect());
+                try {
+                  ConfigurationsUtils.uploadConfigsToZookeeper(SampleUtil.sampleConfigRoot, kafkaWithZKComponent.getZookeeperConnect());
+                } catch (Exception e) {
+                  throw new IllegalStateException(e);
+                }
+                return null;
+              }
+            });
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/e59b1a31/metron-streaming/Metron-Testing/src/main/java/org/apache/metron/integration/EnrichmentIntegrationTest.java
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-Testing/src/main/java/org/apache/metron/integration/EnrichmentIntegrationTest.java b/metron-streaming/Metron-Testing/src/main/java/org/apache/metron/integration/EnrichmentIntegrationTest.java
new file mode 100644
index 0000000..ee4077b
--- /dev/null
+++ b/metron-streaming/Metron-Testing/src/main/java/org/apache/metron/integration/EnrichmentIntegrationTest.java
@@ -0,0 +1,429 @@
+/**
+ * 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;
+
+import com.fasterxml.jackson.core.type.TypeReference;
+import com.google.common.base.Function;
+import com.google.common.base.Joiner;
+import com.google.common.base.Predicate;
+import com.google.common.base.Predicates;
+import com.google.common.base.Splitter;
+import com.google.common.collect.Iterables;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.client.HTableInterface;
+import org.apache.metron.Constants;
+import org.apache.metron.domain.Configurations;
+import org.apache.metron.hbase.TableProvider;
+import org.apache.metron.hbase.converters.threatintel.ThreatIntelKey;
+import org.apache.metron.hbase.converters.threatintel.ThreatIntelValue;
+import org.apache.metron.integration.util.TestUtils;
+import org.apache.metron.integration.util.UnitTestHelper;
+import org.apache.metron.integration.util.integration.ComponentRunner;
+import org.apache.metron.integration.util.integration.InMemoryComponent;
+import org.apache.metron.integration.util.integration.Processor;
+import org.apache.metron.integration.util.integration.components.FluxTopologyComponent;
+import org.apache.metron.integration.util.integration.components.KafkaWithZKComponent;
+import org.apache.metron.integration.util.mock.MockGeoAdapter;
+import org.apache.metron.integration.util.mock.MockHTable;
+import org.apache.metron.integration.util.threatintel.ThreatIntelHelper;
+import org.apache.metron.reference.lookup.LookupKV;
+import org.apache.metron.util.SampleUtil;
+import org.apache.metron.utils.JSONUtils;
+import org.junit.Assert;
+import org.junit.Test;
+
+import javax.annotation.Nullable;
+import java.io.File;
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+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 String fluxPath = "../Metron-Topologies/src/main/resources/Metron_Configs/topologies/enrichment/test.yaml";
+  protected String hdfsDir = "target/enrichmentIntegrationTest/hdfs";
+  private String sampleParsedPath = "../Metron-Testing/src/main/resources/sample/data/SampleParsed/YafExampleParsed";
+  private String sampleIndexedPath = "../Metron-Testing/src/main/resources/sample/data/SampleIndexed/YafIndexed";
+  private Map<String, String> sensorConfigs = new HashMap<>();
+
+  public static class Provider implements TableProvider, Serializable {
+    MockHTable.Provider  provider = new MockHTable.Provider();
+    @Override
+    public HTableInterface getTable(Configuration config, String tableName) throws IOException {
+      return provider.getTable(config, tableName);
+    }
+  }
+
+  public static void cleanHdfsDir(String hdfsDirStr) {
+    File hdfsDir = new File(hdfsDirStr);
+    Stack<File> fs = new Stack<>();
+    if(hdfsDir.exists()) {
+      fs.push(hdfsDir);
+      while(!fs.empty()) {
+        File f = fs.pop();
+        if (f.isDirectory()) {
+          for(File child : f.listFiles()) {
+            fs.push(child);
+          }
+        }
+        else {
+          if (f.getName().startsWith("enrichment") || f.getName().endsWith(".json")) {
+            f.delete();
+          }
+        }
+      }
+    }
+  }
+
+  public static List<Map<String, Object> > readDocsFromDisk(String hdfsDirStr) throws IOException {
+    List<Map<String, Object>> ret = new ArrayList<>();
+    File hdfsDir = new File(hdfsDirStr);
+    Stack<File> fs = new Stack<>();
+    if(hdfsDir.exists()) {
+      fs.push(hdfsDir);
+      while(!fs.empty()) {
+        File f = fs.pop();
+        if(f.isDirectory()) {
+          for (File child : f.listFiles()) {
+            fs.push(child);
+          }
+        }
+        else {
+          System.out.println("Processed " + f);
+          if (f.getName().startsWith("enrichment") || f.getName().endsWith(".json")) {
+            List<byte[]> data = TestUtils.readSampleData(f.getPath());
+            Iterables.addAll(ret, Iterables.transform(data, new Function<byte[], Map<String, Object>>() {
+              @Nullable
+              @Override
+              public Map<String, Object> apply(@Nullable byte[] bytes) {
+                String s = new String(bytes);
+                try {
+                  return JSONUtils.INSTANCE.load(s, new TypeReference<Map<String, Object>>() {
+                  });
+                } catch (IOException e) {
+                  throw new RuntimeException(e);
+                }
+              }
+            }));
+          }
+        }
+      }
+    }
+    return ret;
+  }
+
+
+  @Test
+  public void test() throws Exception {
+    cleanHdfsDir(hdfsDir);
+    final Configurations configurations = SampleUtil.getSampleConfigs();
+    final List<byte[]> inputMessages = TestUtils.readSampleData(sampleParsedPath);
+    final String cf = "cf";
+    final String trackerHBaseTable = "tracker";
+    final String ipThreatIntelTable = "ip_threat_intel";
+    final Properties topologyProperties = new Properties() {{
+      setProperty("org.apache.metron.enrichment.host.known_hosts", "[{\"ip\":\"10.1.128.236\", \"local\":\"YES\", \"type\":\"webserver\", \"asset_value\" : \"important\"},\n" +
+              "{\"ip\":\"10.1.128.237\", \"local\":\"UNKNOWN\", \"type\":\"unknown\", \"asset_value\" : \"important\"},\n" +
+              "{\"ip\":\"10.60.10.254\", \"local\":\"YES\", \"type\":\"printer\", \"asset_value\" : \"important\"},\n" +
+              "{\"ip\":\"10.0.2.15\", \"local\":\"YES\", \"type\":\"printer\", \"asset_value\" : \"important\"}]");
+      setProperty("hbase.provider.impl","" + Provider.class.getName());
+      setProperty("threat.intel.tracker.table", trackerHBaseTable);
+      setProperty("threat.intel.tracker.cf", cf);
+      setProperty("threat.intel.ip.table", ipThreatIntelTable);
+      setProperty("threat.intel.ip.cf", cf);
+      setProperty("index.hdfs.output", hdfsDir);
+    }};
+    setAdditionalProperties(topologyProperties);
+    final KafkaWithZKComponent kafkaComponent = getKafkaComponent(topologyProperties, new ArrayList<KafkaWithZKComponent.Topic>() {{
+      add(new KafkaWithZKComponent.Topic(Constants.ENRICHMENT_TOPIC, 1));
+    }});
+
+    //create MockHBaseTables
+    final MockHTable trackerTable = (MockHTable)MockHTable.Provider.addToCache(trackerHBaseTable, cf);
+    final MockHTable ipTable = (MockHTable)MockHTable.Provider.addToCache(ipThreatIntelTable, cf);
+    ThreatIntelHelper.INSTANCE.load(ipTable, cf, new ArrayList<LookupKV<ThreatIntelKey, ThreatIntelValue>>(){{
+      add(new LookupKV<>(new ThreatIntelKey("10.0.2.3"), new ThreatIntelValue(new HashMap<String, String>())));
+    }});
+
+    FluxTopologyComponent fluxComponent = new FluxTopologyComponent.Builder()
+            .withTopologyLocation(new File(fluxPath))
+            .withTopologyName("test")
+            .withTopologyProperties(topologyProperties)
+            .build();
+
+    InMemoryComponent searchComponent = getSearchComponent(topologyProperties);
+
+    UnitTestHelper.verboseLogging();
+    ComponentRunner runner = new ComponentRunner.Builder()
+            .withComponent("kafka", kafkaComponent)
+            .withComponent("search", searchComponent)
+            .withComponent("storm", fluxComponent)
+            .withMillisecondsBetweenAttempts(10000)
+            .withNumRetries(10)
+            .build();
+    runner.start();
+
+    try {
+      fluxComponent.submitTopology();
+
+      kafkaComponent.writeMessages(Constants.ENRICHMENT_TOPIC, inputMessages);
+      List<Map<String, Object>> docs = runner.process(getProcessor(inputMessages));
+      Assert.assertEquals(inputMessages.size(), docs.size());
+      List<Map<String, Object>> cleanedDocs = cleanDocs(docs);
+      validateAll(cleanedDocs);
+
+      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");
+      validateAll(docsFromDisk);
+    }
+    finally {
+      cleanHdfsDir(hdfsDir);
+      runner.stop();
+    }
+  }
+
+  public List<Map<String, Object>> cleanDocs(List<Map<String, Object>> docs) {
+    List<Map<String, Object>> cleanedDocs = new ArrayList<>();
+    for(Map<String, Object> doc: docs) {
+      Map<String, Object> cleanedFields = new HashMap<>();
+      for(String field: doc.keySet()) {
+        cleanedFields.put(cleanField(field), doc.get(field));
+      }
+      cleanedDocs.add(cleanedFields);
+    }
+    return cleanedDocs;
+  }
+
+  public static void validateAll(List<Map<String, Object>> docs) {
+    for (Map<String, Object> doc : docs) {
+      baseValidation(doc);
+      hostEnrichmentValidation(doc);
+      geoEnrichmentValidation(doc);
+      threatIntelValidation(doc);
+    }
+  }
+
+  public static void baseValidation(Map<String, Object> jsonDoc) {
+    assertEnrichmentsExists("threatintels.", setOf("ip"), jsonDoc.keySet());
+    assertEnrichmentsExists("enrichments.", setOf("geo", "host"), jsonDoc.keySet());
+    for(Map.Entry<String, Object> kv : jsonDoc.entrySet()) {
+      //ensure no values are empty.
+      Assert.assertTrue(kv.getValue().toString().length() > 0);
+    }
+    //ensure we always have a source ip and destination ip
+    Assert.assertNotNull(jsonDoc.get(SRC_IP));
+    Assert.assertNotNull(jsonDoc.get(DST_IP));
+  }
+
+  private static class EvaluationPayload {
+    Map<String, Object> indexedDoc;
+    String key;
+    public EvaluationPayload(Map<String, Object> indexedDoc, String key) {
+      this.indexedDoc = indexedDoc;
+      this.key = key;
+    }
+  }
+
+  private static enum HostEnrichments implements Predicate<EvaluationPayload>{
+    LOCAL_LOCATION(new Predicate<EvaluationPayload>() {
+
+      @Override
+      public boolean apply(@Nullable EvaluationPayload evaluationPayload) {
+        return evaluationPayload.indexedDoc.get("enrichments.host." + evaluationPayload.key + ".known_info.local").equals("YES");
+      }
+    })
+    ,UNKNOWN_LOCATION(new Predicate<EvaluationPayload>() {
+
+      @Override
+      public boolean apply(@Nullable EvaluationPayload evaluationPayload) {
+        return evaluationPayload.indexedDoc.get("enrichments.host." + evaluationPayload.key + ".known_info.local").equals("UNKNOWN");
+      }
+    })
+    ,IMPORTANT(new Predicate<EvaluationPayload>() {
+      @Override
+      public boolean apply(@Nullable EvaluationPayload evaluationPayload) {
+        return evaluationPayload.indexedDoc.get("enrichments.host." + evaluationPayload.key + ".known_info.asset_value").equals("important");
+      }
+    })
+    ,PRINTER_TYPE(new Predicate<EvaluationPayload>() {
+      @Override
+      public boolean apply(@Nullable EvaluationPayload evaluationPayload) {
+        return evaluationPayload.indexedDoc.get("enrichments.host." + evaluationPayload.key + ".known_info.type").equals("printer");
+      }
+    })
+    ,WEBSERVER_TYPE(new Predicate<EvaluationPayload>() {
+      @Override
+      public boolean apply(@Nullable EvaluationPayload evaluationPayload) {
+        return evaluationPayload.indexedDoc.get("enrichments.host." + evaluationPayload.key + ".known_info.type").equals("webserver");
+      }
+    })
+    ,UNKNOWN_TYPE(new Predicate<EvaluationPayload>() {
+      @Override
+      public boolean apply(@Nullable EvaluationPayload evaluationPayload) {
+        return evaluationPayload.indexedDoc.get("enrichments.host." + evaluationPayload.key + ".known_info.type").equals("unknown");
+      }
+    })
+    ;
+
+    Predicate<EvaluationPayload> _predicate;
+    HostEnrichments(Predicate<EvaluationPayload> predicate) {
+      this._predicate = predicate;
+    }
+
+    public boolean apply(EvaluationPayload payload) {
+      return _predicate.apply(payload);
+    }
+
+  }
+
+  private static void assertEnrichmentsExists(String topLevel, Set<String> expectedEnrichments, Set<String> keys) {
+    for(String key : keys) {
+      if(key.startsWith(topLevel)) {
+        String secondLevel = Iterables.get(Splitter.on(".").split(key), 1);
+        String message = "Found an enrichment/threat intel (" + secondLevel + ") that I didn't expect (expected enrichments :"
+                       + Joiner.on(",").join(expectedEnrichments) + "), but it was not there.  If you've created a new"
+                       + " enrichment, then please add a validation method to this unit test.  Otherwise, it's a solid error"
+                       + " and should be investigated.";
+        Assert.assertTrue( message, expectedEnrichments.contains(secondLevel));
+      }
+    }
+  }
+  private static void threatIntelValidation(Map<String, Object> indexedDoc) {
+    if(keyPatternExists("threatintels.", indexedDoc)) {
+      //if we have any threat intel messages, we want to tag is_alert to true
+      Assert.assertEquals(indexedDoc.get("is_alert"), "true");
+    }
+    else {
+      //For YAF this is the case, but if we do snort later on, this will be invalid.
+      Assert.assertNull(indexedDoc.get("is_alert"));
+    }
+    //ip threat intels
+    if(keyPatternExists("threatintels.ip.", indexedDoc)) {
+      if(indexedDoc.get(SRC_IP).equals("10.0.2.3")) {
+        Assert.assertEquals(indexedDoc.get("threatintels.ip." + SRC_IP + ".ip_threat_intel"), "alert");
+      }
+      else if(indexedDoc.get(DST_IP).equals("10.0.2.3")) {
+        Assert.assertEquals(indexedDoc.get("threatintels.ip." + DST_IP + ".ip_threat_intel"), "alert");
+      }
+      else {
+        Assert.fail("There was a threat intels that I did not expect.");
+      }
+    }
+
+  }
+
+  private static void geoEnrichmentValidation(Map<String, Object> indexedDoc) {
+    //should have geo enrichment on every message due to mock geo adapter
+    Assert.assertEquals(indexedDoc.get("enrichments.geo." + DST_IP + ".location_point"), MockGeoAdapter.DEFAULT_LOCATION_POINT);
+    Assert.assertEquals(indexedDoc.get("enrichments.geo." + SRC_IP +".location_point"), MockGeoAdapter.DEFAULT_LOCATION_POINT);
+    Assert.assertEquals(indexedDoc.get("enrichments.geo." + DST_IP + ".longitude"), MockGeoAdapter.DEFAULT_LONGITUDE);
+    Assert.assertEquals(indexedDoc.get("enrichments.geo." + SRC_IP + ".longitude"), MockGeoAdapter.DEFAULT_LONGITUDE);
+    Assert.assertEquals(indexedDoc.get("enrichments.geo." + DST_IP + ".city"), MockGeoAdapter.DEFAULT_CITY);
+    Assert.assertEquals(indexedDoc.get("enrichments.geo." + SRC_IP + ".city"), MockGeoAdapter.DEFAULT_CITY);
+    Assert.assertEquals(indexedDoc.get("enrichments.geo." + DST_IP + ".latitude"), MockGeoAdapter.DEFAULT_LATITUDE);
+    Assert.assertEquals(indexedDoc.get("enrichments.geo." + SRC_IP + ".latitude"), MockGeoAdapter.DEFAULT_LATITUDE);
+    Assert.assertEquals(indexedDoc.get("enrichments.geo." + DST_IP + ".country"), MockGeoAdapter.DEFAULT_COUNTRY);
+    Assert.assertEquals(indexedDoc.get("enrichments.geo." + SRC_IP + ".country"), MockGeoAdapter.DEFAULT_COUNTRY);
+    Assert.assertEquals(indexedDoc.get("enrichments.geo." + DST_IP + ".dmaCode"), MockGeoAdapter.DEFAULT_DMACODE);
+    Assert.assertEquals(indexedDoc.get("enrichments.geo." + SRC_IP + ".dmaCode"), MockGeoAdapter.DEFAULT_DMACODE);
+    Assert.assertEquals(indexedDoc.get("enrichments.geo." + DST_IP + ".postalCode"), MockGeoAdapter.DEFAULT_POSTAL_CODE);
+    Assert.assertEquals(indexedDoc.get("enrichments.geo." + SRC_IP + ".postalCode"), MockGeoAdapter.DEFAULT_POSTAL_CODE);
+  }
+
+  private static void hostEnrichmentValidation(Map<String, Object> indexedDoc) {
+    boolean enriched = false;
+    //important local printers
+    {
+      Set<String> ips = setOf("10.0.2.15", "10.60.10.254");
+      if (ips.contains(indexedDoc.get(SRC_IP))) {
+        //this is a local, important, printer
+        Assert.assertTrue(Predicates.and(HostEnrichments.LOCAL_LOCATION
+                ,HostEnrichments.IMPORTANT
+                ,HostEnrichments.PRINTER_TYPE
+                ).apply(new EvaluationPayload(indexedDoc, SRC_IP))
+        );
+        enriched = true;
+      }
+      if (ips.contains(indexedDoc.get(DST_IP))) {
+        Assert.assertTrue(Predicates.and(HostEnrichments.LOCAL_LOCATION
+                ,HostEnrichments.IMPORTANT
+                ,HostEnrichments.PRINTER_TYPE
+                ).apply(new EvaluationPayload(indexedDoc, DST_IP))
+        );
+        enriched = true;
+      }
+    }
+    //important local webservers
+    {
+      Set<String> ips = setOf("10.1.128.236");
+      if (ips.contains(indexedDoc.get(SRC_IP))) {
+        //this is a local, important, printer
+        Assert.assertTrue(Predicates.and(HostEnrichments.LOCAL_LOCATION
+                ,HostEnrichments.IMPORTANT
+                ,HostEnrichments.WEBSERVER_TYPE
+                ).apply(new EvaluationPayload(indexedDoc, SRC_IP))
+        );
+        enriched = true;
+      }
+      if (ips.contains(indexedDoc.get(DST_IP))) {
+        Assert.assertTrue(Predicates.and(HostEnrichments.LOCAL_LOCATION
+                ,HostEnrichments.IMPORTANT
+                ,HostEnrichments.WEBSERVER_TYPE
+                ).apply(new EvaluationPayload(indexedDoc, DST_IP))
+        );
+        enriched = true;
+      }
+    }
+    if(!enriched) {
+      Assert.assertFalse(keyPatternExists("enrichments.host", indexedDoc));
+    }
+  }
+
+
+  private static boolean keyPatternExists(String pattern, Map<String, Object> indexedObj) {
+    for(String k : indexedObj.keySet()) {
+      if(k.startsWith(pattern)) {
+        return true;
+      }
+    }
+    return false;
+  }
+  private static Set<String> setOf(String... items) {
+    Set<String> ret = new HashSet<>();
+    for(String item : items) {
+      ret.add(item);
+    }
+    return ret;
+  }
+
+  abstract InMemoryComponent getSearchComponent(Properties topologyProperties) throws Exception;
+  abstract Processor<List<Map<String, Object>>> getProcessor(List<byte[]> inputMessages);
+  abstract void setAdditionalProperties(Properties topologyProperties);
+  abstract String cleanField(String field);
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/e59b1a31/metron-streaming/Metron-Testing/src/main/java/org/apache/metron/integration/util/TestUtils.java
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-Testing/src/main/java/org/apache/metron/integration/util/TestUtils.java b/metron-streaming/Metron-Testing/src/main/java/org/apache/metron/integration/util/TestUtils.java
new file mode 100644
index 0000000..4197169
--- /dev/null
+++ b/metron-streaming/Metron-Testing/src/main/java/org/apache/metron/integration/util/TestUtils.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.integration.util;
+
+import java.io.BufferedReader;
+import java.io.FileReader;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+public class TestUtils {
+
+  public static List<byte[]> readSampleData(String samplePath) throws IOException {
+    BufferedReader br = new BufferedReader(new FileReader(samplePath));
+    List<byte[]> ret = new ArrayList<>();
+    for (String line = null; (line = br.readLine()) != null; ) {
+      ret.add(line.getBytes());
+    }
+    br.close();
+    return ret;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/e59b1a31/metron-streaming/Metron-Testing/src/main/java/org/apache/metron/integration/util/integration/components/ElasticSearchComponent.java
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-Testing/src/main/java/org/apache/metron/integration/util/integration/components/ElasticSearchComponent.java b/metron-streaming/Metron-Testing/src/main/java/org/apache/metron/integration/util/integration/components/ElasticSearchComponent.java
deleted file mode 100644
index 42d7a08..0000000
--- a/metron-streaming/Metron-Testing/src/main/java/org/apache/metron/integration/util/integration/components/ElasticSearchComponent.java
+++ /dev/null
@@ -1,188 +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.integration.util.integration.components;
-
-import org.apache.commons.io.FileUtils;
-import org.apache.metron.integration.util.integration.InMemoryComponent;
-import org.apache.metron.integration.util.integration.UnableToStartException;
-import org.elasticsearch.ElasticsearchTimeoutException;
-import org.elasticsearch.action.admin.cluster.health.ClusterHealthAction;
-import org.elasticsearch.action.admin.cluster.health.ClusterHealthRequest;
-import org.elasticsearch.action.admin.cluster.health.ClusterHealthResponse;
-import org.elasticsearch.action.admin.cluster.health.ClusterHealthStatus;
-import org.elasticsearch.action.admin.indices.refresh.RefreshRequest;
-import org.elasticsearch.action.admin.indices.stats.IndicesStatsRequest;
-import org.elasticsearch.action.search.SearchResponse;
-import org.elasticsearch.client.Client;
-import org.elasticsearch.client.ElasticsearchClient;
-import org.elasticsearch.client.transport.TransportClient;
-import org.elasticsearch.common.settings.ImmutableSettings;
-import org.elasticsearch.common.settings.Settings;
-import org.elasticsearch.common.transport.InetSocketTransportAddress;
-import org.elasticsearch.common.unit.TimeValue;
-import org.elasticsearch.node.Node;
-import org.elasticsearch.node.NodeBuilder;
-import org.elasticsearch.search.SearchHit;
-
-import java.io.File;
-import java.io.IOException;
-import java.nio.file.Files;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-
-public class ElasticSearchComponent implements InMemoryComponent {
-
-    public static class Builder{
-        private int httpPort;
-        private File indexDir;
-        private Map<String, String> extraElasticSearchSettings = null;
-        public Builder withHttpPort(int httpPort) {
-            this.httpPort = httpPort;
-            return this;
-        }
-        public Builder withIndexDir(File indexDir) {
-            this.indexDir = indexDir;
-            return this;
-        }
-        public Builder withExtraElasticSearchSettings(Map<String, String> extraElasticSearchSettings) {
-            this.extraElasticSearchSettings = extraElasticSearchSettings;
-            return this;
-        }
-        public ElasticSearchComponent build() {
-            return new ElasticSearchComponent(httpPort, indexDir, extraElasticSearchSettings);
-        }
-    }
-
-    private Client client;
-    private Node node;
-    private int httpPort;
-    private File indexDir;
-    private Map<String, String> extraElasticSearchSettings;
-
-    public ElasticSearchComponent(int httpPort, File indexDir) {
-        this(httpPort, indexDir, null);
-    }
-    public ElasticSearchComponent(int httpPort, File indexDir, Map<String, String> extraElasticSearchSettings) {
-        this.httpPort = httpPort;
-        this.indexDir = indexDir;
-        this.extraElasticSearchSettings = extraElasticSearchSettings;
-    }
-    public Client getClient() {
-        return client;
-    }
-
-    private void cleanDir(File dir) throws IOException {
-        if(dir.exists()) {
-            FileUtils.deleteDirectory(dir);
-        }
-        dir.mkdirs();
-    }
-    public void start() throws UnableToStartException {
-        File logDir= new File(indexDir, "/logs");
-        File dataDir= new File(indexDir, "/data");
-        try {
-            cleanDir(logDir);
-            cleanDir(dataDir);
-
-        } catch (IOException e) {
-            throw new UnableToStartException("Unable to clean log or data directories", e);
-        }
-        ImmutableSettings.Builder immutableSettings = ImmutableSettings.settingsBuilder()
-                .put("node.http.enabled", true)
-                .put("http.port", httpPort)
-                .put("cluster.name", "metron")
-                .put("path.logs",logDir.getAbsolutePath())
-                .put("path.data",dataDir.getAbsolutePath())
-                .put("gateway.type", "none")
-                .put("index.store.type", "memory")
-                .put("index.number_of_shards", 1)
-                .put("node.mode", "network")
-                .put("index.number_of_replicas", 1);
-        if(extraElasticSearchSettings != null) {
-            immutableSettings = immutableSettings.put(extraElasticSearchSettings);
-        }
-        Settings settings = immutableSettings.build();
-
-        node = NodeBuilder.nodeBuilder().settings(settings).node();
-        node.start();
-        settings = ImmutableSettings.settingsBuilder()
-					.put("cluster.name", "metron").build();
-		client = new TransportClient(settings)
-					.addTransportAddress(new InetSocketTransportAddress("localhost",
-							9300));
-
-        waitForCluster(client, ClusterHealthStatus.YELLOW, new TimeValue(60000));
-    }
-
-    public static void waitForCluster(ElasticsearchClient client, ClusterHealthStatus status, TimeValue timeout) throws UnableToStartException {
-        try {
-            ClusterHealthResponse healthResponse =
-                    (ClusterHealthResponse)client.execute(ClusterHealthAction.INSTANCE, new ClusterHealthRequest().waitForStatus(status).timeout(timeout)).actionGet();
-            if (healthResponse != null && healthResponse.isTimedOut()) {
-                throw new UnableToStartException("cluster state is " + healthResponse.getStatus().name()
-                        + " and not " + status.name()
-                        + ", from here on, everything will fail!");
-            }
-        } catch (ElasticsearchTimeoutException e) {
-            throw new UnableToStartException("timeout, cluster does not respond to health request, cowardly refusing to continue with operations");
-        }
-    }
-
-    public List<Map<String, Object>> getAllIndexedDocs(String index, String sourceType) throws IOException {
-       return getAllIndexedDocs(index, sourceType, null);
-    }
-    public List<Map<String, Object>> getAllIndexedDocs(String index, String sourceType, String subMessage) throws IOException {
-        getClient().admin().indices().refresh(new RefreshRequest());
-        SearchResponse response = getClient().prepareSearch(index)
-                .setTypes(sourceType)
-                .setSource("message")
-                .setFrom(0)
-                .setSize(1000)
-                .execute().actionGet();
-        List<Map<String, Object>> ret = new ArrayList<Map<String, Object>>();
-        for (SearchHit hit : response.getHits()) {
-            Object o = null;
-            if(subMessage == null) {
-                o = hit.getSource();
-            }
-            else {
-                o = hit.getSource().get(subMessage);
-            }
-            ret.add((Map<String, Object>)(o));
-        }
-        return ret;
-    }
-    public boolean hasIndex(String indexName) {
-        Set<String> indices = getClient().admin()
-                                    .indices()
-                                    .stats(new IndicesStatsRequest())
-                                    .actionGet()
-                                    .getIndices()
-                                    .keySet();
-        return indices.contains(indexName);
-
-    }
-
-    public void stop() {
-        node.stop();
-        node = null;
-        client = null;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/e59b1a31/metron-streaming/Metron-Testing/src/main/java/org/apache/metron/integration/util/mock/MockGeoAdapter.java
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-Testing/src/main/java/org/apache/metron/integration/util/mock/MockGeoAdapter.java b/metron-streaming/Metron-Testing/src/main/java/org/apache/metron/integration/util/mock/MockGeoAdapter.java
new file mode 100644
index 0000000..ee71cda
--- /dev/null
+++ b/metron-streaming/Metron-Testing/src/main/java/org/apache/metron/integration/util/mock/MockGeoAdapter.java
@@ -0,0 +1,63 @@
+/**
+ * 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.util.mock;
+
+import com.google.common.base.Joiner;
+import org.apache.metron.enrichment.interfaces.EnrichmentAdapter;
+import org.json.simple.JSONObject;
+
+import java.io.Serializable;
+
+public class MockGeoAdapter implements EnrichmentAdapter<String>,
+        Serializable {
+
+  public static final String DEFAULT_LOC_ID = "1";
+  public static final String DEFAULT_COUNTRY = "test country";
+  public static final String DEFAULT_CITY = "test city";
+  public static final String DEFAULT_POSTAL_CODE = "test postalCode";
+  public static final String DEFAULT_LATITUDE = "test latitude";
+  public static final String DEFAULT_LONGITUDE = "test longitude";
+  public static final String DEFAULT_DMACODE= "test dmaCode";
+  public static final String DEFAULT_LOCATION_POINT= Joiner.on(',').join(DEFAULT_LONGITUDE, DEFAULT_LATITUDE);
+
+  @Override
+  public void logAccess(String value) {
+
+  }
+
+  public JSONObject enrich(String metadata) {
+    JSONObject enriched = new JSONObject();
+    enriched.put("locID", DEFAULT_LOC_ID);
+    enriched.put("country", DEFAULT_COUNTRY);
+    enriched.put("city", DEFAULT_CITY);
+    enriched.put("postalCode", DEFAULT_POSTAL_CODE);
+    enriched.put("latitude", DEFAULT_LATITUDE);
+    enriched.put("longitude", DEFAULT_LONGITUDE);
+    enriched.put("dmaCode", DEFAULT_DMACODE);
+    enriched.put("location_point", DEFAULT_LOCATION_POINT);
+    return enriched;
+  }
+
+  public boolean initializeAdapter() {
+    return true;
+  }
+
+  public void cleanup() {
+
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/e59b1a31/metron-streaming/Metron-Testing/src/main/java/org/apache/metron/integration/util/mock/MockHBaseConnector.java
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-Testing/src/main/java/org/apache/metron/integration/util/mock/MockHBaseConnector.java b/metron-streaming/Metron-Testing/src/main/java/org/apache/metron/integration/util/mock/MockHBaseConnector.java
new file mode 100644
index 0000000..b6b4a9d
--- /dev/null
+++ b/metron-streaming/Metron-Testing/src/main/java/org/apache/metron/integration/util/mock/MockHBaseConnector.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.integration.util.mock;
+
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.RetriesExhaustedWithDetailsException;
+import org.apache.metron.hbase.Connector;
+import org.apache.metron.hbase.TupleTableConfig;
+
+import java.io.IOException;
+import java.io.InterruptedIOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+public class MockHBaseConnector extends Connector {
+    static List<Put> puts = Collections.synchronizedList(new ArrayList<Put>());
+    public MockHBaseConnector(TupleTableConfig conf, String _quorum, String _port) throws IOException {
+        super(conf, _quorum, _port);
+    }
+
+    @Override
+    public void put(Put put) throws InterruptedIOException, RetriesExhaustedWithDetailsException {
+        puts.add(put);
+    }
+
+    @Override
+    public void close() {
+
+    }
+    public static void clear() {
+        puts.clear();
+    }
+    public static List<Put> getPuts() {
+        return puts;
+    }
+}


[06/15] incubator-metron git commit: METRON 86: Adding Solr indexing support (merrimanr via cestella) closes apache/incubator-metron#67

Posted by ce...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/e59b1a31/metron-streaming/Metron-Topologies/src/main/bash/zk_load_configs.sh
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-Topologies/src/main/bash/zk_load_configs.sh b/metron-streaming/Metron-Topologies/src/main/bash/zk_load_configs.sh
index 4273b7d..7a6d471 100755
--- a/metron-streaming/Metron-Topologies/src/main/bash/zk_load_configs.sh
+++ b/metron-streaming/Metron-Topologies/src/main/bash/zk_load_configs.sh
@@ -30,4 +30,4 @@ export METRON_VERSION=0.1BETA
 export METRON_HOME=/usr/metron/$METRON_VERSION
 export TOPOLOGIES_JAR=Metron-Topologies-$METRON_VERSION.jar
 export ZK_HOME=${ZK_HOME:-/usr/hdp/current/hbase-client}
-java -cp $METRON_HOME/lib/$TOPOLOGIES_JAR org.apache.metron.utils.SourceConfigUtils "$@"
+java -cp $METRON_HOME/lib/$TOPOLOGIES_JAR org.apache.metron.utils.ConfigurationsUtils "$@"
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/e59b1a31/metron-streaming/Metron-Topologies/src/main/java/org/apache/metron/utils/SourceConfigUtils.java
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-Topologies/src/main/java/org/apache/metron/utils/SourceConfigUtils.java b/metron-streaming/Metron-Topologies/src/main/java/org/apache/metron/utils/SourceConfigUtils.java
deleted file mode 100644
index 127ca1f..0000000
--- a/metron-streaming/Metron-Topologies/src/main/java/org/apache/metron/utils/SourceConfigUtils.java
+++ /dev/null
@@ -1,143 +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.utils;
-
-import org.apache.commons.cli.*;
-import org.apache.curator.RetryPolicy;
-import org.apache.curator.framework.CuratorFramework;
-import org.apache.curator.framework.CuratorFrameworkFactory;
-import org.apache.curator.retry.ExponentialBackoffRetry;
-import org.apache.metron.Constants;
-import org.apache.metron.domain.SourceConfig;
-import org.apache.zookeeper.KeeperException;
-
-import java.io.ByteArrayInputStream;
-import java.io.File;
-import java.io.PrintWriter;
-import java.nio.file.Files;
-import java.nio.file.Paths;
-import java.util.List;
-
-public class SourceConfigUtils {
-
-  public static CuratorFramework getClient(String zookeeperUrl) {
-    RetryPolicy retryPolicy = new ExponentialBackoffRetry(1000, 3);
-    return CuratorFrameworkFactory.newClient(zookeeperUrl, retryPolicy);
-  }
-
-  public static void writeToZookeeperFromFile(String sourceName, String filePath, String zookeeperUrl) throws Exception {
-    writeToZookeeper(sourceName, Files.readAllBytes(Paths.get(filePath)), zookeeperUrl);
-  }
-
-  public static void writeToZookeeper(String sourceName, byte[] configData, String zookeeperUrl) throws Exception {
-    CuratorFramework client = getClient(zookeeperUrl);
-    client.start();
-    try {
-      client.setData().forPath(Constants.ZOOKEEPER_TOPOLOGY_ROOT + "/" + sourceName, configData);
-    } catch(KeeperException.NoNodeException e) {
-      client.create().creatingParentsIfNeeded().forPath(Constants.ZOOKEEPER_TOPOLOGY_ROOT + "/" + sourceName, configData);
-    }
-    client.close();
-  }
-
-  public static byte[] readConfigBytesFromZookeeper(String sourceName, String zookeeperUrl) throws Exception {
-    CuratorFramework client = getClient(zookeeperUrl);
-    client.start();
-    byte[] data = client.getData().forPath(Constants.ZOOKEEPER_TOPOLOGY_ROOT + "/" + sourceName);
-    client.close();
-    return data;
-  }
-
-  public static SourceConfig readConfigFromZookeeper(String sourceName, String zookeeperUrl) throws Exception {
-    byte[] data = readConfigBytesFromZookeeper(sourceName, zookeeperUrl);
-    return SourceConfig.load(new ByteArrayInputStream(data));
-  }
-
-  public static void dumpConfigs(String zookeeperUrl) throws Exception {
-    CuratorFramework client = getClient(zookeeperUrl);
-    client.start();
-    List<String> children = client.getChildren().forPath(Constants.ZOOKEEPER_TOPOLOGY_ROOT);
-    for(String child: children) {
-      byte[] data = client.getData().forPath(Constants.ZOOKEEPER_TOPOLOGY_ROOT + "/" + child);
-      System.out.println("Config for source " + child);
-      System.out.println(new String(data));
-      System.out.println();
-    }
-    client.close();
-  }
-
-  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("p", "config_files", true, "Path to the source config files.  Must be named like \"$source\"-config.json");
-      o.setArgName("DIR_NAME");
-      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);
-    }
-
-    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("SourceConfigUtils", null, options, null, true);
-        System.exit(-1);
-      }
-      if( cmd.hasOption("h") ){
-        final HelpFormatter usageFormatter = new HelpFormatter();
-        usageFormatter.printHelp("SourceConfigUtils", null, options, null, true);
-        System.exit(0);
-      }
-
-      String zkQuorum = cmd.getOptionValue("z");
-      if(cmd.hasOption("p")) {
-        String sourcePath = cmd.getOptionValue("p");
-        File root = new File(sourcePath);
-
-        if (root.isDirectory()) {
-          for (File child : root.listFiles()) {
-            writeToZookeeperFromFile(child.getName().replaceFirst("-config.json", ""), child.getPath(), zkQuorum);
-          }
-        }
-      }
-
-      SourceConfigUtils.dumpConfigs(zkQuorum);
-
-    } catch (Exception e) {
-      e.printStackTrace();
-      System.exit(-1);
-    }
-
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/e59b1a31/metron-streaming/Metron-Topologies/src/main/resources/Metron_Configs/etc/env/config.properties
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-Topologies/src/main/resources/Metron_Configs/etc/env/config.properties b/metron-streaming/Metron-Topologies/src/main/resources/Metron_Configs/etc/env/config.properties
index 232f2be..8784689 100644
--- a/metron-streaming/Metron-Topologies/src/main/resources/Metron_Configs/etc/env/config.properties
+++ b/metron-streaming/Metron-Topologies/src/main/resources/Metron_Configs/etc/env/config.properties
@@ -29,6 +29,9 @@ spout.kafka.topic.pcap=pcap
 spout.kafka.topic.snort=snort
 spout.kafka.topic.yaf=yaf
 
+##### Indexing #####
+writer.class.name=
+
 ##### ElasticSearch #####
 
 es.ip=10.22.0.214

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/e59b1a31/metron-streaming/Metron-Topologies/src/main/resources/Metron_Configs/topologies/enrichment/remote.yaml
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-Topologies/src/main/resources/Metron_Configs/topologies/enrichment/remote.yaml b/metron-streaming/Metron-Topologies/src/main/resources/Metron_Configs/topologies/enrichment/remote.yaml
index eb2d2d6..5423bf1 100644
--- a/metron-streaming/Metron-Topologies/src/main/resources/Metron_Configs/topologies/enrichment/remote.yaml
+++ b/metron-streaming/Metron-Topologies/src/main/resources/Metron_Configs/topologies/enrichment/remote.yaml
@@ -131,12 +131,7 @@ components:
                     - ref: "hdfsRotationPolicy"
 
     -   id: "indexWriter"
-        className: "org.apache.metron.writer.ElasticsearchWriter"
-        constructorArgs:
-            - "${es.clustername}"
-            - "${es.ip}"
-            - ${es.port}
-            - "yyyy.MM.dd.hh"
+        className: "${writer.class.name}"
 
 #kafka/zookeeper
     -   id: "zkHosts"

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/e59b1a31/metron-streaming/Metron-Topologies/src/main/resources/Metron_Configs/topologies/enrichment/test.yaml
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-Topologies/src/main/resources/Metron_Configs/topologies/enrichment/test.yaml b/metron-streaming/Metron-Topologies/src/main/resources/Metron_Configs/topologies/enrichment/test.yaml
index e1e0ad1..1296cb3 100644
--- a/metron-streaming/Metron-Topologies/src/main/resources/Metron_Configs/topologies/enrichment/test.yaml
+++ b/metron-streaming/Metron-Topologies/src/main/resources/Metron_Configs/topologies/enrichment/test.yaml
@@ -103,12 +103,7 @@ components:
                 args:
                     - ref: "fileNameFormat"
     -   id: "indexWriter"
-        className: "org.apache.metron.writer.ElasticsearchWriter"
-        constructorArgs:
-            - "${es.clustername}"
-            - "${es.ip}"
-            - ${es.port}
-            - "${index.date.format}"
+        className: "${writer.class.name}"
 
 #kafka/zookeeper
     -   id: "zkHosts"
@@ -139,7 +134,7 @@ spouts:
         configMethods:
             -   name: "withFilename"
                 args:
-                    - "SampleInput/YafExampleOutput"
+                    - "../Metron-Testing/src/main/resources/sample/data/SampleInput/YafExampleOutput"
             -   name: "withRepeating"
                 args:
                     - true
@@ -332,7 +327,6 @@ streams:
             streamId: "message"
             type: FIELDS
             args: ["key"]
-
     -   name: "threatIntelJoin -> hdfs"
         from: "threatIntelJoinBolt"
         to: "hdfsIndexingBolt"

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/e59b1a31/metron-streaming/Metron-Topologies/src/main/resources/SampleIndexed/YafIndexed
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-Topologies/src/main/resources/SampleIndexed/YafIndexed b/metron-streaming/Metron-Topologies/src/main/resources/SampleIndexed/YafIndexed
deleted file mode 100644
index 1c38406..0000000
--- a/metron-streaming/Metron-Topologies/src/main/resources/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/e59b1a31/metron-streaming/Metron-Topologies/src/main/resources/SampleInput/.PCAPExampleOutput.crc
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-Topologies/src/main/resources/SampleInput/.PCAPExampleOutput.crc b/metron-streaming/Metron-Topologies/src/main/resources/SampleInput/.PCAPExampleOutput.crc
deleted file mode 100644
index 6e53497..0000000
Binary files a/metron-streaming/Metron-Topologies/src/main/resources/SampleInput/.PCAPExampleOutput.crc and /dev/null differ

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/e59b1a31/metron-streaming/Metron-Topologies/src/main/resources/SampleInput/AsaOutput
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-Topologies/src/main/resources/SampleInput/AsaOutput b/metron-streaming/Metron-Topologies/src/main/resources/SampleInput/AsaOutput
deleted file mode 100644
index 6009d48..0000000
--- a/metron-streaming/Metron-Topologies/src/main/resources/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


[15/15] incubator-metron git commit: METRON 86: Adding Solr indexing support (merrimanr via cestella) closes apache/incubator-metron#67

Posted by ce...@apache.org.
METRON 86: Adding Solr indexing support (merrimanr via cestella) closes apache/incubator-metron#67


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

Branch: refs/heads/master
Commit: e59b1a31da16a3bff5f00b0947cff899f4dc1d32
Parents: edeec01
Author: merrimanr <me...@gmail.com>
Authored: Tue Apr 5 15:41:31 2016 -0400
Committer: cstella <ce...@gmail.com>
Committed: Tue Apr 5 15:41:31 2016 -0400

----------------------------------------------------------------------
 deployment/amazon-ec2/conf/defaults.yml         |    12 +-
 .../inventory/metron_example/group_vars/all     |    11 +-
 .../inventory/multinode-vagrant/group_vars/all  |    32 +-
 .../inventory/singlenode-vagrant/group_vars/all |     6 +-
 deployment/playbooks/metron_install.yml         |    15 +-
 .../roles/metron_streaming/defaults/main.yml    |     6 +-
 .../files/config/sensors/bro.json               |    14 +
 .../files/config/sensors/pcap.json              |    14 +
 .../files/config/sensors/snort.json             |    14 +
 .../files/config/sensors/yaf.json               |    14 +
 .../files/source/bro-config.json                |    14 -
 .../files/source/pcap-config.json               |    14 -
 .../files/source/snort-config.json              |    14 -
 .../files/source/yaf-config.json                |    14 -
 .../roles/metron_streaming/handlers/main.yml    |     4 +-
 .../roles/metron_streaming/tasks/main.yml       |    59 +-
 .../metron_streaming/tasks/metron_topology.yml  |    13 +-
 .../metron_streaming/tasks/source_config.yml    |    29 +-
 .../templates/config/elasticsearch.global.json  |     6 +
 .../templates/config/solr.global.json           |     6 +
 deployment/roles/metron_streaming/vars/main.yml |    18 +-
 deployment/roles/solr/defaults/main.yml         |    29 +
 deployment/roles/solr/files/schema.xml          |   191 +
 deployment/roles/solr/meta/main.yml             |    21 +
 deployment/roles/solr/tasks/main.yml            |    74 +
 deployment/roles/solr/templates/solr.xml        |    52 +
 deployment/roles/solr/templates/solrconfig.xml  |   583 +
 metron-streaming/Metron-Common/pom.xml          |     6 +
 .../main/java/org/apache/metron/Constants.java  |     9 +-
 .../metron/bolt/BulkMessageWriterBolt.java      |    38 +-
 .../org/apache/metron/bolt/ConfiguredBolt.java  |    70 +-
 .../apache/metron/domain/Configurations.java    |    92 +
 .../metron/domain/SensorEnrichmentConfig.java   |    62 +
 .../org/apache/metron/domain/SourceConfig.java  |    88 -
 .../metron/helpers/topology/ErrorUtils.java     |     2 +-
 .../java/org/apache/metron/pcap/PcapUtils.java  |    12 +
 .../apache/metron/topology/TopologyUtils.java   |     4 +-
 .../metron/utils/ConfigurationsUtils.java       |   179 +
 .../java/org/apache/metron/utils/JSONUtils.java |     4 +
 .../org/apache/metron/writer/HBaseWriter.java   |     4 +-
 .../writer/interfaces/BulkMessageWriter.java    |     6 +-
 .../metron/writer/interfaces/MessageWriter.java |     4 +-
 .../org/apache/metron/pcap/PcapUtilsTest.java   |    31 +
 .../resources/config/source/bro-config.json     |    14 -
 .../resources/config/source/pcap-config.json    |    13 -
 .../resources/config/source/snort-config.json   |    14 -
 .../resources/config/source/yaf-config.json     |    14 -
 metron-streaming/Metron-Elasticsearch/pom.xml   |   202 +
 .../src/main/assembly/assembly.xml              |    41 +
 .../metron/writer/ElasticsearchWriter.java      |    94 +
 .../etc/env/elasticsearch.properties            |   109 +
 .../ElasticsearchEnrichmentIntegrationTest.java |    88 +
 .../components/ElasticSearchComponent.java      |   186 +
 .../enrichment/bolt/EnrichmentJoinBolt.java     |    12 +-
 .../enrichment/bolt/EnrichmentSplitterBolt.java |    11 +-
 .../enrichment/bolt/GenericEnrichmentBolt.java  |    25 +-
 .../enrichment/bolt/ThreatIntelJoinBolt.java    |     4 +-
 .../bolt/ThreatIntelSplitterBolt.java           |     4 +-
 metron-streaming/Metron-Indexing/pom.xml        |     5 -
 .../metron/indexing/AbstractIndexingBolt.java   |   110 -
 .../metron/indexing/TelemetryIndexingBolt.java  |   256 -
 .../indexing/adapters/AbstractIndexAdapter.java |    42 -
 .../indexing/adapters/ESBaseBulkAdapter.java    |   165 -
 .../adapters/ESBulkRotatingAdapter.java         |   177 -
 .../adapters/ESTimedRotatingAdapter.java        |   208 -
 .../metron/indexing/adapters/SolrAdapter.java   |    22 -
 .../metron/writer/ElasticsearchWriter.java      |   101 -
 .../apache/metron/writer/hdfs/HdfsWriter.java   |     6 +-
 .../java/org/apache/metron/bolt/ParserBolt.java |    10 +-
 .../org/apache/metron/writer/KafkaWriter.java   |     4 +-
 metron-streaming/Metron-Solr/pom.xml            |   204 +
 .../Metron-Solr/src/main/assembly/assembly.xml  |    41 +
 .../org/apache/metron/solr/SolrConstants.java   |    29 +
 .../metron/writer/solr/MetronSolrClient.java    |    72 +
 .../apache/metron/writer/solr/SolrWriter.java   |   108 +
 .../Metron_Configs/etc/env/solr.properties      |   109 +
 .../SolrEnrichmentIntegrationTest.java          |   107 +
 .../integration/components/SolrComponent.java   |   153 +
 .../writer/solr/MetronSolrClientTest.java       |    82 +
 .../metron/writer/solr/SolrWriterTest.java      |   139 +
 .../test/resources/solr/conf/_rest_managed.json |     1 +
 .../src/test/resources/solr/conf/currency.xml   |    67 +
 .../resources/solr/conf/lang/stopwords_en.txt   |    54 +
 .../src/test/resources/solr/conf/protwords.txt  |    21 +
 .../src/test/resources/solr/conf/schema.xml     |   191 +
 .../src/test/resources/solr/conf/solrconfig.xml |   583 +
 .../src/test/resources/solr/conf/stopwords.txt  |    14 +
 .../src/test/resources/solr/conf/synonyms.txt   |    29 +
 .../src/test/resources/solr/solr.xml            |    14 +
 metron-streaming/Metron-Testing/pom.xml         |    11 +-
 .../metron/integration/BaseIntegrationTest.java |    48 +
 .../integration/EnrichmentIntegrationTest.java  |   429 +
 .../metron/integration/util/TestUtils.java      |    37 +
 .../components/ElasticSearchComponent.java      |   188 -
 .../integration/util/mock/MockGeoAdapter.java   |    63 +
 .../util/mock/MockHBaseConnector.java           |    52 +
 .../util/threatintel/ThreatIntelHelper.java     |    39 +
 .../java/org/apache/metron/util/SampleUtil.java |    40 +
 .../main/resources/sample/config/global.json    |    10 +
 .../resources/sample/config/sensors/bro.json    |    14 +
 .../resources/sample/config/sensors/pcap.json   |    13 +
 .../resources/sample/config/sensors/snort.json  |    14 +
 .../resources/sample/config/sensors/yaf.json    |    14 +
 .../sample/data/SampleIndexed/YafIndexed        |    10 +
 .../data/SampleInput/.PCAPExampleOutput.crc     |   Bin 0 -> 44 bytes
 .../resources/sample/data/SampleInput/AsaOutput |   100 +
 .../sample/data/SampleInput/BroExampleOutput    | 23411 +++++++++++++++++
 .../data/SampleInput/FireeyeExampleOutput       |    90 +
 .../sample/data/SampleInput/ISESampleOutput     |   308 +
 .../data/SampleInput/LancopeExampleOutput       |    40 +
 .../sample/data/SampleInput/PCAPExampleOutput   |   Bin 0 -> 4510 bytes
 .../sample/data/SampleInput/PaloaltoOutput      |   100 +
 .../sample/data/SampleInput/SnortOutput         |     3 +
 .../data/SampleInput/SourcefireExampleOutput    |     2 +
 .../sample/data/SampleInput/YafExampleOutput    |    10 +
 .../sample/data/SampleParsed/SnortParsed        |     3 +
 .../sample/data/SampleParsed/YafExampleParsed   |    10 +
 metron-streaming/Metron-Topologies/pom.xml      |    20 +-
 .../src/main/assembly/assembly.xml              |     6 +-
 .../main/bash/start_elasticsearch_topology.sh   |    22 +
 .../src/main/bash/start_solr_topology.sh        |    22 +
 .../src/main/bash/start_topology.sh             |    22 -
 .../src/main/bash/zk_load_configs.sh            |     2 +-
 .../apache/metron/utils/SourceConfigUtils.java  |   143 -
 .../Metron_Configs/etc/env/config.properties    |     3 +
 .../topologies/enrichment/remote.yaml           |     7 +-
 .../topologies/enrichment/test.yaml             |    10 +-
 .../src/main/resources/SampleIndexed/YafIndexed |    10 -
 .../SampleInput/.PCAPExampleOutput.crc          |   Bin 44 -> 0 bytes
 .../src/main/resources/SampleInput/AsaOutput    |   100 -
 .../main/resources/SampleInput/BroExampleOutput | 23411 -----------------
 .../resources/SampleInput/FireeyeExampleOutput  |    90 -
 .../main/resources/SampleInput/ISESampleOutput  |   308 -
 .../resources/SampleInput/LancopeExampleOutput  |    40 -
 .../resources/SampleInput/PCAPExampleOutput     |   Bin 4510 -> 0 bytes
 .../main/resources/SampleInput/PaloaltoOutput   |   100 -
 .../src/main/resources/SampleInput/SnortOutput  |     3 -
 .../SampleInput/SourcefireExampleOutput         |     2 -
 .../main/resources/SampleInput/YafExampleOutput |    10 -
 .../src/main/resources/SampleParsed/SnortParsed |     3 -
 .../resources/SampleParsed/YafExampleParsed     |    10 -
 .../integration/EnrichmentIntegrationTest.java  |   469 -
 .../integration/ParserIntegrationTest.java      |    35 +-
 .../integration/PcapParserIntegrationTest.java  |    58 +-
 .../integration/SnortIntegrationTest.java       |    24 +-
 .../metron/integration/YafIntegrationTest.java  |    24 +-
 .../metron/integration/util/TestUtils.java      |    37 -
 .../integration/util/mock/MockGeoAdapter.java   |    63 -
 .../util/mock/MockHBaseConnector.java           |    52 -
 .../util/threatintel/ThreatIntelHelper.java     |    40 -
 metron-streaming/pom.xml                        |     8 +-
 pom.xml                                         |   134 +-
 152 files changed, 29457 insertions(+), 26735 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/e59b1a31/deployment/amazon-ec2/conf/defaults.yml
----------------------------------------------------------------------
diff --git a/deployment/amazon-ec2/conf/defaults.yml b/deployment/amazon-ec2/conf/defaults.yml
index abe96b8..77e841d 100644
--- a/deployment/amazon-ec2/conf/defaults.yml
+++ b/deployment/amazon-ec2/conf/defaults.yml
@@ -42,11 +42,6 @@ threatintel_ip_hbase_table: malicious_ip
 num_partitions: 3
 retention_in_gb: 25
 
-# elasticsearch
-elasticsearch_transport_port: 9300
-elasticsearch_network_interface: eth0
-elasticsearch_web_port: 9200
-
 # metron variables
 metron_version: 0.1BETA
 java_home: /usr/jdk64/jdk1.8.0_40
@@ -74,3 +69,10 @@ jhs_recovery_store_ldb_path: "/data1/hadoop/mapreduce/jhs"
 storm_local_dir: "/data1/hadoop/storm"
 kafka_log_dirs: "/data2/kafka-log"
 elasticsearch_data_dir: "/data1/elasticsearch,/data2/elasticsearch"
+
+#Search
+install_elasticsearch: True
+install_solr: False
+elasticsearch_transport_port: 9300
+elasticsearch_network_interface: eth1
+elasticsearch_web_port: 9200
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/e59b1a31/deployment/inventory/metron_example/group_vars/all
----------------------------------------------------------------------
diff --git a/deployment/inventory/metron_example/group_vars/all b/deployment/inventory/metron_example/group_vars/all
index 3a26769..c5d81b0 100644
--- a/deployment/inventory/metron_example/group_vars/all
+++ b/deployment/inventory/metron_example/group_vars/all
@@ -31,11 +31,6 @@ pcap_hbase_table: pcap
 tracker_hbase_table: access_tracker
 threatintel_ip_hbase_table: malicious_ip
 
-#elasticsearch
-elasticsearch_transport_port: 9300
-elasticsearch_network_interface: eth0
-elasticsearch_web_port: 9200
-
 # metron variables
 metron_version: 0.1BETA
 java_home: /usr/jdk64/jdk1.8.0_40
@@ -70,3 +65,9 @@ pcap_replay_interface: eth1
 #kafka_log_dirs: "/newdir/kafka-log"
 #elasticsearch_data_dir: "/newdir1/elasticsearch"
 
+#Search
+install_elasticsearch: True
+install_solr: False
+elasticsearch_transport_port: 9300
+elasticsearch_network_interface: eth1
+elasticsearch_web_port: 9200
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/e59b1a31/deployment/inventory/multinode-vagrant/group_vars/all
----------------------------------------------------------------------
diff --git a/deployment/inventory/multinode-vagrant/group_vars/all b/deployment/inventory/multinode-vagrant/group_vars/all
index fc3b56d..e486296 100644
--- a/deployment/inventory/multinode-vagrant/group_vars/all
+++ b/deployment/inventory/multinode-vagrant/group_vars/all
@@ -49,17 +49,23 @@ snort_version: "2.9.8.0-1"
 snort_alert_csv_path: "/var/log/snort/alert.csv"
 
 #data directories
-zookeeper_data_dir: "/newdir/hadoop/zookeeper"
-namenode_checkpoint_dir: "/newdir/hadoop/hdfs/namesecondary"
-namenode_name_dir: "/newdir/hadoop/hdfs/namenode"
-datanode_data_dir: "/newdir/hadoop/hdfs/data"
-journalnode_edits_dir: "/newdir/hadoop/hdfs/journalnode"
-nodemanager_local_dirs: "/newdir/hadoop/yarn/local"
-timeline_ldb_store_path: "/newdir/hadoop/yarn/timeline"
-timeline_ldb_state_path: "/newdir/hadoop/yarn/timeline"
-nodemanager_log_dirs: "/newdir/hadoop/yarn/log"
-jhs_recovery_store_ldb_path: "/newdir/hadoop/mapreduce/jhs"
-storm_local_dir: "/newdir/hadoop/storm"
-kafka_log_dirs: "/newdir/kafka-log"
-elasticsearch_data_dir: "/newdir1/elasticsearch"
+#zookeeper_data_dir: "/newdir/hadoop/zookeeper"
+#namenode_checkpoint_dir: "/newdir/hadoop/hdfs/namesecondary"
+#namenode_name_dir: "/newdir/hadoop/hdfs/namenode"
+#datanode_data_dir: "/newdir/hadoop/hdfs/data"
+#journalnode_edits_dir: "/newdir/hadoop/hdfs/journalnode"
+#nodemanager_local_dirs: "/newdir/hadoop/yarn/local"
+#timeline_ldb_store_path: "/newdir/hadoop/yarn/timeline"
+#timeline_ldb_state_path: "/newdir/hadoop/yarn/timeline"
+#nodemanager_log_dirs: "/newdir/hadoop/yarn/log"
+#jhs_recovery_store_ldb_path: "/newdir/hadoop/mapreduce/jhs"
+#storm_local_dir: "/newdir/hadoop/storm"
+#kafka_log_dirs: "/newdir/kafka-log"
+#elasticsearch_data_dir: "/newdir1/elasticsearch"
 
+#Search
+install_elasticsearch: True
+install_solr: False
+elasticsearch_transport_port: 9300
+elasticsearch_network_interface: eth1
+elasticsearch_web_port: 9200
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/e59b1a31/deployment/inventory/singlenode-vagrant/group_vars/all
----------------------------------------------------------------------
diff --git a/deployment/inventory/singlenode-vagrant/group_vars/all b/deployment/inventory/singlenode-vagrant/group_vars/all
index c1c199b..364598f 100644
--- a/deployment/inventory/singlenode-vagrant/group_vars/all
+++ b/deployment/inventory/singlenode-vagrant/group_vars/all
@@ -76,6 +76,10 @@ pcapservice_port: 8081
 
 #Search
 install_elasticsearch: True
+install_solr: False
+solr_collection_name: Metron
+solr_number_shards: 1
+solr_replication_factor: 1
 elasticsearch_transport_port: 9300
 elasticsearch_network_interface: eth1
-elasticsearch_web_port: 9200
+elasticsearch_web_port: 9200
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/e59b1a31/deployment/playbooks/metron_install.yml
----------------------------------------------------------------------
diff --git a/deployment/playbooks/metron_install.yml b/deployment/playbooks/metron_install.yml
index c8c181e..89db980 100644
--- a/deployment/playbooks/metron_install.yml
+++ b/deployment/playbooks/metron_install.yml
@@ -41,9 +41,16 @@
   vars:
     es_hosts: "{% set comma = joiner(',') %}{% for host in groups['search'] -%}{{ comma() }}{{ host }}{%- endfor %}"
   roles:
-    - role: elasticsearch
+    - { role: elasticsearch, when: install_elasticsearch | default(True) == True }
   tags:
-    - elasticsearch
+    - search
+
+- hosts: search
+  become: true
+  roles:
+    - { role: solr, when: install_solr | default(False) == True  }
+  tags:
+    - search
 
 - hosts: mysql
   become: true
@@ -82,7 +89,7 @@
 - hosts: web
   become: true
   roles:
-    - role: metron_ui
-    - role: metron_pcapservice
+    - { role: metron_ui, when: install_elasticsearch | default(True) == True }
+    - { role: metron_pcapservice, when: install_elasticsearch | default(True) == True }
   tags:
     - web

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/e59b1a31/deployment/roles/metron_streaming/defaults/main.yml
----------------------------------------------------------------------
diff --git a/deployment/roles/metron_streaming/defaults/main.yml b/deployment/roles/metron_streaming/defaults/main.yml
index 0dd21e3..a65680e 100644
--- a/deployment/roles/metron_streaming/defaults/main.yml
+++ b/deployment/roles/metron_streaming/defaults/main.yml
@@ -15,7 +15,9 @@
 #  limitations under the License.
 #
 ---
-source_config_path: "{{ metron_directory }}/config/source"
+config_path: "{{ metron_directory }}/config"
+zookeeper_config_path: "{{ config_path }}/zookeeper"
+zookeeper_global_config_path: "{{ zookeeper_config_path }}/global.json"
 threat_intel_bulk_load: True
 threat_intel_bin: "{{ metron_directory }}/bin/threatintel_bulk_load.sh"
 threat_intel_host: "{{ groups.ambari_master[0] }}"
@@ -43,4 +45,4 @@ hdfs_config_path: "/etc/hadoop/conf"
 metron_hdfs_output_dir: "/apps/metron"
 metron_hdfs_rotation_policy: org.apache.storm.hdfs.bolt.rotation.TimedRotationPolicy
 metron_hdfs_rotation_policy_count: 1
-metron_hdfs_rotation_policy_units: DAYS
\ No newline at end of file
+metron_hdfs_rotation_policy_units: DAYS

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/e59b1a31/deployment/roles/metron_streaming/files/config/sensors/bro.json
----------------------------------------------------------------------
diff --git a/deployment/roles/metron_streaming/files/config/sensors/bro.json b/deployment/roles/metron_streaming/files/config/sensors/bro.json
new file mode 100644
index 0000000..34109b8
--- /dev/null
+++ b/deployment/roles/metron_streaming/files/config/sensors/bro.json
@@ -0,0 +1,14 @@
+{
+  "index": "bro",
+  "batchSize": 5,
+  "enrichmentFieldMap":
+  {
+    "geo": ["ip_dst_addr", "ip_src_addr"],
+    "host": ["host"]
+  },
+  "threatIntelFieldMap":
+  {
+    "ip": ["ip_dst_addr", "ip_src_addr"]
+  }
+}
+

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/e59b1a31/deployment/roles/metron_streaming/files/config/sensors/pcap.json
----------------------------------------------------------------------
diff --git a/deployment/roles/metron_streaming/files/config/sensors/pcap.json b/deployment/roles/metron_streaming/files/config/sensors/pcap.json
new file mode 100644
index 0000000..4b9c639
--- /dev/null
+++ b/deployment/roles/metron_streaming/files/config/sensors/pcap.json
@@ -0,0 +1,14 @@
+{
+  "index": "pcap",
+  "batchSize": 5,
+  "enrichmentFieldMap":
+  {
+    "geo": ["ip_src_addr", "ip_dst_addr"],
+    "host": ["ip_src_addr", "ip_dst_addr"]
+  },
+  "threatIntelFieldMap":
+  {
+    "ip": ["ip_src_addr", "ip_dst_addr"]
+  }
+}
+

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/e59b1a31/deployment/roles/metron_streaming/files/config/sensors/snort.json
----------------------------------------------------------------------
diff --git a/deployment/roles/metron_streaming/files/config/sensors/snort.json b/deployment/roles/metron_streaming/files/config/sensors/snort.json
new file mode 100644
index 0000000..1208637
--- /dev/null
+++ b/deployment/roles/metron_streaming/files/config/sensors/snort.json
@@ -0,0 +1,14 @@
+{
+  "index": "snort",
+  "batchSize": 1,
+  "enrichmentFieldMap":
+  {
+    "geo": ["ip_dst_addr", "ip_src_addr"],
+    "host": ["host"]
+  },
+  "threatIntelFieldMap":
+  {
+    "ip": ["ip_dst_addr", "ip_src_addr"]
+  }
+}
+

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/e59b1a31/deployment/roles/metron_streaming/files/config/sensors/yaf.json
----------------------------------------------------------------------
diff --git a/deployment/roles/metron_streaming/files/config/sensors/yaf.json b/deployment/roles/metron_streaming/files/config/sensors/yaf.json
new file mode 100644
index 0000000..65de961
--- /dev/null
+++ b/deployment/roles/metron_streaming/files/config/sensors/yaf.json
@@ -0,0 +1,14 @@
+{
+  "index": "yaf",
+  "batchSize": 5,
+  "enrichmentFieldMap":
+  {
+    "geo": ["ip_dst_addr", "ip_src_addr"],
+    "host": ["host"]
+  },
+  "threatIntelFieldMap":
+  {
+    "ip": ["ip_dst_addr", "ip_src_addr"]
+  }
+}
+

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/e59b1a31/deployment/roles/metron_streaming/files/source/bro-config.json
----------------------------------------------------------------------
diff --git a/deployment/roles/metron_streaming/files/source/bro-config.json b/deployment/roles/metron_streaming/files/source/bro-config.json
deleted file mode 100644
index 34109b8..0000000
--- a/deployment/roles/metron_streaming/files/source/bro-config.json
+++ /dev/null
@@ -1,14 +0,0 @@
-{
-  "index": "bro",
-  "batchSize": 5,
-  "enrichmentFieldMap":
-  {
-    "geo": ["ip_dst_addr", "ip_src_addr"],
-    "host": ["host"]
-  },
-  "threatIntelFieldMap":
-  {
-    "ip": ["ip_dst_addr", "ip_src_addr"]
-  }
-}
-

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/e59b1a31/deployment/roles/metron_streaming/files/source/pcap-config.json
----------------------------------------------------------------------
diff --git a/deployment/roles/metron_streaming/files/source/pcap-config.json b/deployment/roles/metron_streaming/files/source/pcap-config.json
deleted file mode 100644
index 4b9c639..0000000
--- a/deployment/roles/metron_streaming/files/source/pcap-config.json
+++ /dev/null
@@ -1,14 +0,0 @@
-{
-  "index": "pcap",
-  "batchSize": 5,
-  "enrichmentFieldMap":
-  {
-    "geo": ["ip_src_addr", "ip_dst_addr"],
-    "host": ["ip_src_addr", "ip_dst_addr"]
-  },
-  "threatIntelFieldMap":
-  {
-    "ip": ["ip_src_addr", "ip_dst_addr"]
-  }
-}
-

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/e59b1a31/deployment/roles/metron_streaming/files/source/snort-config.json
----------------------------------------------------------------------
diff --git a/deployment/roles/metron_streaming/files/source/snort-config.json b/deployment/roles/metron_streaming/files/source/snort-config.json
deleted file mode 100644
index 1208637..0000000
--- a/deployment/roles/metron_streaming/files/source/snort-config.json
+++ /dev/null
@@ -1,14 +0,0 @@
-{
-  "index": "snort",
-  "batchSize": 1,
-  "enrichmentFieldMap":
-  {
-    "geo": ["ip_dst_addr", "ip_src_addr"],
-    "host": ["host"]
-  },
-  "threatIntelFieldMap":
-  {
-    "ip": ["ip_dst_addr", "ip_src_addr"]
-  }
-}
-

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/e59b1a31/deployment/roles/metron_streaming/files/source/yaf-config.json
----------------------------------------------------------------------
diff --git a/deployment/roles/metron_streaming/files/source/yaf-config.json b/deployment/roles/metron_streaming/files/source/yaf-config.json
deleted file mode 100644
index 65de961..0000000
--- a/deployment/roles/metron_streaming/files/source/yaf-config.json
+++ /dev/null
@@ -1,14 +0,0 @@
-{
-  "index": "yaf",
-  "batchSize": 5,
-  "enrichmentFieldMap":
-  {
-    "geo": ["ip_dst_addr", "ip_src_addr"],
-    "host": ["host"]
-  },
-  "threatIntelFieldMap":
-  {
-    "ip": ["ip_dst_addr", "ip_src_addr"]
-  }
-}
-

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/e59b1a31/deployment/roles/metron_streaming/handlers/main.yml
----------------------------------------------------------------------
diff --git a/deployment/roles/metron_streaming/handlers/main.yml b/deployment/roles/metron_streaming/handlers/main.yml
index 634d591..5aee8b4 100644
--- a/deployment/roles/metron_streaming/handlers/main.yml
+++ b/deployment/roles/metron_streaming/handlers/main.yml
@@ -15,5 +15,5 @@
 #  limitations under the License.
 #
 ---
-- name: Load Source Config
-  shell: java -cp {{ metron_directory }}/lib/{{ metron_jar_name }}::/usr/hdp/current/hadoop-client/lib/slf4j-api-1.7.10.jar org.apache.metron.utils.SourceConfigUtils -p {{ source_config_path }} -z {{ zookeeper_url }} && touch {{ source_config_path }}/configured
+- name: Load Config
+  shell: "{{ metron_directory }}/bin/zk_load_configs.sh -p {{ zookeeper_config_path }} -z {{ zookeeper_url }} && touch {{ zookeeper_config_path }}/configured"

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/e59b1a31/deployment/roles/metron_streaming/tasks/main.yml
----------------------------------------------------------------------
diff --git a/deployment/roles/metron_streaming/tasks/main.yml b/deployment/roles/metron_streaming/tasks/main.yml
index 55ace91..863ecd8 100644
--- a/deployment/roles/metron_streaming/tasks/main.yml
+++ b/deployment/roles/metron_streaming/tasks/main.yml
@@ -23,6 +23,16 @@
       - { name: 'config'}
 
 
+- name: Copy Metron Solr bundle
+  copy:
+    src: "{{ metron_solr_bundle_path }}"
+    dest: "{{ metron_directory }}"
+
+- name: Copy Metron Elasticsearch bundle
+  copy:
+    src: "{{ metron_elasticsearch_bundle_path }}"
+    dest: "{{ metron_directory }}"
+
 - name: Copy Metron Topologies bundle
   copy:
     src: "{{ metron_topologies_bundle_path }}"
@@ -34,16 +44,17 @@
     dest: "{{ metron_directory }}"
 
 - name: Unbundle Metron bundles
-  shell: cd {{ metron_directory }} && tar xzvf Metron-Topologies*.tar.gz && tar xzvf Metron-DataLoads*.tar.gz && rm *.tar.gz
-
-- name: Add hbase-site.xml to topology jar
-  shell: cd {{ hbase_config_path }} && jar -uf {{ metron_directory }}/lib/{{ metron_jar_name }} hbase-site.xml
+  shell: cd {{ metron_directory }} && tar xzvf Metron-Solr*.tar.gz && tar xzvf Metron-Elasticsearch*.tar.gz && tar xzvf Metron-Topologies*.tar.gz && tar xzvf Metron-DataLoads*.tar.gz && rm *.tar.gz
 
-- name: Add core-site.xml to topology jar
-  shell: cd {{ hdfs_config_path }} && jar -uf {{ metron_directory }}/lib/{{ metron_jar_name }} core-site.xml
-
-- name: Add hdfs-site.xml to topology jar
-  shell: cd {{ hdfs_config_path }} && jar -uf {{ metron_directory }}/lib/{{ metron_jar_name }} hdfs-site.xml
+- name: Add *-site.xml files to topology jars
+  shell: cd {{ item.config_path }} && jar -uf {{ metron_directory }}/lib/{{ item.jar_name }} {{ item.file_name }}
+  with_items:
+      - { config_path: "{{ hbase_config_path }}", jar_name: "{{ metron_solr_jar_name }}", file_name: "hbase-site.xml" }
+      - { config_path: "{{ hdfs_config_path }}", jar_name: "{{ metron_solr_jar_name }}", file_name: "core-site.xml" }
+      - { config_path: "{{ hdfs_config_path }}", jar_name: "{{ metron_solr_jar_name }}", file_name: "hdfs-site.xml" }
+      - { config_path: "{{ hbase_config_path }}", jar_name: "{{ metron_elasticsearch_jar_name }}", file_name: "hbase-site.xml" }
+      - { config_path: "{{ hdfs_config_path }}", jar_name: "{{ metron_elasticsearch_jar_name }}", file_name: "core-site.xml" }
+      - { config_path: "{{ hdfs_config_path }}", jar_name: "{{ metron_elasticsearch_jar_name }}", file_name: "hdfs-site.xml" }
 
 - name: Get Default mysql passowrd
   include_vars: "../roles/mysql_server/defaults/main.yml"
@@ -55,9 +66,35 @@
 - include: grok_upload.yml
   run_once: true
 
-- name: Configure Metron topologies
+- name: Configure Metron Solr topologies
+  lineinfile: >
+    dest={{ metron_solr_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 }}" }
+    - { regexp: "es.ip=", line: "es.ip={{ groups.search[0] }}" }
+    - { regexp: "es.port=", line: "es.port={{ elasticsearch_transport_port }}" }
+    - { regexp: "es.clustername=", line: "es.clustername={{ elasticsearch_cluster_name }}" }
+    - { regexp: "bolt.hdfs.file.system.url=", line: "bolt.hdfs.file.system.url={{ hdfs_url }}" }
+    - { regexp: "spout.kafka.topic.pcap=", line: "spout.kafka.topic.pcap={{ pycapa_topic }}" }
+    - { regexp: "spout.kafka.topic.bro=", line: "spout.kafka.topic.bro={{ bro_topic }}" }
+    - { regexp: "bolt.hbase.table.name=", line: "bolt.hbase.table.name={{ pcap_hbase_table }}" }
+    - { regexp: "threat.intel.tracker.table=", line: "threat.intel.tracker.table={{ tracker_hbase_table }}" }
+    - { regexp: "threat.intel.tracker.cf=", line: "threat.intel.tracker.cf=t" }
+    - { regexp: "threat.intel.ip.table=", line: "threat.intel.ip.table={{ threatintel_ip_hbase_table }}" }
+    - { regexp: "threat.intel.ip.cf=", line: "threat.intel.ip.cf=t" }
+    - { regexp: "mysql.ip=", line: "mysql.ip={{ groups.mysql[0] }}" }
+    - { regexp: "mysql.password=", line: "mysql.password={{ mysql_root_password }}" }
+    - { regexp: "index.hdfs.output=", line: "index.hdfs.output={{ metron_hdfs_output_dir }}/enrichment/indexed" }
+    - { regexp: "bolt.hdfs.rotation.policy=", line: "bolt.hdfs.rotation.policy={{ metron_hdfs_rotation_policy }}" }
+    - { regexp: "bolt.hdfs.rotation.policy.count=", line: "bolt.hdfs.rotation.policy.count={{ metron_hdfs_rotation_policy_count}}" }
+    - { regexp: "bolt.hdfs.rotation.policy.units=", line: "bolt.hdfs.rotation.policy.units={{ metron_hdfs_rotation_policy_units }}" }
+
+- name: Configure Metron Elasticsearch topologies
   lineinfile: >
-    dest={{ metron_properties_config_path }}
+    dest={{ metron_elasticsearch_properties_config_path }}
     regexp="{{ item.regexp }}"
     line="{{ item.line }}"
   with_items:

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/e59b1a31/deployment/roles/metron_streaming/tasks/metron_topology.yml
----------------------------------------------------------------------
diff --git a/deployment/roles/metron_streaming/tasks/metron_topology.yml b/deployment/roles/metron_streaming/tasks/metron_topology.yml
index 1da24f7..f8bf539 100644
--- a/deployment/roles/metron_streaming/tasks/metron_topology.yml
+++ b/deployment/roles/metron_streaming/tasks/metron_topology.yml
@@ -16,7 +16,14 @@
 #
 ---
 
-- name: Submit Metron topologies
-  command: storm jar {{ metron_directory }}/lib/{{ metron_jar_name }} org.apache.storm.flux.Flux  --filter {{ metron_properties_config_path }} --remote {{ item }}
+- name: Submit Solr Metron topologies
+  command: storm jar {{ metron_directory }}/lib/{{ metron_solr_jar_name }} org.apache.storm.flux.Flux  --filter {{ metron_solr_properties_config_path }} --remote {{ item }}
   with_items:
-    "{{ storm_topologies }}"
+      - "{{ storm_topologies }}"
+  when: install_solr | default(False) == True
+
+- name: Submit Elasticsearch Metron topologies
+  command: storm jar {{ metron_directory }}/lib/{{ metron_elasticsearch_jar_name }} org.apache.storm.flux.Flux  --filter {{ metron_elasticsearch_properties_config_path }} --remote {{ item }}
+  with_items:
+      - "{{ storm_topologies }}"
+  when: install_elasticsearch | default(False) == True

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/e59b1a31/deployment/roles/metron_streaming/tasks/source_config.yml
----------------------------------------------------------------------
diff --git a/deployment/roles/metron_streaming/tasks/source_config.yml b/deployment/roles/metron_streaming/tasks/source_config.yml
index 9233bac..959056c 100644
--- a/deployment/roles/metron_streaming/tasks/source_config.yml
+++ b/deployment/roles/metron_streaming/tasks/source_config.yml
@@ -17,15 +17,32 @@
 ---
 - name: Create Source Config Directory
   file:
-    path: "{{ source_config_path }}"
+    path: "{{ zookeeper_config_path }}"
     state: directory
 
-- name: Copy Source Config Files
+- name: Copy Elasticsearch Global Config File
+  template:
+    src: "templates/config/elasticsearch.global.json"
+    dest: "{{ zookeeper_global_config_path }}"
+    mode: 0644
+  when: install_elasticsearch | default(False) == True
+
+- name: Copy Solr Global Config File
+  template:
+    src: "../roles/metron_streaming/templates/config/solr.global.json"
+    dest: "{{ zookeeper_global_config_path }}"
+    mode: 0644
+  when: install_solr | default(False) == True
+
+- name: Copy Sensor Config Files
   copy:
     src: "{{ item }}"
-    dest: "{{ source_config_path }}"
+    dest: "{{ zookeeper_config_path }}"
     mode: 0644
-  with_fileglob:
-    - ../roles/metron_streaming/files/source/*.json
-  notify: Load Source Config
+  with_items:
+    - ../roles/metron_streaming/files/config/
+  notify: Load Config
+
+
+
 

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/e59b1a31/deployment/roles/metron_streaming/templates/config/elasticsearch.global.json
----------------------------------------------------------------------
diff --git a/deployment/roles/metron_streaming/templates/config/elasticsearch.global.json b/deployment/roles/metron_streaming/templates/config/elasticsearch.global.json
new file mode 100644
index 0000000..aa1076c
--- /dev/null
+++ b/deployment/roles/metron_streaming/templates/config/elasticsearch.global.json
@@ -0,0 +1,6 @@
+{
+  "es.clustername": "{{ elasticsearch_cluster_name }}",
+  "es.ip": "{{ groups.search[0] }}",
+  "es.port": {{ elasticsearch_transport_port }},
+  "es.date.format": "yyyy.MM.dd.hh"
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/e59b1a31/deployment/roles/metron_streaming/templates/config/solr.global.json
----------------------------------------------------------------------
diff --git a/deployment/roles/metron_streaming/templates/config/solr.global.json b/deployment/roles/metron_streaming/templates/config/solr.global.json
new file mode 100644
index 0000000..5cb7a4d
--- /dev/null
+++ b/deployment/roles/metron_streaming/templates/config/solr.global.json
@@ -0,0 +1,6 @@
+{
+  "solr.zookeeper": "{{ zookeeper_url }}",
+  "solr.collection": "{{ solr_collection_name }}",
+  "solr.numShards": {{ solr_number_shards }},
+  "solr.replicationFactor": {{ solr_replication_factor }}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/e59b1a31/deployment/roles/metron_streaming/vars/main.yml
----------------------------------------------------------------------
diff --git a/deployment/roles/metron_streaming/vars/main.yml b/deployment/roles/metron_streaming/vars/main.yml
index b356762..f84b949 100644
--- a/deployment/roles/metron_streaming/vars/main.yml
+++ b/deployment/roles/metron_streaming/vars/main.yml
@@ -15,13 +15,23 @@
 #  limitations under the License.
 #
 ---
-metron_jar_name: Metron-Topologies-{{ metron_version }}.jar
-metron_jar_path: "{{ playbook_dir }}/../../metron-streaming/Metron-Topologies/target/{{ metron_jar_name }}"
+metron_solr_jar_name: Metron-Solr-{{ metron_version }}.jar
+metron_elasticsearch_jar_name: Metron-Elasticsearch-{{ metron_version }}.jar
 metron_directory: /usr/metron/{{ metron_version }}
 metron_dataloads_name: Metron-DataLoads-{{ metron_version }}-archive.tar.gz
+metron_solr_bundle_name: Metron-Solr-{{ metron_version }}-archive.tar.gz
+metron_elasticsearch_bundle_name: Metron-Elasticsearch-{{ metron_version }}-archive.tar.gz
 metron_topologies_bundle_name: Metron-Topologies-{{ metron_version }}-archive.tar.gz
 metron_dataloads_path: "{{ playbook_dir }}/../../metron-streaming/Metron-DataLoads/target/{{ metron_dataloads_name }}"
+metron_solr_bundle_path: "{{ playbook_dir }}/../../metron-streaming/Metron-Solr/target/{{ metron_solr_bundle_name }}"
+metron_elasticsearch_bundle_path: "{{ playbook_dir }}/../../metron-streaming/Metron-Elasticsearch/target/{{ metron_elasticsearch_bundle_name }}"
 metron_topologies_bundle_path: "{{ playbook_dir }}/../../metron-streaming/Metron-Topologies/target/{{ metron_topologies_bundle_name }}"
 metron_src_config_path: "{{ playbook_dir }}/../../metron-streaming/Metron-Topologies/src/main/resources/Metron_Configs"
-metron_properties_config_path: "{{ metron_directory }}/config/etc/env/config.properties"
-
+metron_solr_properties_config_path: "{{ metron_directory }}/config/etc/env/solr.properties"
+metron_elasticsearch_properties_config_path: "{{ metron_directory }}/config/etc/env/elasticsearch.properties"
+elasticsearch_config_path: /etc/elasticsearch
+elasticsearch_cluster_name: metron
+elasticsearch_transport_port: 9300
+hbase_config_path: "/etc/hbase/conf"
+hdfs_config_path: "/etc/hadoop/conf"
+metron_hdfs_output_dir: "/apps/metron"

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/e59b1a31/deployment/roles/solr/defaults/main.yml
----------------------------------------------------------------------
diff --git a/deployment/roles/solr/defaults/main.yml b/deployment/roles/solr/defaults/main.yml
new file mode 100644
index 0000000..b40d534
--- /dev/null
+++ b/deployment/roles/solr/defaults/main.yml
@@ -0,0 +1,29 @@
+#
+#  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.
+#
+---
+rhel_hdp_utils_install_url: http://public-repo-1.hortonworks.com/HDP-UTILS-1.1.0.20/repos/centos6/hdp-util.repo
+solr_install_path: /opt/lucidworks-hdpsearch/solr
+solr_user: solr
+solr_collection_name: Metron
+solr_config_dir: "{{ solr_install_path }}/server/solr/configsets/basic_configs/conf"
+solr_bin_dir: "/opt/lucidworks-hdpsearch/solr/bin"
+solr_config_name: "metron_conf"
+solr_number_shards: "{{ groups['search'] | length }}"
+solr_replication_factor: 1
+solr_autoSoftCommit_maxTime: 60
+solr_cmd: "{{ solr_bin_dir}}/solr create_collection -c  {{ solr_collection_name }} -d {{ solr_config_dir }} -n {{ solr_config_name }} -shards {{ solr_number_shards }} -replicationFactor {{ solr_replication_factor }}"
+hdp_utils_repo_path: /etc/yum.repos.d/HDP-UTILS.repo
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/e59b1a31/deployment/roles/solr/files/schema.xml
----------------------------------------------------------------------
diff --git a/deployment/roles/solr/files/schema.xml b/deployment/roles/solr/files/schema.xml
new file mode 100644
index 0000000..43452a2
--- /dev/null
+++ b/deployment/roles/solr/files/schema.xml
@@ -0,0 +1,191 @@
+<?xml version="1.0" encoding="UTF-8" ?>
+<!--
+ 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.
+-->
+
+<schema name="metron" version="1.5">
+
+    <field name="_version_" type="long" indexed="true" stored="true"/>
+    <field name="id" type="string" indexed="true" stored="true" required="true" multiValued="false"/>
+    <field name="sensorType" type="string" indexed="true" stored="true" required="true"/>;
+
+    <dynamicField name="*_i" type="int" indexed="true" stored="true"/>
+    <dynamicField name="*_is" type="int" indexed="true" stored="true" multiValued="true"/>
+    <dynamicField name="*_s" type="string" indexed="true" stored="true"/>
+    <dynamicField name="*_ss" type="string" indexed="true" stored="true" multiValued="true"/>
+    <dynamicField name="*_l" type="long" indexed="true" stored="true"/>
+    <dynamicField name="*_ls" type="long" indexed="true" stored="true" multiValued="true"/>
+    <dynamicField name="*_t" type="text_general" indexed="true" stored="true"/>
+    <dynamicField name="*_txt" type="text_general" indexed="true" stored="true" multiValued="true"/>
+    <dynamicField name="*_en" type="text_en" indexed="true" stored="true" multiValued="true"/>
+    <dynamicField name="*_b" type="boolean" indexed="true" stored="true"/>
+    <dynamicField name="*_bs" type="boolean" indexed="true" stored="true" multiValued="true"/>
+    <dynamicField name="*_f" type="float" indexed="true" stored="true"/>
+    <dynamicField name="*_fs" type="float" indexed="true" stored="true" multiValued="true"/>
+    <dynamicField name="*_d" type="double" indexed="true" stored="true"/>
+    <dynamicField name="*_ds" type="double" indexed="true" stored="true" multiValued="true"/>
+    <dynamicField name="*_coordinate" type="tdouble" indexed="true" stored="false"/>
+    <dynamicField name="*_dt" type="date" indexed="true" stored="true"/>
+    <dynamicField name="*_dts" type="date" indexed="true" stored="true" multiValued="true"/>
+    <dynamicField name="*_p" type="location" indexed="true" stored="true"/>
+    <dynamicField name="*_ti" type="tint" indexed="true" stored="true"/>
+    <dynamicField name="*_tl" type="tlong" indexed="true" stored="true"/>
+    <dynamicField name="*_tf" type="tfloat" indexed="true" stored="true"/>
+    <dynamicField name="*_td" type="tdouble" indexed="true" stored="true"/>
+    <dynamicField name="*_tdt" type="tdate" indexed="true" stored="true"/>
+    <dynamicField name="*_c" type="currency" indexed="true" stored="true"/>
+    <dynamicField name="ignored_*" type="ignored" multiValued="true"/>
+    <dynamicField name="attr_*" type="text_general" indexed="true" stored="true" multiValued="true"/>
+    <dynamicField name="random_*" type="random"/>
+
+    <uniqueKey>id</uniqueKey>
+
+    <fieldType name="string" class="solr.StrField" sortMissingLast="true"/>
+    <fieldType name="boolean" class="solr.BoolField" sortMissingLast="true"/>
+    <fieldType name="int" class="solr.TrieIntField" precisionStep="0" positionIncrementGap="0"/>
+    <fieldType name="float" class="solr.TrieFloatField" precisionStep="0" positionIncrementGap="0"/>
+    <fieldType name="long" class="solr.TrieLongField" precisionStep="0" positionIncrementGap="0"/>
+    <fieldType name="double" class="solr.TrieDoubleField" precisionStep="0" positionIncrementGap="0"/>
+    <fieldType name="tint" class="solr.TrieIntField" precisionStep="8" positionIncrementGap="0"/>
+    <fieldType name="tfloat" class="solr.TrieFloatField" precisionStep="8" positionIncrementGap="0"/>
+    <fieldType name="tlong" class="solr.TrieLongField" precisionStep="8" positionIncrementGap="0"/>
+    <fieldType name="tdouble" class="solr.TrieDoubleField" precisionStep="8" positionIncrementGap="0"/>
+    <fieldType name="date" class="solr.TrieDateField" precisionStep="0" positionIncrementGap="0"/>
+    <fieldType name="tdate" class="solr.TrieDateField" precisionStep="6" positionIncrementGap="0"/>
+    <fieldType name="binary" class="solr.BinaryField"/>
+    <fieldType name="random" class="solr.RandomSortField" indexed="true"/>
+    <fieldType name="text_ws" class="solr.TextField" positionIncrementGap="100">
+        <analyzer>
+            <tokenizer class="solr.WhitespaceTokenizerFactory"/>
+        </analyzer>
+    </fieldType>
+    <fieldType name="text_general" class="solr.TextField" positionIncrementGap="100">
+        <analyzer type="index">
+            <tokenizer class="solr.StandardTokenizerFactory"/>
+            <filter class="solr.StopFilterFactory" ignoreCase="true" words="stopwords.txt"/>
+            <filter class="solr.LowerCaseFilterFactory"/>
+        </analyzer>
+        <analyzer type="query">
+            <tokenizer class="solr.StandardTokenizerFactory"/>
+            <filter class="solr.StopFilterFactory" ignoreCase="true" words="stopwords.txt"/>
+            <filter class="solr.SynonymFilterFactory" synonyms="synonyms.txt" ignoreCase="true" expand="true"/>
+            <filter class="solr.LowerCaseFilterFactory"/>
+        </analyzer>
+    </fieldType>
+    <fieldType name="text_en" class="solr.TextField" positionIncrementGap="100">
+        <analyzer type="index">
+            <tokenizer class="solr.StandardTokenizerFactory"/>
+            <filter class="solr.StopFilterFactory"
+                    ignoreCase="true"
+                    words="lang/stopwords_en.txt"
+            />
+            <filter class="solr.LowerCaseFilterFactory"/>
+            <filter class="solr.EnglishPossessiveFilterFactory"/>
+            <filter class="solr.KeywordMarkerFilterFactory" protected="protwords.txt"/>
+            <filter class="solr.PorterStemFilterFactory"/>
+        </analyzer>
+        <analyzer type="query">
+            <tokenizer class="solr.StandardTokenizerFactory"/>
+            <filter class="solr.SynonymFilterFactory" synonyms="synonyms.txt" ignoreCase="true" expand="true"/>
+            <filter class="solr.StopFilterFactory"
+                    ignoreCase="true"
+                    words="lang/stopwords_en.txt"
+            />
+            <filter class="solr.LowerCaseFilterFactory"/>
+            <filter class="solr.EnglishPossessiveFilterFactory"/>
+            <filter class="solr.KeywordMarkerFilterFactory" protected="protwords.txt"/>
+            <filter class="solr.PorterStemFilterFactory"/>
+        </analyzer>
+    </fieldType>
+    <fieldType name="text_en_splitting" class="solr.TextField" positionIncrementGap="100" autoGeneratePhraseQueries="true">
+        <analyzer type="index">
+            <tokenizer class="solr.WhitespaceTokenizerFactory"/>
+            <filter class="solr.StopFilterFactory"
+                    ignoreCase="true"
+                    words="lang/stopwords_en.txt"
+            />
+            <filter class="solr.WordDelimiterFilterFactory" generateWordParts="1" generateNumberParts="1" catenateWords="1" catenateNumbers="1" catenateAll="0" splitOnCaseChange="1"/>
+            <filter class="solr.LowerCaseFilterFactory"/>
+            <filter class="solr.KeywordMarkerFilterFactory" protected="protwords.txt"/>
+            <filter class="solr.PorterStemFilterFactory"/>
+        </analyzer>
+        <analyzer type="query">
+            <tokenizer class="solr.WhitespaceTokenizerFactory"/>
+            <filter class="solr.SynonymFilterFactory" synonyms="synonyms.txt" ignoreCase="true" expand="true"/>
+            <filter class="solr.StopFilterFactory"
+                    ignoreCase="true"
+                    words="lang/stopwords_en.txt"
+            />
+            <filter class="solr.WordDelimiterFilterFactory" generateWordParts="1" generateNumberParts="1" catenateWords="0" catenateNumbers="0" catenateAll="0" splitOnCaseChange="1"/>
+            <filter class="solr.LowerCaseFilterFactory"/>
+            <filter class="solr.KeywordMarkerFilterFactory" protected="protwords.txt"/>
+            <filter class="solr.PorterStemFilterFactory"/>
+        </analyzer>
+    </fieldType>
+
+    <fieldType name="text_en_splitting_tight" class="solr.TextField" positionIncrementGap="100" autoGeneratePhraseQueries="true">
+        <analyzer>
+            <tokenizer class="solr.WhitespaceTokenizerFactory"/>
+            <filter class="solr.SynonymFilterFactory" synonyms="synonyms.txt" ignoreCase="true" expand="false"/>
+            <filter class="solr.StopFilterFactory" ignoreCase="true" words="lang/stopwords_en.txt"/>
+            <filter class="solr.WordDelimiterFilterFactory" generateWordParts="0" generateNumberParts="0" catenateWords="1" catenateNumbers="1" catenateAll="0"/>
+            <filter class="solr.LowerCaseFilterFactory"/>
+            <filter class="solr.KeywordMarkerFilterFactory" protected="protwords.txt"/>
+            <filter class="solr.EnglishMinimalStemFilterFactory"/>
+            <filter class="solr.RemoveDuplicatesTokenFilterFactory"/>
+        </analyzer>
+    </fieldType>
+    <fieldType name="text_general_rev" class="solr.TextField" positionIncrementGap="100">
+        <analyzer type="index">
+            <tokenizer class="solr.StandardTokenizerFactory"/>
+            <filter class="solr.StopFilterFactory" ignoreCase="true" words="stopwords.txt"/>
+            <filter class="solr.LowerCaseFilterFactory"/>
+            <filter class="solr.ReversedWildcardFilterFactory" withOriginal="true"
+                    maxPosAsterisk="3" maxPosQuestion="2" maxFractionAsterisk="0.33"/>
+        </analyzer>
+        <analyzer type="query">
+            <tokenizer class="solr.StandardTokenizerFactory"/>
+            <filter class="solr.SynonymFilterFactory" synonyms="synonyms.txt" ignoreCase="true" expand="true"/>
+            <filter class="solr.StopFilterFactory" ignoreCase="true" words="stopwords.txt"/>
+            <filter class="solr.LowerCaseFilterFactory"/>
+        </analyzer>
+    </fieldType>
+    <fieldType name="alphaOnlySort" class="solr.TextField" sortMissingLast="true" omitNorms="true">
+        <analyzer>
+            <tokenizer class="solr.KeywordTokenizerFactory"/>
+            <filter class="solr.LowerCaseFilterFactory"/>
+            <filter class="solr.TrimFilterFactory"/>
+            <filter class="solr.PatternReplaceFilterFactory"
+                    pattern="([^a-z])" replacement="" replace="all"
+            />
+        </analyzer>
+    </fieldType>
+    <fieldType name="lowercase" class="solr.TextField" positionIncrementGap="100">
+        <analyzer>
+            <tokenizer class="solr.KeywordTokenizerFactory"/>
+            <filter class="solr.LowerCaseFilterFactory"/>
+        </analyzer>
+    </fieldType>
+    <fieldType name="ignored" stored="false" indexed="false" multiValued="true" class="solr.StrField"/>
+    <fieldType name="point" class="solr.PointType" dimension="2" subFieldSuffix="_d"/>
+    <fieldType name="location" class="solr.LatLonType" subFieldSuffix="_coordinate"/>
+    <fieldType name="location_rpt" class="solr.SpatialRecursivePrefixTreeFieldType"
+               geo="true" distErrPct="0.025" maxDistErr="0.001" distanceUnits="kilometers"/>
+    <fieldType name="bbox" class="solr.BBoxField"
+               geo="true" distanceUnits="kilometers" numberType="_bbox_coord"/>
+    <fieldType name="_bbox_coord" class="solr.TrieDoubleField" precisionStep="8" docValues="true" stored="false"/>
+    <fieldType name="currency" class="solr.CurrencyField" precisionStep="8" defaultCurrency="USD" currencyConfig="currency.xml"/>
+</schema>

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/e59b1a31/deployment/roles/solr/meta/main.yml
----------------------------------------------------------------------
diff --git a/deployment/roles/solr/meta/main.yml b/deployment/roles/solr/meta/main.yml
new file mode 100644
index 0000000..454dd37
--- /dev/null
+++ b/deployment/roles/solr/meta/main.yml
@@ -0,0 +1,21 @@
+#
+#  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.
+#
+---
+dependencies:
+  - ambari_gather_facts
+  - java_jdk
+

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/e59b1a31/deployment/roles/solr/tasks/main.yml
----------------------------------------------------------------------
diff --git a/deployment/roles/solr/tasks/main.yml b/deployment/roles/solr/tasks/main.yml
new file mode 100644
index 0000000..cfbb6b5
--- /dev/null
+++ b/deployment/roles/solr/tasks/main.yml
@@ -0,0 +1,74 @@
+#
+#  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: Check for Metron jar path
+  stat: path={{ hdp_utils_repo_path }}
+  register: hdp_utils
+
+
+- name: Install HDP-UTILs Repo
+  get_url:
+    url: "{{ rhel_hdp_utils_install_url }}"
+    dest: /etc/yum.repos.d/HDP-UTILS.repo
+  when: hdp_utils.stat.exists == False
+
+- name: Install HDP-UTIL gpg key
+  rpm_key:
+    state: present
+    key: http://pgp.mit.edu/pks/lookup?op=get&search=0xB9733A7A07513CAD
+  when: hdp_utils.stat.exists == False
+
+- name: Install Solr
+  yum:
+    name: lucidworks-hdpsearch
+    state: present
+
+- name: Create solr.xml from template
+  template:
+    src: solr.xml
+    dest: "{{ solr_install_path }}/server/solr"
+    mode: 0644
+    owner: "{{ solr_user }}"
+    group: "{{ solr_user }}"
+
+- name: Copy solrschema.xml to {{ inventory_hostname }}
+  copy:
+    src: schema.xml
+    dest: "{{ solr_config_dir }}"
+    mode: 0644
+    owner: "{{ solr_user }}"
+    group: "{{ solr_user }}"
+
+- name: Create solrconfig.xml from template
+  template:
+    src: solrconfig.xml
+    dest: "{{ solr_config_dir }}"
+    mode: 0644
+    owner: "{{ solr_user }}"
+    group: "{{ solr_user }}"
+
+- name: Start Solr
+  service:
+    name: solr
+    state: restarted
+    enabled: yes
+
+- name: Create Collection {{ solr_collection_name }} with {{ solr_number_shards }} shard(s) and replication factor {{ solr_replication_factor }}
+  shell: "{{ solr_cmd }}"
+  ignore_errors: yes
+  register: result
+  failed_when: result.rc == 1 and result.stderr.find("already exists!") == -1

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/e59b1a31/deployment/roles/solr/templates/solr.xml
----------------------------------------------------------------------
diff --git a/deployment/roles/solr/templates/solr.xml b/deployment/roles/solr/templates/solr.xml
new file mode 100644
index 0000000..407df13
--- /dev/null
+++ b/deployment/roles/solr/templates/solr.xml
@@ -0,0 +1,52 @@
+<?xml version="1.0" encoding="UTF-8" ?>
+<!--
+ 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.
+-->
+
+<!--
+   This is an example of a simple "solr.xml" file for configuring one or 
+   more Solr Cores, as well as allowing Cores to be added, removed, and 
+   reloaded via HTTP requests.
+
+   More information about options available in this configuration file, 
+   and Solr Core administration can be found online:
+   http://wiki.apache.org/solr/CoreAdmin
+-->
+
+<solr>
+
+  <solrcloud>
+
+    <str name="host">${host:}</str>
+    <int name="hostPort">${jetty.port:8983}</int>
+    <str name="hostContext">${hostContext:solr}</str>
+
+    <bool name="genericCoreNodeNames">${genericCoreNodeNames:true}</bool>
+
+    <str name="zkHost">{{ zookeeper_url }}</str>
+    <int name="zkClientTimeout">${zkClientTimeout:30000}</int>
+    <int name="distribUpdateSoTimeout">${distribUpdateSoTimeout:600000}</int>
+    <int name="distribUpdateConnTimeout">${distribUpdateConnTimeout:60000}</int>
+
+  </solrcloud>
+
+  <shardHandlerFactory name="shardHandlerFactory"
+    class="HttpShardHandlerFactory">
+    <int name="socketTimeout">${socketTimeout:600000}</int>
+    <int name="connTimeout">${connTimeout:60000}</int>
+  </shardHandlerFactory>
+
+</solr>

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/e59b1a31/deployment/roles/solr/templates/solrconfig.xml
----------------------------------------------------------------------
diff --git a/deployment/roles/solr/templates/solrconfig.xml b/deployment/roles/solr/templates/solrconfig.xml
new file mode 100644
index 0000000..b00af0f
--- /dev/null
+++ b/deployment/roles/solr/templates/solrconfig.xml
@@ -0,0 +1,583 @@
+<?xml version="1.0" encoding="UTF-8" ?>
+<!--
+ 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.
+-->
+
+<!-- 
+     For more details about configurations options that may appear in
+     this file, see http://wiki.apache.org/solr/SolrConfigXml. 
+-->
+<config>
+  <!-- In all configuration below, a prefix of "solr." for class names
+       is an alias that causes solr to search appropriate packages,
+       including org.apache.solr.(search|update|request|core|analysis)
+
+       You may also specify a fully qualified Java classname if you
+       have your own custom plugins.
+    -->
+
+  <!-- Controls what version of Lucene various components of Solr
+       adhere to.  Generally, you want to use the latest version to
+       get all bug fixes and improvements. It is highly recommended
+       that you fully re-index after changing this setting as it can
+       affect both how text is indexed and queried.
+  -->
+  <luceneMatchVersion>5.2.1</luceneMatchVersion>
+
+  <!-- Data Directory
+
+       Used to specify an alternate directory to hold all index data
+       other than the default ./data under the Solr home.  If
+       replication is in use, this should match the replication
+       configuration.
+    -->
+  <dataDir>${solr.data.dir:}</dataDir>
+
+
+  <!-- The DirectoryFactory to use for indexes.
+       
+       solr.StandardDirectoryFactory is filesystem
+       based and tries to pick the best implementation for the current
+       JVM and platform.  solr.NRTCachingDirectoryFactory, the default,
+       wraps solr.StandardDirectoryFactory and caches small files in memory
+       for better NRT performance.
+
+       One can force a particular implementation via solr.MMapDirectoryFactory,
+       solr.NIOFSDirectoryFactory, or solr.SimpleFSDirectoryFactory.
+
+       solr.RAMDirectoryFactory is memory based, not
+       persistent, and doesn't work with replication.
+    -->
+  <directoryFactory name="DirectoryFactory" 
+                    class="${solr.directoryFactory:solr.NRTCachingDirectoryFactory}">
+  </directoryFactory> 
+
+  <!-- The CodecFactory for defining the format of the inverted index.
+       The default implementation is SchemaCodecFactory, which is the official Lucene
+       index format, but hooks into the schema to provide per-field customization of
+       the postings lists and per-document values in the fieldType element
+       (postingsFormat/docValuesFormat). Note that most of the alternative implementations
+       are experimental, so if you choose to customize the index format, it's a good
+       idea to convert back to the official format e.g. via IndexWriter.addIndexes(IndexReader)
+       before upgrading to a newer version to avoid unnecessary reindexing.
+  -->
+  <codecFactory class="solr.SchemaCodecFactory"/>
+
+  <schemaFactory class="ClassicIndexSchemaFactory"/>
+
+  <!-- ~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
+       Index Config - These settings control low-level behavior of indexing
+       Most example settings here show the default value, but are commented
+       out, to more easily see where customizations have been made.
+       
+       Note: This replaces <indexDefaults> and <mainIndex> from older versions
+       ~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~ -->
+  <indexConfig>
+
+    <!-- LockFactory 
+
+         This option specifies which Lucene LockFactory implementation
+         to use.
+      
+         single = SingleInstanceLockFactory - suggested for a
+                  read-only index or when there is no possibility of
+                  another process trying to modify the index.
+         native = NativeFSLockFactory - uses OS native file locking.
+                  Do not use when multiple solr webapps in the same
+                  JVM are attempting to share a single index.
+         simple = SimpleFSLockFactory  - uses a plain file for locking
+
+         Defaults: 'native' is default for Solr3.6 and later, otherwise
+                   'simple' is the default
+
+         More details on the nuances of each LockFactory...
+         http://wiki.apache.org/lucene-java/AvailableLockFactories
+    -->
+    <lockType>${solr.lock.type:native}</lockType>
+
+    <!-- Lucene Infostream
+       
+         To aid in advanced debugging, Lucene provides an "InfoStream"
+         of detailed information when indexing.
+
+         Setting the value to true will instruct the underlying Lucene
+         IndexWriter to write its info stream to solr's log. By default,
+         this is enabled here, and controlled through log4j.properties.
+      -->
+     <infoStream>true</infoStream>
+  </indexConfig>
+
+
+  <!-- JMX
+       
+       This example enables JMX if and only if an existing MBeanServer
+       is found, use this if you want to configure JMX through JVM
+       parameters. Remove this to disable exposing Solr configuration
+       and statistics to JMX.
+
+       For more details see http://wiki.apache.org/solr/SolrJmx
+    -->
+  <jmx />
+  <!-- If you want to connect to a particular server, specify the
+       agentId 
+    -->
+  <!-- <jmx agentId="myAgent" /> -->
+  <!-- If you want to start a new MBeanServer, specify the serviceUrl -->
+  <!-- <jmx serviceUrl="service:jmx:rmi:///jndi/rmi://localhost:9999/solr"/>
+    -->
+
+  <!-- The default high-performance update handler -->
+  <updateHandler class="solr.DirectUpdateHandler2">
+
+    <!-- Enables a transaction log, used for real-time get, durability, and
+         and solr cloud replica recovery.  The log can grow as big as
+         uncommitted changes to the index, so use of a hard autoCommit
+         is recommended (see below).
+         "dir" - the target directory for transaction logs, defaults to the
+                solr data directory.
+         "numVersionBuckets" - sets the number of buckets used to keep
+                track of max version values when checking for re-ordered
+                updates; increase this value to reduce the cost of
+                synchronizing access to version buckets during high-volume
+                indexing, this requires 8 bytes (long) * numVersionBuckets
+                of heap space per Solr core.
+    -->
+    <updateLog>
+      <str name="dir">${solr.ulog.dir:}</str>
+      <int name="numVersionBuckets">${solr.ulog.numVersionBuckets:65536}</int>
+    </updateLog>
+ 
+    <!-- AutoCommit
+
+         Perform a hard commit automatically under certain conditions.
+         Instead of enabling autoCommit, consider using "commitWithin"
+         when adding documents. 
+
+         http://wiki.apache.org/solr/UpdateXmlMessages
+
+         maxDocs - Maximum number of documents to add since the last
+                   commit before automatically triggering a new commit.
+
+         maxTime - Maximum amount of time in ms that is allowed to pass
+                   since a document was added before automatically
+                   triggering a new commit. 
+         openSearcher - if false, the commit causes recent index changes
+           to be flushed to stable storage, but does not cause a new
+           searcher to be opened to make those changes visible.
+
+         If the updateLog is enabled, then it's highly recommended to
+         have some sort of hard autoCommit to limit the log size.
+      -->
+     <autoCommit> 
+       <maxTime>${solr.autoCommit.maxTime:15000}</maxTime> 
+       <openSearcher>false</openSearcher> 
+     </autoCommit>
+
+    <!-- softAutoCommit is like autoCommit except it causes a
+         'soft' commit which only ensures that changes are visible
+         but does not ensure that data is synced to disk.  This is
+         faster and more near-realtime friendly than a hard commit.
+      -->
+     <autoSoftCommit>
+       <maxTime>${solr.autoSoftCommit.maxTime:{{ solr_autoSoftCommit_maxTime }}}</maxTime>
+     </autoSoftCommit>
+
+  </updateHandler>
+  
+  <!-- ~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
+       Query section - these settings control query time things like caches
+       ~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~ -->
+  <query>
+    <!-- Max Boolean Clauses
+
+         Maximum number of clauses in each BooleanQuery,  an exception
+         is thrown if exceeded.
+
+         ** WARNING **
+         
+         This option actually modifies a global Lucene property that
+         will affect all SolrCores.  If multiple solrconfig.xml files
+         disagree on this property, the value at any given moment will
+         be based on the last SolrCore to be initialized.
+         
+      -->
+    <maxBooleanClauses>1024</maxBooleanClauses>
+
+
+    <!-- Solr Internal Query Caches
+
+         There are two implementations of cache available for Solr,
+         LRUCache, based on a synchronized LinkedHashMap, and
+         FastLRUCache, based on a ConcurrentHashMap.  
+
+         FastLRUCache has faster gets and slower puts in single
+         threaded operation and thus is generally faster than LRUCache
+         when the hit ratio of the cache is high (> 75%), and may be
+         faster under other scenarios on multi-cpu systems.
+    -->
+
+    <!-- Filter Cache
+
+         Cache used by SolrIndexSearcher for filters (DocSets),
+         unordered sets of *all* documents that match a query.  When a
+         new searcher is opened, its caches may be prepopulated or
+         "autowarmed" using data from caches in the old searcher.
+         autowarmCount is the number of items to prepopulate.  For
+         LRUCache, the autowarmed items will be the most recently
+         accessed items.
+
+         Parameters:
+           class - the SolrCache implementation LRUCache or
+               (LRUCache or FastLRUCache)
+           size - the maximum number of entries in the cache
+           initialSize - the initial capacity (number of entries) of
+               the cache.  (see java.util.HashMap)
+           autowarmCount - the number of entries to prepopulate from
+               and old cache.  
+      -->
+    <filterCache class="solr.FastLRUCache"
+                 size="512"
+                 initialSize="512"
+                 autowarmCount="0"/>
+
+    <!-- Query Result Cache
+
+        Caches results of searches - ordered lists of document ids
+        (DocList) based on a query, a sort, and the range of documents requested.
+        Additional supported parameter by LRUCache:
+           maxRamMB - the maximum amount of RAM (in MB) that this cache is allowed
+                      to occupy
+     -->
+    <queryResultCache class="solr.LRUCache"
+                     size="512"
+                     initialSize="512"
+                     autowarmCount="0"/>
+   
+    <!-- Document Cache
+
+         Caches Lucene Document objects (the stored fields for each
+         document).  Since Lucene internal document ids are transient,
+         this cache will not be autowarmed.  
+      -->
+    <documentCache class="solr.LRUCache"
+                   size="512"
+                   initialSize="512"
+                   autowarmCount="0"/>
+    
+    <!-- custom cache currently used by block join --> 
+    <cache name="perSegFilter"
+      class="solr.search.LRUCache"
+      size="10"
+      initialSize="0"
+      autowarmCount="10"
+      regenerator="solr.NoOpRegenerator" />
+
+    <!-- Lazy Field Loading
+
+         If true, stored fields that are not requested will be loaded
+         lazily.  This can result in a significant speed improvement
+         if the usual case is to not load all stored fields,
+         especially if the skipped fields are large compressed text
+         fields.
+    -->
+    <enableLazyFieldLoading>true</enableLazyFieldLoading>
+
+   <!-- Result Window Size
+
+        An optimization for use with the queryResultCache.  When a search
+        is requested, a superset of the requested number of document ids
+        are collected.  For example, if a search for a particular query
+        requests matching documents 10 through 19, and queryWindowSize is 50,
+        then documents 0 through 49 will be collected and cached.  Any further
+        requests in that range can be satisfied via the cache.  
+     -->
+   <queryResultWindowSize>20</queryResultWindowSize>
+
+   <!-- Maximum number of documents to cache for any entry in the
+        queryResultCache. 
+     -->
+   <queryResultMaxDocsCached>200</queryResultMaxDocsCached>
+
+    <!-- Use Cold Searcher
+
+         If a search request comes in and there is no current
+         registered searcher, then immediately register the still
+         warming searcher and use it.  If "false" then all requests
+         will block until the first searcher is done warming.
+      -->
+    <useColdSearcher>false</useColdSearcher>
+
+    <!-- Max Warming Searchers
+         
+         Maximum number of searchers that may be warming in the
+         background concurrently.  An error is returned if this limit
+         is exceeded.
+
+         Recommend values of 1-2 for read-only slaves, higher for
+         masters w/o cache warming.
+      -->
+    <maxWarmingSearchers>2</maxWarmingSearchers>
+
+  </query>
+
+
+  <!-- Request Dispatcher
+
+       This section contains instructions for how the SolrDispatchFilter
+       should behave when processing requests for this SolrCore.
+
+       handleSelect is a legacy option that affects the behavior of requests
+       such as /select?qt=XXX
+
+       handleSelect="true" will cause the SolrDispatchFilter to process
+       the request and dispatch the query to a handler specified by the 
+       "qt" param, assuming "/select" isn't already registered.
+
+       handleSelect="false" will cause the SolrDispatchFilter to
+       ignore "/select" requests, resulting in a 404 unless a handler
+       is explicitly registered with the name "/select"
+
+       handleSelect="true" is not recommended for new users, but is the default
+       for backwards compatibility
+    -->
+  <requestDispatcher handleSelect="false" >
+    <!-- Request Parsing
+
+         These settings indicate how Solr Requests may be parsed, and
+         what restrictions may be placed on the ContentStreams from
+         those requests
+
+         enableRemoteStreaming - enables use of the stream.file
+         and stream.url parameters for specifying remote streams.
+
+         multipartUploadLimitInKB - specifies the max size (in KiB) of
+         Multipart File Uploads that Solr will allow in a Request.
+         
+         formdataUploadLimitInKB - specifies the max size (in KiB) of
+         form data (application/x-www-form-urlencoded) sent via
+         POST. You can use POST to pass request parameters not
+         fitting into the URL.
+         
+         addHttpRequestToContext - if set to true, it will instruct
+         the requestParsers to include the original HttpServletRequest
+         object in the context map of the SolrQueryRequest under the 
+         key "httpRequest". It will not be used by any of the existing
+         Solr components, but may be useful when developing custom 
+         plugins.
+         
+         *** WARNING ***
+         The settings below authorize Solr to fetch remote files, You
+         should make sure your system has some authentication before
+         using enableRemoteStreaming="true"
+
+      --> 
+    <requestParsers enableRemoteStreaming="true" 
+                    multipartUploadLimitInKB="2048000"
+                    formdataUploadLimitInKB="2048"
+                    addHttpRequestToContext="false"/>
+
+    <!-- HTTP Caching
+
+         Set HTTP caching related parameters (for proxy caches and clients).
+
+         The options below instruct Solr not to output any HTTP Caching
+         related headers
+      -->
+    <httpCaching never304="true" />
+
+  </requestDispatcher>
+
+  <!-- Request Handlers 
+
+       http://wiki.apache.org/solr/SolrRequestHandler
+
+       Incoming queries will be dispatched to a specific handler by name
+       based on the path specified in the request.
+
+       Legacy behavior: If the request path uses "/select" but no Request
+       Handler has that name, and if handleSelect="true" has been specified in
+       the requestDispatcher, then the Request Handler is dispatched based on
+       the qt parameter.  Handlers without a leading '/' are accessed this way
+       like so: http://host/app/[core/]select?qt=name  If no qt is
+       given, then the requestHandler that declares default="true" will be
+       used or the one named "standard".
+
+       If a Request Handler is declared with startup="lazy", then it will
+       not be initialized until the first request that uses it.
+
+    -->
+  <!-- SearchHandler
+
+       http://wiki.apache.org/solr/SearchHandler
+
+       For processing Search Queries, the primary Request Handler
+       provided with Solr is "SearchHandler" It delegates to a sequent
+       of SearchComponents (see below) and supports distributed
+       queries across multiple shards
+    -->
+  <requestHandler name="/select" class="solr.SearchHandler">
+    <!-- default values for query parameters can be specified, these
+         will be overridden by parameters in the request
+      -->
+     <lst name="defaults">
+       <str name="echoParams">explicit</str>
+       <int name="rows">10</int>
+     </lst>
+
+    </requestHandler>
+
+  <!-- A request handler that returns indented JSON by default -->
+  <requestHandler name="/query" class="solr.SearchHandler">
+     <lst name="defaults">
+       <str name="echoParams">explicit</str>
+       <str name="wt">json</str>
+       <str name="indent">true</str>
+       <str name="df">text</str>
+     </lst>
+  </requestHandler>
+
+  <!--
+    The export request handler is used to export full sorted result sets.
+    Do not change these defaults.
+  -->
+  <requestHandler name="/export" class="solr.SearchHandler">
+    <lst name="invariants">
+      <str name="rq">{!xport}</str>
+      <str name="wt">xsort</str>
+      <str name="distrib">false</str>
+    </lst>
+
+    <arr name="components">
+      <str>query</str>
+    </arr>
+  </requestHandler>
+
+
+  <initParams path="/update/**,/query,/select,/tvrh,/elevate,/spell">
+    <lst name="defaults">
+      <str name="df">text</str>
+    </lst>
+  </initParams>
+
+  <!-- Field Analysis Request Handler
+
+       RequestHandler that provides much the same functionality as
+       analysis.jsp. Provides the ability to specify multiple field
+       types and field names in the same request and outputs
+       index-time and query-time analysis for each of them.
+
+       Request parameters are:
+       analysis.fieldname - field name whose analyzers are to be used
+
+       analysis.fieldtype - field type whose analyzers are to be used
+       analysis.fieldvalue - text for index-time analysis
+       q (or analysis.q) - text for query time analysis
+       analysis.showmatch (true|false) - When set to true and when
+           query analysis is performed, the produced tokens of the
+           field value analysis will be marked as "matched" for every
+           token that is produces by the query analysis
+   -->
+  <requestHandler name="/analysis/field" 
+                  startup="lazy"
+                  class="solr.FieldAnalysisRequestHandler" />
+
+
+  <!-- Document Analysis Handler
+
+       http://wiki.apache.org/solr/AnalysisRequestHandler
+
+       An analysis handler that provides a breakdown of the analysis
+       process of provided documents. This handler expects a (single)
+       content stream with the following format:
+
+       <docs>
+         <doc>
+           <field name="id">1</field>
+           <field name="name">The Name</field>
+           <field name="text">The Text Value</field>
+         </doc>
+         <doc>...</doc>
+         <doc>...</doc>
+         ...
+       </docs>
+
+    Note: Each document must contain a field which serves as the
+    unique key. This key is used in the returned response to associate
+    an analysis breakdown to the analyzed document.
+
+    Like the FieldAnalysisRequestHandler, this handler also supports
+    query analysis by sending either an "analysis.query" or "q"
+    request parameter that holds the query text to be analyzed. It
+    also supports the "analysis.showmatch" parameter which when set to
+    true, all field tokens that match the query tokens will be marked
+    as a "match". 
+  -->
+  <requestHandler name="/analysis/document" 
+                  class="solr.DocumentAnalysisRequestHandler" 
+                  startup="lazy" />
+
+  <!-- Echo the request contents back to the client -->
+  <requestHandler name="/debug/dump" class="solr.DumpRequestHandler" >
+    <lst name="defaults">
+     <str name="echoParams">explicit</str> 
+     <str name="echoHandler">true</str>
+    </lst>
+  </requestHandler>
+  
+
+
+  <!-- Search Components
+
+       Search components are registered to SolrCore and used by 
+       instances of SearchHandler (which can access them by name)
+       
+       By default, the following components are available:
+       
+       <searchComponent name="query"     class="solr.QueryComponent" />
+       <searchComponent name="facet"     class="solr.FacetComponent" />
+       <searchComponent name="mlt"       class="solr.MoreLikeThisComponent" />
+       <searchComponent name="highlight" class="solr.HighlightComponent" />
+       <searchComponent name="stats"     class="solr.StatsComponent" />
+       <searchComponent name="debug"     class="solr.DebugComponent" />
+       
+     -->
+
+  <!-- Terms Component
+
+       http://wiki.apache.org/solr/TermsComponent
+
+       A component to return terms and document frequency of those
+       terms
+    -->
+  <searchComponent name="terms" class="solr.TermsComponent"/>
+
+  <!-- A request handler for demonstrating the terms component -->
+  <requestHandler name="/terms" class="solr.SearchHandler" startup="lazy">
+     <lst name="defaults">
+      <bool name="terms">true</bool>
+      <bool name="distrib">false</bool>
+    </lst>     
+    <arr name="components">
+      <str>terms</str>
+    </arr>
+  </requestHandler>
+
+  <!-- Legacy config for the admin interface -->
+  <admin>
+    <defaultQuery>*:*</defaultQuery>
+  </admin>
+
+</config>

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/e59b1a31/metron-streaming/Metron-Common/pom.xml
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-Common/pom.xml b/metron-streaming/Metron-Common/pom.xml
index 605c7ed..22460d0 100644
--- a/metron-streaming/Metron-Common/pom.xml
+++ b/metron-streaming/Metron-Common/pom.xml
@@ -175,6 +175,12 @@
             <groupId>org.apache.storm</groupId>
             <artifactId>storm-kafka</artifactId>
             <version>${global_storm_version}</version>
+            <exclusions>
+                <exclusion>
+                    <artifactId>org.apache.curator</artifactId>
+                    <groupId>curator-client</groupId>
+                </exclusion>
+            </exclusions>
         </dependency>
     </dependencies>
 

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/e59b1a31/metron-streaming/Metron-Common/src/main/java/org/apache/metron/Constants.java
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-Common/src/main/java/org/apache/metron/Constants.java b/metron-streaming/Metron-Common/src/main/java/org/apache/metron/Constants.java
index c6eafe9..78efa5c 100644
--- a/metron-streaming/Metron-Common/src/main/java/org/apache/metron/Constants.java
+++ b/metron-streaming/Metron-Common/src/main/java/org/apache/metron/Constants.java
@@ -19,9 +19,16 @@ package org.apache.metron;
 
 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 SOURCE_TYPE = "source.type";
+  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";
   public static final String ERROR_STREAM = "error";
+
 }
+

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/e59b1a31/metron-streaming/Metron-Common/src/main/java/org/apache/metron/bolt/BulkMessageWriterBolt.java
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-Common/src/main/java/org/apache/metron/bolt/BulkMessageWriterBolt.java b/metron-streaming/Metron-Common/src/main/java/org/apache/metron/bolt/BulkMessageWriterBolt.java
index e9a9237..1f12f7a 100644
--- a/metron-streaming/Metron-Common/src/main/java/org/apache/metron/bolt/BulkMessageWriterBolt.java
+++ b/metron-streaming/Metron-Common/src/main/java/org/apache/metron/bolt/BulkMessageWriterBolt.java
@@ -22,9 +22,8 @@ import backtype.storm.task.TopologyContext;
 import backtype.storm.topology.OutputFieldsDeclarer;
 import backtype.storm.tuple.Fields;
 import backtype.storm.tuple.Tuple;
-import backtype.storm.tuple.Values;
 import org.apache.metron.Constants;
-import org.apache.metron.domain.SourceConfig;
+import org.apache.metron.domain.SensorEnrichmentConfig;
 import org.apache.metron.helpers.topology.ErrorUtils;
 import org.apache.metron.topology.TopologyUtils;
 import org.apache.metron.writer.interfaces.BulkMessageWriter;
@@ -36,14 +35,12 @@ import java.util.*;
 
 public class BulkMessageWriterBolt extends ConfiguredBolt {
 
-  int count = 0;
-
   private static final Logger LOG = LoggerFactory
           .getLogger(BulkMessageWriterBolt.class);
   private OutputCollector collector;
   private BulkMessageWriter<JSONObject> bulkMessageWriter;
-  private Map<String, List<Tuple>> sourceTupleMap = new HashMap<>();
-  private Map<String, List<JSONObject>> sourceMessageMap = new HashMap<>();
+  private Map<String, List<Tuple>> sensorTupleMap = new HashMap<>();
+  private Map<String, List<JSONObject>> sensorMessageMap = new HashMap<>();
 
   public BulkMessageWriterBolt(String zookeeperUrl) {
     super(zookeeperUrl);
@@ -58,7 +55,11 @@ public class BulkMessageWriterBolt extends ConfiguredBolt {
   public void prepare(Map stormConf, TopologyContext context, OutputCollector collector) {
     this.collector = collector;
     super.prepare(stormConf, context, collector);
-    bulkMessageWriter.init(stormConf);
+    try {
+      bulkMessageWriter.init(stormConf, configurations);
+    } catch (Exception e) {
+      throw new RuntimeException(e);
+    }
   }
 
   @SuppressWarnings("unchecked")
@@ -66,23 +67,22 @@ public class BulkMessageWriterBolt extends ConfiguredBolt {
   public void execute(Tuple tuple) {
     JSONObject message = (JSONObject)((JSONObject) tuple.getValueByField("message")).clone();
     message.put("index." + bulkMessageWriter.getClass().getSimpleName().toLowerCase() + ".ts", "" + System.currentTimeMillis());
-    String sourceType = TopologyUtils.getSourceType(message);
-    SourceConfig configuration = configurations.get(sourceType);
-    int batchSize = configuration != null ? configuration.getBatchSize() : 1;
-    List<Tuple> tupleList = sourceTupleMap.get(sourceType);
+    String sensorType = TopologyUtils.getSensorType(message);
+    SensorEnrichmentConfig sensorEnrichmentConfig = configurations.getSensorEnrichmentConfig(sensorType);
+    int batchSize = sensorEnrichmentConfig != null ? sensorEnrichmentConfig.getBatchSize() : 1;
+    List<Tuple> tupleList = sensorTupleMap.get(sensorType);
     if (tupleList == null) tupleList = new ArrayList<>();
     tupleList.add(tuple);
-    List<JSONObject> messageList = sourceMessageMap.get(sourceType);
+    List<JSONObject> messageList = sensorMessageMap.get(sensorType);
     if (messageList == null) messageList = new ArrayList<>();
     messageList.add(message);
     if (messageList.size() < batchSize) {
-      sourceTupleMap.put(sourceType, tupleList);
-      sourceMessageMap.put(sourceType, messageList);
+      sensorTupleMap.put(sensorType, tupleList);
+      sensorMessageMap.put(sensorType, messageList);
     } else {
       try {
-
-        String esType = sourceType + "_doc";
-        bulkMessageWriter.write(esType, configuration, tupleList, messageList);
+        String esType = sensorType + "_doc";
+        bulkMessageWriter.write(esType, configurations, tupleList, messageList);
         for(Tuple t: tupleList) {
           collector.ack(t);
         }
@@ -92,8 +92,8 @@ public class BulkMessageWriterBolt extends ConfiguredBolt {
         }
         ErrorUtils.handleError(collector, e, Constants.ERROR_STREAM);
       }
-      sourceTupleMap.remove(sourceType);
-      sourceMessageMap.remove(sourceType);
+      sensorTupleMap.remove(sensorType);
+      sensorMessageMap.remove(sensorType);
     }
   }
 


[03/15] incubator-metron git commit: METRON 86: Adding Solr indexing support (merrimanr via cestella) closes apache/incubator-metron#67

Posted by ce...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/e59b1a31/metron-streaming/Metron-Topologies/src/main/resources/SampleInput/ISESampleOutput
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-Topologies/src/main/resources/SampleInput/ISESampleOutput b/metron-streaming/Metron-Topologies/src/main/resources/SampleInput/ISESampleOutput
deleted file mode 100644
index 1cb0678..0000000
--- a/metron-streaming/Metron-Topologies/src/main/resources/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, FailureReason=11007 Could not locate Network Device or AAA Client, Step=

<TRUNCATED>


[05/15] incubator-metron git commit: METRON 86: Adding Solr indexing support (merrimanr via cestella) closes apache/incubator-metron#67

Posted by ce...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/e59b1a31/metron-streaming/Metron-Topologies/src/main/resources/SampleInput/BroExampleOutput
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-Topologies/src/main/resources/SampleInput/BroExampleOutput b/metron-streaming/Metron-Topologies/src/main/resources/SampleInput/BroExampleOutput
deleted file mode 100644
index 6ded8a8..0000000
--- a/metron-streaming/Metron-Topologies/src/main/resources/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":1,"method":"GET","host":"www.cisco.com","uri":"/","user_agent":"cur

<TRUNCATED>


[11/15] incubator-metron git commit: METRON 86: Adding Solr indexing support (merrimanr via cestella) closes apache/incubator-metron#67

Posted by ce...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/e59b1a31/metron-streaming/Metron-Testing/src/main/java/org/apache/metron/integration/util/threatintel/ThreatIntelHelper.java
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-Testing/src/main/java/org/apache/metron/integration/util/threatintel/ThreatIntelHelper.java b/metron-streaming/Metron-Testing/src/main/java/org/apache/metron/integration/util/threatintel/ThreatIntelHelper.java
new file mode 100644
index 0000000..b44a48a
--- /dev/null
+++ b/metron-streaming/Metron-Testing/src/main/java/org/apache/metron/integration/util/threatintel/ThreatIntelHelper.java
@@ -0,0 +1,39 @@
+/**
+ * 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.util.threatintel;
+
+import org.apache.hadoop.hbase.client.HTableInterface;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.metron.hbase.converters.threatintel.ThreatIntelConverter;
+import org.apache.metron.hbase.converters.threatintel.ThreatIntelKey;
+import org.apache.metron.hbase.converters.threatintel.ThreatIntelValue;
+import org.apache.metron.reference.lookup.LookupKV;
+
+import java.io.IOException;
+
+public enum ThreatIntelHelper {
+    INSTANCE;
+    ThreatIntelConverter converter = new ThreatIntelConverter();
+
+    public void load(HTableInterface table, String cf, Iterable<LookupKV<ThreatIntelKey, ThreatIntelValue>> results) throws IOException {
+        for(LookupKV<ThreatIntelKey, ThreatIntelValue> result : results) {
+            Put put = converter.toPut(cf, result.getKey(), result.getValue());
+            table.put(put);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/e59b1a31/metron-streaming/Metron-Testing/src/main/java/org/apache/metron/util/SampleUtil.java
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-Testing/src/main/java/org/apache/metron/util/SampleUtil.java b/metron-streaming/Metron-Testing/src/main/java/org/apache/metron/util/SampleUtil.java
new file mode 100644
index 0000000..b15e682
--- /dev/null
+++ b/metron-streaming/Metron-Testing/src/main/java/org/apache/metron/util/SampleUtil.java
@@ -0,0 +1,40 @@
+/**
+ * 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.util;
+
+import org.apache.metron.domain.Configurations;
+import org.apache.metron.utils.ConfigurationsUtils;
+
+import java.io.IOException;
+import java.util.Map;
+
+public class SampleUtil {
+
+  public static final String sampleConfigRoot = "../Metron-Testing/src/main/resources/sample/config/";
+
+  public static Configurations getSampleConfigs() throws IOException {
+    Configurations configurations = new Configurations();
+    configurations.updateGlobalConfig(ConfigurationsUtils.readGlobalConfigFromFile(sampleConfigRoot));
+    Map<String, byte[]> sensorEnrichmentConfigs = ConfigurationsUtils.readSensorEnrichmentConfigsFromFile(sampleConfigRoot);
+    for(String sensorType: sensorEnrichmentConfigs.keySet()) {
+      configurations.updateSensorEnrichmentConfig(sensorType, sensorEnrichmentConfigs.get(sensorType));
+    }
+    return configurations;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/e59b1a31/metron-streaming/Metron-Testing/src/main/resources/sample/config/global.json
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-Testing/src/main/resources/sample/config/global.json b/metron-streaming/Metron-Testing/src/main/resources/sample/config/global.json
new file mode 100644
index 0000000..b8b449b
--- /dev/null
+++ b/metron-streaming/Metron-Testing/src/main/resources/sample/config/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/e59b1a31/metron-streaming/Metron-Testing/src/main/resources/sample/config/sensors/bro.json
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-Testing/src/main/resources/sample/config/sensors/bro.json b/metron-streaming/Metron-Testing/src/main/resources/sample/config/sensors/bro.json
new file mode 100644
index 0000000..34109b8
--- /dev/null
+++ b/metron-streaming/Metron-Testing/src/main/resources/sample/config/sensors/bro.json
@@ -0,0 +1,14 @@
+{
+  "index": "bro",
+  "batchSize": 5,
+  "enrichmentFieldMap":
+  {
+    "geo": ["ip_dst_addr", "ip_src_addr"],
+    "host": ["host"]
+  },
+  "threatIntelFieldMap":
+  {
+    "ip": ["ip_dst_addr", "ip_src_addr"]
+  }
+}
+

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/e59b1a31/metron-streaming/Metron-Testing/src/main/resources/sample/config/sensors/pcap.json
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-Testing/src/main/resources/sample/config/sensors/pcap.json b/metron-streaming/Metron-Testing/src/main/resources/sample/config/sensors/pcap.json
new file mode 100644
index 0000000..82c7c5e
--- /dev/null
+++ b/metron-streaming/Metron-Testing/src/main/resources/sample/config/sensors/pcap.json
@@ -0,0 +1,13 @@
+{
+  "index": "pcap",
+  "batchSize": 5,
+  "enrichmentFieldMap":
+  {
+    "geo": ["ip_src_addr", "ip_dst_addr"],
+    "host": ["ip_src_addr", "ip_dst_addr"]
+  },
+  "threatIntelFieldMap":
+  {
+    "ip": ["ip_src_addr", "ip_dst_addr"]
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/e59b1a31/metron-streaming/Metron-Testing/src/main/resources/sample/config/sensors/snort.json
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-Testing/src/main/resources/sample/config/sensors/snort.json b/metron-streaming/Metron-Testing/src/main/resources/sample/config/sensors/snort.json
new file mode 100644
index 0000000..1208637
--- /dev/null
+++ b/metron-streaming/Metron-Testing/src/main/resources/sample/config/sensors/snort.json
@@ -0,0 +1,14 @@
+{
+  "index": "snort",
+  "batchSize": 1,
+  "enrichmentFieldMap":
+  {
+    "geo": ["ip_dst_addr", "ip_src_addr"],
+    "host": ["host"]
+  },
+  "threatIntelFieldMap":
+  {
+    "ip": ["ip_dst_addr", "ip_src_addr"]
+  }
+}
+

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/e59b1a31/metron-streaming/Metron-Testing/src/main/resources/sample/config/sensors/yaf.json
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-Testing/src/main/resources/sample/config/sensors/yaf.json b/metron-streaming/Metron-Testing/src/main/resources/sample/config/sensors/yaf.json
new file mode 100644
index 0000000..cfdcbc2
--- /dev/null
+++ b/metron-streaming/Metron-Testing/src/main/resources/sample/config/sensors/yaf.json
@@ -0,0 +1,14 @@
+{
+  "index": "yaf",
+  "batchSize": 5,
+  "enrichmentFieldMap":
+  {
+    "geo": ["ip_src_addr", "ip_dst_addr"],
+    "host": ["ip_src_addr", "ip_dst_addr"]
+  },
+  "threatIntelFieldMap":
+  {
+    "ip": ["ip_src_addr", "ip_dst_addr"]
+  }
+}
+

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/e59b1a31/metron-streaming/Metron-Testing/src/main/resources/sample/data/SampleIndexed/YafIndexed
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-Testing/src/main/resources/sample/data/SampleIndexed/YafIndexed b/metron-streaming/Metron-Testing/src/main/resources/sample/data/SampleIndexed/YafIndexed
new file mode 100644
index 0000000..1c38406
--- /dev/null
+++ b/metron-streaming/Metron-Testing/src/main/resources/sample/data/SampleIndexed/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/e59b1a31/metron-streaming/Metron-Testing/src/main/resources/sample/data/SampleInput/.PCAPExampleOutput.crc
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-Testing/src/main/resources/sample/data/SampleInput/.PCAPExampleOutput.crc b/metron-streaming/Metron-Testing/src/main/resources/sample/data/SampleInput/.PCAPExampleOutput.crc
new file mode 100644
index 0000000..6e53497
Binary files /dev/null and b/metron-streaming/Metron-Testing/src/main/resources/sample/data/SampleInput/.PCAPExampleOutput.crc differ

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/e59b1a31/metron-streaming/Metron-Testing/src/main/resources/sample/data/SampleInput/AsaOutput
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-Testing/src/main/resources/sample/data/SampleInput/AsaOutput b/metron-streaming/Metron-Testing/src/main/resources/sample/data/SampleInput/AsaOutput
new file mode 100644
index 0000000..6009d48
--- /dev/null
+++ b/metron-streaming/Metron-Testing/src/main/resources/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


[09/15] incubator-metron git commit: METRON 86: Adding Solr indexing support (merrimanr via cestella) closes apache/incubator-metron#67

Posted by ce...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/e59b1a31/metron-streaming/Metron-Testing/src/main/resources/sample/data/SampleInput/FireeyeExampleOutput
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-Testing/src/main/resources/sample/data/SampleInput/FireeyeExampleOutput b/metron-streaming/Metron-Testing/src/main/resources/sample/data/SampleInput/FireeyeExampleOutput
new file mode 100644
index 0000000..0210010
--- /dev/null
+++ b/metron-streaming/Metron-Testing/src/main/resources/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



[02/15] incubator-metron git commit: METRON 86: Adding Solr indexing support (merrimanr via cestella) closes apache/incubator-metron#67

Posted by ce...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/e59b1a31/metron-streaming/Metron-Topologies/src/main/resources/SampleInput/LancopeExampleOutput
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-Topologies/src/main/resources/SampleInput/LancopeExampleOutput b/metron-streaming/Metron-Topologies/src/main/resources/SampleInput/LancopeExampleOutput
deleted file mode 100644
index b1bccf9..0000000
--- a/metron-streaming/Metron-Topologies/src/main/resources/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/e59b1a31/metron-streaming/Metron-Topologies/src/main/resources/SampleInput/PCAPExampleOutput
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-Topologies/src/main/resources/SampleInput/PCAPExampleOutput b/metron-streaming/Metron-Topologies/src/main/resources/SampleInput/PCAPExampleOutput
deleted file mode 100644
index e730181..0000000
Binary files a/metron-streaming/Metron-Topologies/src/main/resources/SampleInput/PCAPExampleOutput and /dev/null differ

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/e59b1a31/metron-streaming/Metron-Topologies/src/main/resources/SampleInput/PaloaltoOutput
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-Topologies/src/main/resources/SampleInput/PaloaltoOutput b/metron-streaming/Metron-Topologies/src/main/resources/SampleInput/PaloaltoOutput
deleted file mode 100644
index 16793a2..0000000
--- a/metron-streaming/Metron-Topologies/src/main/resources/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/e59b1a31/metron-streaming/Metron-Topologies/src/main/resources/SampleInput/SnortOutput
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-Topologies/src/main/resources/SampleInput/SnortOutput b/metron-streaming/Metron-Topologies/src/main/resources/SampleInput/SnortOutput
deleted file mode 100644
index 0497b0f..0000000
--- a/metron-streaming/Metron-Topologies/src/main/resources/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/e59b1a31/metron-streaming/Metron-Topologies/src/main/resources/SampleInput/SourcefireExampleOutput
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-Topologies/src/main/resources/SampleInput/SourcefireExampleOutput b/metron-streaming/Metron-Topologies/src/main/resources/SampleInput/SourcefireExampleOutput
deleted file mode 100644
index 5f177df..0000000
--- a/metron-streaming/Metron-Topologies/src/main/resources/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/e59b1a31/metron-streaming/Metron-Topologies/src/main/resources/SampleInput/YafExampleOutput
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-Topologies/src/main/resources/SampleInput/YafExampleOutput b/metron-streaming/Metron-Topologies/src/main/resources/SampleInput/YafExampleOutput
deleted file mode 100644
index 8f3ff44..0000000
--- a/metron-streaming/Metron-Topologies/src/main/resources/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

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/e59b1a31/metron-streaming/Metron-Topologies/src/main/resources/SampleParsed/SnortParsed
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-Topologies/src/main/resources/SampleParsed/SnortParsed b/metron-streaming/Metron-Topologies/src/main/resources/SampleParsed/SnortParsed
deleted file mode 100644
index 9b99cb7..0000000
--- a/metron-streaming/Metron-Topologies/src/main/resources/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":"test","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":"test","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":"test","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/e59b1a31/metron-streaming/Metron-Topologies/src/main/resources/SampleParsed/YafExampleParsed
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-Topologies/src/main/resources/SampleParsed/YafExampleParsed b/metron-streaming/Metron-Topologies/src/main/resources/SampleParsed/YafExampleParsed
deleted file mode 100644
index bbb90f3..0000000
--- a/metron-streaming/Metron-Topologies/src/main/resources/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":"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}


[13/15] incubator-metron git commit: METRON 86: Adding Solr indexing support (merrimanr via cestella) closes apache/incubator-metron#67

Posted by ce...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/e59b1a31/metron-streaming/Metron-Indexing/src/main/java/org/apache/metron/indexing/TelemetryIndexingBolt.java
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-Indexing/src/main/java/org/apache/metron/indexing/TelemetryIndexingBolt.java b/metron-streaming/Metron-Indexing/src/main/java/org/apache/metron/indexing/TelemetryIndexingBolt.java
deleted file mode 100644
index 21ecb18..0000000
--- a/metron-streaming/Metron-Indexing/src/main/java/org/apache/metron/indexing/TelemetryIndexingBolt.java
+++ /dev/null
@@ -1,256 +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.indexing;
-
-import java.io.IOException;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.Map;
-import java.util.Set;
-
-import org.apache.commons.configuration.Configuration;
-import org.json.simple.JSONObject;
-
-import backtype.storm.task.OutputCollector;
-import backtype.storm.task.TopologyContext;
-import backtype.storm.topology.OutputFieldsDeclarer;
-import backtype.storm.tuple.Fields;
-import backtype.storm.tuple.Tuple;
-import backtype.storm.tuple.Values;
-
-import org.apache.metron.helpers.topology.ErrorUtils;
-import org.apache.metron.index.interfaces.IndexAdapter;
-import org.apache.metron.json.serialization.JSONEncoderHelper;
-import org.apache.metron.metrics.MetricReporter;
-
-/**
- * 
- * Bolt for indexing telemetry messages into Elastic Search, Solr, Druid, etc...
- * For a list of all adapters provided please see org.apache.metron.indexing.adapters
- * 
- * As of release of this code the following adapters for indexing are provided:
- * <p>
- * <ul>
- * 
- * <li>ESBulkAdapter = adapter that can bulk index messages into ES
- * <li>ESBulkRotatingAdapter = adapter that can bulk index messages into ES,
- * rotate the index, and apply an alias to the rotated index
- * <ul>
- * <p>
- *
- */
-
-@SuppressWarnings("serial")
-public class TelemetryIndexingBolt extends AbstractIndexingBolt {
-
-	private JSONObject metricConfiguration;
-	private String _indexDateFormat;
-	
-	private Set<Tuple> tuple_queue = new HashSet<Tuple>();
-
-	public TelemetryIndexingBolt(String zookeeperUrl) {
-		super(zookeeperUrl);
-	}
-
-	/**
-	 * 
-	 * @param IndexIP
-	 *            ip of ElasticSearch/Solr/etc...
-	 * @return instance of bolt
-	 */
-	public TelemetryIndexingBolt withIndexIP(String IndexIP) {
-		_IndexIP = IndexIP;
-		return this;
-	}
-
-	/**
-	 * 
-	 * @param IndexPort
-	 *            port of ElasticSearch/Solr/etc...
-	 * @return instance of bolt
-	 */
-
-	public TelemetryIndexingBolt withIndexPort(int IndexPort) {
-		_IndexPort = IndexPort;
-		return this;
-	}
-
-	/**
-	 *
-	 * @param IndexName
-	 *            name of the index in ElasticSearch/Solr/etc...
-	 * @return instance of bolt
-	 */
-	public TelemetryIndexingBolt withIndexName(String IndexName) {
-		_IndexName = IndexName;
-		return this;
-	}
-
-	/**
-	 * 
-	 * @param ClusterName
-	 *            name of cluster to index into in ElasticSearch/Solr/etc...
-	 * @return instance of bolt
-	 */
-	public TelemetryIndexingBolt withClusterName(String ClusterName) {
-		_ClusterName = ClusterName;
-		return this;
-	}
-
-	/**
-	 * 
-	 * @param DocumentName
-	 *            name of document to be indexed in ElasticSearch/Solr/etc...
-	 * @return
-	 */
-
-	public TelemetryIndexingBolt withDocumentName(String DocumentName) {
-		_DocumentName = DocumentName;
-		return this;
-	}
-
-	/**
-	 * 
-	 * @param BulkIndexNumber
-	 *            number of documents to bulk index together
-	 * @return instance of bolt
-	 */
-	public TelemetryIndexingBolt withBulk(int BulkIndexNumber) {
-		_BulkIndexNumber = BulkIndexNumber;
-		return this;
-	}
-
-	/**
-	 * 
-	 * @param adapter
-	 *            adapter that handles indexing of JSON strings
-	 * @return instance of bolt
-	 */
-	public TelemetryIndexingBolt withIndexAdapter(IndexAdapter adapter) {
-		_adapter = adapter;
-
-		return this;
-	}
-	
-	/**
-	 * 
-	 * @param indexTimestamp
-	 *           timestamp to append to index names
-	 * @return instance of bolt
-	 */
-	public TelemetryIndexingBolt withIndexTimestamp(String indexTimestamp) {
-		_indexDateFormat = indexTimestamp;
-
-		return this;
-	}
-	/**
-	 * 
-	 * @param config
-	 *            - configuration for pushing metrics into graphite
-	 * @return instance of bolt
-	 */
-	public TelemetryIndexingBolt withMetricConfiguration(Configuration config) {
-		this.metricConfiguration = JSONEncoderHelper.getJSON(config
-				.subset("org.apache.metron.metrics"));
-		return this;
-	}
-
-	@SuppressWarnings("rawtypes")
-	@Override
-	void doPrepare(Map conf, TopologyContext topologyContext,
-			OutputCollector collector) throws IOException {
-
-		try {
-			
-			_adapter.initializeConnection(_IndexIP, _IndexPort,
-					_ClusterName, _IndexName, _DocumentName, _BulkIndexNumber, _indexDateFormat);
-			
-			_reporter = new MetricReporter();
-			_reporter.initialize(metricConfiguration,
-					TelemetryIndexingBolt.class);
-			this.registerCounters();
-		} catch (Exception e) {
-			
-			e.printStackTrace();
-					
-			JSONObject error = ErrorUtils.generateErrorMessage(new String("bulk index problem"), e);
-			_collector.emit("error", new Values(error));
-		}
-
-	}
-
-	public void execute(Tuple tuple) {
-
-		JSONObject message = null;
-
-		try {
-			LOG.trace("[Metron] Indexing bolt gets:  " + message);
-
-			message = (JSONObject) tuple.getValueByField("message");
-
-			if (message == null || message.isEmpty())
-				throw new Exception(
-						"Could not parse message from binary stream");
-
-			int result_code = _adapter.bulkIndex(message);
-
-			if (result_code == 0) {
-				tuple_queue.add(tuple);
-			} else if (result_code == 1) {
-				tuple_queue.add(tuple);
-				
-				Iterator<Tuple> iterator = tuple_queue.iterator();
-				while(iterator.hasNext())
-				{
-					Tuple setElement = iterator.next();
-					_collector.ack(setElement);
-					ackCounter.inc();
-				}
-				tuple_queue.clear();
-			} else if (result_code == 2) {
-				throw new Exception("Failed to index elements with client");
-			}
-
-		} catch (Exception e) {
-			e.printStackTrace();
-			
-			
-			Iterator<Tuple> iterator = tuple_queue.iterator();
-			while(iterator.hasNext())
-			{
-				Tuple setElement = iterator.next();
-				_collector.fail(setElement);
-				failCounter.inc();
-				
-				
-				JSONObject error = ErrorUtils.generateErrorMessage(new String("bulk index problem"), e);
-				_collector.emit("error", new Values(error));
-			}
-			tuple_queue.clear();
-
-			
-		}
-	}
-
-	@Override
-	public void declareOutputFields(OutputFieldsDeclarer declearer) {
-		declearer.declareStream("error", new Fields("Index"));
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/e59b1a31/metron-streaming/Metron-Indexing/src/main/java/org/apache/metron/indexing/adapters/AbstractIndexAdapter.java
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-Indexing/src/main/java/org/apache/metron/indexing/adapters/AbstractIndexAdapter.java b/metron-streaming/Metron-Indexing/src/main/java/org/apache/metron/indexing/adapters/AbstractIndexAdapter.java
deleted file mode 100644
index 58f5bed..0000000
--- a/metron-streaming/Metron-Indexing/src/main/java/org/apache/metron/indexing/adapters/AbstractIndexAdapter.java
+++ /dev/null
@@ -1,42 +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.indexing.adapters;
-
-import java.io.Serializable;
-
-import org.json.simple.JSONObject;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import org.apache.metron.index.interfaces.IndexAdapter;
-import org.apache.metron.indexing.AbstractIndexingBolt;
-
-@SuppressWarnings("serial")
-public abstract class AbstractIndexAdapter implements IndexAdapter, Serializable{
-	
-	protected static final Logger _LOG = LoggerFactory
-			.getLogger(AbstractIndexingBolt.class);
-
-
-	
-
-	abstract public boolean initializeConnection(String ip, int port,
-			String cluster_name, String index_name, String document_name,
-			int bulk, String date_format) throws Exception;
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/e59b1a31/metron-streaming/Metron-Indexing/src/main/java/org/apache/metron/indexing/adapters/ESBaseBulkAdapter.java
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-Indexing/src/main/java/org/apache/metron/indexing/adapters/ESBaseBulkAdapter.java b/metron-streaming/Metron-Indexing/src/main/java/org/apache/metron/indexing/adapters/ESBaseBulkAdapter.java
deleted file mode 100644
index 5e64b86..0000000
--- a/metron-streaming/Metron-Indexing/src/main/java/org/apache/metron/indexing/adapters/ESBaseBulkAdapter.java
+++ /dev/null
@@ -1,165 +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.indexing.adapters;
-
-import java.io.Serializable;
-import java.util.Iterator;
-import java.util.Map;
-
-import org.apache.commons.collections.Bag;
-import org.apache.commons.collections.bag.HashBag;
-import org.elasticsearch.action.bulk.BulkRequestBuilder;
-import org.elasticsearch.action.bulk.BulkResponse;
-import org.elasticsearch.action.index.IndexRequestBuilder;
-import org.elasticsearch.client.transport.TransportClient;
-import org.elasticsearch.common.settings.ImmutableSettings;
-import org.elasticsearch.common.settings.Settings;
-import org.elasticsearch.common.transport.InetSocketTransportAddress;
-import org.json.simple.JSONObject;
-
-@SuppressWarnings("serial")
-public class ESBaseBulkAdapter extends AbstractIndexAdapter implements
-		Serializable {
-
-	private int _bulk_size;
-	private String _index_name;
-	private String _document_name;
-	private String _cluster_name;
-	private int _port;
-	private String _ip;
-	public transient TransportClient client;
-
-	private Bag bulk_set;
-
-	private Settings settings;
-
-	@Override
-	public boolean initializeConnection(String ip, int port,
-			String cluster_name, String index_name, String document_name,
-			int bulk_size, String date_format) throws Exception {
-
-		bulk_set = new HashBag();
-
-		_LOG.trace("[Metron] Initializing ESBulkAdapter...");
-
-		try {
-			_ip = ip;
-			_port = port;
-			_cluster_name = cluster_name;
-			_index_name = index_name;
-			_document_name = document_name;
-			_bulk_size = bulk_size;
-
-			_LOG.trace("[Metron] Bulk indexing is set to: " + _bulk_size);
-
-			settings = ImmutableSettings.settingsBuilder()
-					.put("cluster.name", _cluster_name).build();
-			client = new TransportClient(settings)
-					.addTransportAddress(new InetSocketTransportAddress(_ip,
-							_port));
-
-			return true;
-		} catch (Exception e) {
-			e.printStackTrace();
-			return false;
-		}
-	}
-
-	/**
-	 * @param raw_message
-	 *            message to bulk index in Elastic Search
-	 * @return integer (0) loaded into a bulk queue, (1) bulk indexing executed,
-	 *         (2) error
-	 */
-	@SuppressWarnings("unchecked")
-	public int bulkIndex(JSONObject raw_message) {
-
-		boolean success = true;
-		int set_size = 0;
-
-		synchronized (bulk_set) {
-			bulk_set.add(raw_message);
-			set_size = bulk_set.size();
-			
-			_LOG.trace("[Metron] Bulk size is now: " + bulk_set.size());
-		}
-
-		try {
-
-			if (set_size >= _bulk_size) {
-				success = doIndex();
-
-				if (success)
-					return 1;
-				else
-					return 2;
-			}
-
-			return 0;
-
-		} catch (Exception e) {
-			e.printStackTrace();
-			return 2;
-		}
-	}
-
-	public boolean doIndex() throws Exception {
-
-		try {
-
-			synchronized (bulk_set) {
-				if (client == null)
-					throw new Exception("client is null");
-
-				BulkRequestBuilder bulkRequest = client.prepareBulk();
-
-				Iterator<JSONObject> iterator = bulk_set.iterator();
-
-				while (iterator.hasNext()) {
-					JSONObject setElement = iterator.next();
-
-					IndexRequestBuilder a = client.prepareIndex(_index_name,
-							_document_name);
-					a.setSource(setElement.toString());
-					bulkRequest.add(a);
-
-				}
-
-				_LOG.trace("[Metron] Performing bulk load of size: "
-						+ bulkRequest.numberOfActions());
-
-				BulkResponse resp = bulkRequest.execute().actionGet();
-				_LOG.trace("[Metron] Received bulk response: "
-						+ resp.toString());
-				bulk_set.clear();
-			}
-
-			return true;
-		}
-
-		catch (Exception e) {
-			e.printStackTrace();
-			return false;
-		}
-	}
-
-	public void setOptionalSettings(Map<String, String> settings) {
-		// TODO Auto-generated method stub
-		
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/e59b1a31/metron-streaming/Metron-Indexing/src/main/java/org/apache/metron/indexing/adapters/ESBulkRotatingAdapter.java
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-Indexing/src/main/java/org/apache/metron/indexing/adapters/ESBulkRotatingAdapter.java b/metron-streaming/Metron-Indexing/src/main/java/org/apache/metron/indexing/adapters/ESBulkRotatingAdapter.java
deleted file mode 100644
index 1f8c50e..0000000
--- a/metron-streaming/Metron-Indexing/src/main/java/org/apache/metron/indexing/adapters/ESBulkRotatingAdapter.java
+++ /dev/null
@@ -1,177 +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.indexing.adapters;
-
-import java.text.DateFormat;
-import java.text.SimpleDateFormat;
-import java.util.Date;
-import java.util.Map;
-
-import org.apache.http.HttpResponse;
-import org.apache.http.client.HttpClient;
-import org.apache.http.client.methods.HttpPost;
-import org.apache.http.entity.StringEntity;
-import org.apache.http.impl.client.DefaultHttpClient;
-import org.apache.http.util.EntityUtils;
-import org.elasticsearch.action.bulk.BulkRequestBuilder;
-import org.elasticsearch.action.bulk.BulkResponse;
-import org.elasticsearch.client.Client;
-import org.elasticsearch.client.transport.TransportClient;
-import org.elasticsearch.common.settings.ImmutableSettings;
-import org.elasticsearch.common.settings.Settings;
-import org.elasticsearch.common.transport.InetSocketTransportAddress;
-import org.json.simple.JSONObject;
-
-@SuppressWarnings({ "deprecation", "serial" })
-public class ESBulkRotatingAdapter extends AbstractIndexAdapter {
-
-	private Client client;
-	private BulkRequestBuilder bulkRequest;
-	private int _bulk_size;
-	private String _index_name;
-	private String _document_name;
-	private int element_count;
-	private String index_postfix;
-	private String running_index_postfix;
-
-	private HttpClient httpclient;
-	private HttpPost post;
-
-	private DateFormat dateFormat;
-
-	public boolean initializeConnection(String ip, int port,
-			String cluster_name, String index_name, String document_name,
-			int bulk_size, String date_format) {
-
-		_LOG.info("Initializing ESBulkAdapter...");
-
-		try {
-			httpclient = new DefaultHttpClient();
-			String alias_link = "http://" + ip + ":" + 9200 + "/_aliases";
-			post = new HttpPost(alias_link);
-
-			_index_name = index_name;
-			_document_name = document_name;
-
-			_bulk_size = bulk_size - 1;
-			
-
-			dateFormat = new SimpleDateFormat(date_format);
-			
-			element_count = 0;
-			running_index_postfix = "NONE";
-
-			Settings settings = ImmutableSettings.settingsBuilder()
-					.put("cluster.name", cluster_name).build();
-			client = new TransportClient(settings)
-					.addTransportAddress(new InetSocketTransportAddress(ip,
-							port));
-
-			bulkRequest = client.prepareBulk();
-
-			return true;
-		} catch (Exception e) {
-			e.printStackTrace();
-			return false;
-		}
-	}
-
-	@SuppressWarnings("unchecked")
-	public int bulkIndex(JSONObject raw_message) {
-
-		index_postfix = dateFormat.format(new Date());
-
-		bulkRequest.add(client.prepareIndex(_index_name + "_" + index_postfix,
-				_document_name).setSource(raw_message));
-
-		return doIndex();
-	}
-
-	public int bulkIndex(String raw_message) {
-
-		index_postfix = dateFormat.format(new Date());
-
-		bulkRequest.add(client.prepareIndex(_index_name + "_" + index_postfix,
-				_document_name).setSource(raw_message));
-
-		return doIndex();
-	}
-
-	public int doIndex() {
-
-		element_count++;
-		
-		if (element_count != _bulk_size)
-			return 0;
-
-		if (element_count == _bulk_size) {
-			_LOG.debug("Starting bulk load of size: " + _bulk_size);
-			BulkResponse resp = bulkRequest.execute().actionGet();
-			element_count = 0;
-			_LOG.debug("Received bulk response: " + resp.toString());
-
-			if (resp.hasFailures()) {
-				_LOG.error("Bulk update failed");
-				return 2;
-			}
-
-			if (!running_index_postfix.equals(index_postfix)) {
-
-				_LOG.debug("Attempting to apply a new alias");
-
-				try {
-
-					String alias = "{\"actions\" : [{ \"add\" : { \"index\" : \""
-							+ _index_name
-							+ "-"
-							+ index_postfix
-							+ "\", \"alias\" : \"" + _index_name + "\" } } ]}";
-
-					post.setEntity(new StringEntity(alias));
-
-					HttpResponse response = httpclient.execute(post);
-					String res = EntityUtils.toString(response.getEntity());
-
-					_LOG.debug("Alias request received the following response: "
-							+ res);
-
-					running_index_postfix = index_postfix;
-				}
-
-				catch (Exception e) {
-					e.printStackTrace();
-					_LOG.error("Alias request failed...");
-					return 2;
-				}
-			}
-
-			index_postfix = dateFormat.format(new Date());
-		}
-
-		_LOG.debug("Adding to bulk load: element " + element_count
-				+ " of bulk size " + _bulk_size);
-
-		return 1;
-	}
-
-	public void setOptionalSettings(Map<String, String> settings) {
-		// TODO Auto-generated method stub
-		
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/e59b1a31/metron-streaming/Metron-Indexing/src/main/java/org/apache/metron/indexing/adapters/ESTimedRotatingAdapter.java
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-Indexing/src/main/java/org/apache/metron/indexing/adapters/ESTimedRotatingAdapter.java b/metron-streaming/Metron-Indexing/src/main/java/org/apache/metron/indexing/adapters/ESTimedRotatingAdapter.java
deleted file mode 100644
index fd4c067..0000000
--- a/metron-streaming/Metron-Indexing/src/main/java/org/apache/metron/indexing/adapters/ESTimedRotatingAdapter.java
+++ /dev/null
@@ -1,208 +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.indexing.adapters;
-
-import java.io.Serializable;
-import java.text.DateFormat;
-import java.text.SimpleDateFormat;
-import java.util.Date;
-import java.util.Iterator;
-import java.util.Map;
-
-import org.apache.commons.collections.Bag;
-import org.apache.commons.collections.HashBag;
-import org.elasticsearch.action.bulk.BulkItemResponse;
-import org.elasticsearch.action.bulk.BulkRequestBuilder;
-import org.elasticsearch.action.bulk.BulkResponse;
-import org.elasticsearch.action.index.IndexRequestBuilder;
-import org.elasticsearch.client.transport.TransportClient;
-import org.elasticsearch.common.settings.ImmutableSettings;
-import org.elasticsearch.common.settings.Settings;
-import org.elasticsearch.common.transport.InetSocketTransportAddress;
-import org.json.simple.JSONObject;
-
-@SuppressWarnings("serial")
-public class ESTimedRotatingAdapter extends AbstractIndexAdapter implements
-		Serializable {
-
-	private int _bulk_size;
-	private String _index_name;
-	private String _document_name;
-	private String _cluster_name;
-	private int _port;
-	private String _ip;
-	public transient TransportClient client;
-	private DateFormat dateFormat;
-	
-	private Map<String, String> tuning_settings;
-
-	private Bag bulk_set;
-
-	private Settings settings;
-	
-	public void setOptionalSettings(Map<String, String> settings)
-	{
-		tuning_settings = settings;
-	}
-
-	@Override
-	public boolean initializeConnection(String ip, int port,
-			String cluster_name, String index_name, String document_name,
-			int bulk_size, String date_format) throws Exception {
-
-		bulk_set = new HashBag();
-
-		_LOG.trace("[Metron] Initializing ESBulkAdapter...");
-
-		try {
-			_ip = ip;
-			_port = port;
-			_cluster_name = cluster_name;
-			_index_name = index_name;
-			_document_name = document_name;
-			_bulk_size = bulk_size;
-			
-
-			dateFormat = new SimpleDateFormat(date_format);
-
-			System.out.println("Bulk indexing is set to: " + _bulk_size);
-
-			ImmutableSettings.Builder builder = ImmutableSettings.settingsBuilder() ;	
-			
-			if(tuning_settings != null && tuning_settings.size() > 0)
-			{
-					builder.put(tuning_settings);
-			}
-			
-			builder.put("cluster.name", _cluster_name);
-			builder.put("client.transport.ping_timeout","500s");
-			
-			
-			settings = builder.build();
-					
-			client = new TransportClient(settings)
-					.addTransportAddress(new InetSocketTransportAddress(_ip,
-							_port));
-
-			return true;
-		} catch (Exception e) {
-			e.printStackTrace();
-			return false;
-		}
-	}
-
-	/**
-	 * @param raw_message
-	 *            message to bulk index in Elastic Search
-	 * @return integer (0) loaded into a bulk queue, (1) bulk indexing executed,
-	 *         (2) error
-	 */
-	@SuppressWarnings("unchecked")
-	public int bulkIndex(JSONObject raw_message) {
-
-		boolean success = true;
-		int set_size = 0;
-
-		synchronized (bulk_set) {
-			bulk_set.add(raw_message);
-			set_size = bulk_set.size();
-			
-			_LOG.trace("[Metron] Incremented bulk size to: " + bulk_set.size());
-		}
-
-		try {
-
-			if (set_size >= _bulk_size) {
-				success = doIndex();
-
-				if (success)
-					return 1;
-				else
-					return 2;
-			}
-
-			return 0;
-
-		} catch (Exception e) {
-			e.printStackTrace();
-			return 2;
-		}
-	}
-
-	public boolean doIndex() throws Exception {
-
-		try {
-
-			synchronized (bulk_set) {
-				if (client == null)
-					throw new Exception("client is null");
-
-				BulkRequestBuilder bulkRequest = client.prepareBulk();
-
-				Iterator<JSONObject> iterator = bulk_set.iterator();
-				
-				String index_postfix = dateFormat.format(new Date());
-
-				while (iterator.hasNext()) {
-					JSONObject setElement = iterator.next();
-					
-					_LOG.trace("[Metron] Flushing to index: " + _index_name+ "_" + index_postfix);
-
-					IndexRequestBuilder a = client.prepareIndex(_index_name+ "_" + index_postfix,
-							_document_name);
-					a.setSource(setElement.toString());
-					bulkRequest.add(a);
-
-				}
-
-				_LOG.trace("[Metron] Performing bulk load of size: "
-						+ bulkRequest.numberOfActions());
-
-				BulkResponse resp = bulkRequest.execute().actionGet();
-				
-				for(BulkItemResponse r: resp.getItems())
-				{
-					r.getResponse();
-					_LOG.trace("[Metron] ES SUCCESS MESSAGE: " + r.getFailureMessage());
-				}
-
-
-				bulk_set.clear();
-				
-				if (resp.hasFailures()) {
-					_LOG.error("[Metron] Received bulk response error: "
-							+ resp.buildFailureMessage());
-					
-					for(BulkItemResponse r: resp.getItems())
-					{
-						r.getResponse();
-						_LOG.error("[Metron] ES FAILURE MESSAGE: " + r.getFailureMessage());
-					}
-				}
-				
-			}
-
-			return true;
-		}
-
-		catch (Exception e) {
-			e.printStackTrace();
-			return false;
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/e59b1a31/metron-streaming/Metron-Indexing/src/main/java/org/apache/metron/indexing/adapters/SolrAdapter.java
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-Indexing/src/main/java/org/apache/metron/indexing/adapters/SolrAdapter.java b/metron-streaming/Metron-Indexing/src/main/java/org/apache/metron/indexing/adapters/SolrAdapter.java
deleted file mode 100644
index 13f02f4..0000000
--- a/metron-streaming/Metron-Indexing/src/main/java/org/apache/metron/indexing/adapters/SolrAdapter.java
+++ /dev/null
@@ -1,22 +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.indexing.adapters;
-
-public class SolrAdapter {
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/e59b1a31/metron-streaming/Metron-Indexing/src/main/java/org/apache/metron/writer/ElasticsearchWriter.java
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-Indexing/src/main/java/org/apache/metron/writer/ElasticsearchWriter.java b/metron-streaming/Metron-Indexing/src/main/java/org/apache/metron/writer/ElasticsearchWriter.java
deleted file mode 100644
index e8d654d..0000000
--- a/metron-streaming/Metron-Indexing/src/main/java/org/apache/metron/writer/ElasticsearchWriter.java
+++ /dev/null
@@ -1,101 +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.writer;
-
-import backtype.storm.tuple.Tuple;
-import org.apache.metron.domain.SourceConfig;
-import org.apache.metron.writer.interfaces.BulkMessageWriter;
-import org.elasticsearch.action.bulk.BulkRequestBuilder;
-import org.elasticsearch.action.bulk.BulkResponse;
-import org.elasticsearch.action.index.IndexRequestBuilder;
-import org.elasticsearch.client.transport.TransportClient;
-import org.elasticsearch.common.settings.ImmutableSettings;
-import org.elasticsearch.common.transport.InetSocketTransportAddress;
-import org.json.simple.JSONObject;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.Serializable;
-import java.text.SimpleDateFormat;
-import java.util.Date;
-import java.util.List;
-import java.util.Map;
-
-public class ElasticsearchWriter implements BulkMessageWriter<JSONObject>, Serializable {
-
-  private String clusterName;
-  private Map<String, String> optionalSettings;
-  private transient TransportClient client;
-  private String host;
-  private int port;
-  private SimpleDateFormat dateFormat;
-  private static final Logger LOG = LoggerFactory
-          .getLogger(ElasticsearchWriter.class);
-
-  public ElasticsearchWriter(String clusterName, String host, int port, String dateFormat) {
-    this.clusterName = clusterName;
-    this.host = host;
-    this.port = port;
-    this.dateFormat = new SimpleDateFormat(dateFormat);
-  }
-
-  public ElasticsearchWriter withOptionalSettings(Map<String, String> optionalSettings) {
-    this.optionalSettings = optionalSettings;
-    return this;
-  }
-
-  @Override
-  public void init(Map stormConf) {
-    ImmutableSettings.Builder builder = ImmutableSettings.settingsBuilder();
-    builder.put("cluster.name", clusterName);
-    builder.put("client.transport.ping_timeout","500s");
-    if (optionalSettings != null) {
-      builder.put(optionalSettings);
-    }
-    client = new TransportClient(builder.build())
-            .addTransportAddress(new InetSocketTransportAddress(host, port))
-            ;
-
-  }
-
-  @Override
-  public void write(String sourceType, SourceConfig configuration, List<Tuple> tuples, List<JSONObject> messages) throws Exception {
-    String indexPostfix = dateFormat.format(new Date());
-    BulkRequestBuilder bulkRequest = client.prepareBulk();
-    for(JSONObject message: messages) {
-      String indexName = sourceType;
-      if (configuration != null) {
-        indexName = configuration.getIndex();
-      }
-      IndexRequestBuilder indexRequestBuilder = client.prepareIndex(indexName + "_index_" + indexPostfix,
-              sourceType);
-
-      indexRequestBuilder.setSource(message.toJSONString());
-      bulkRequest.add(indexRequestBuilder);
-    }
-    BulkResponse resp = bulkRequest.execute().actionGet();
-    if (resp.hasFailures()) {
-      throw new Exception(resp.buildFailureMessage());
-    }
-  }
-
-  @Override
-  public void close() throws Exception {
-    client.close();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/e59b1a31/metron-streaming/Metron-Indexing/src/main/java/org/apache/metron/writer/hdfs/HdfsWriter.java
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-Indexing/src/main/java/org/apache/metron/writer/hdfs/HdfsWriter.java b/metron-streaming/Metron-Indexing/src/main/java/org/apache/metron/writer/hdfs/HdfsWriter.java
index d2cc827..591f9e3 100644
--- a/metron-streaming/Metron-Indexing/src/main/java/org/apache/metron/writer/hdfs/HdfsWriter.java
+++ b/metron-streaming/Metron-Indexing/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.domain.SourceConfig;
+import org.apache.metron.domain.Configurations;
 import org.apache.metron.writer.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) {
+  public void init(Map stormConfig, Configurations configurations) {
     this.stormConfig = stormConfig;
   }
 
   @Override
   public void write( String sourceType
-                   , SourceConfig configuration
+                   , Configurations configurations
                    , List<Tuple> tuples
                    , List<JSONObject> messages
                    ) throws Exception

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/e59b1a31/metron-streaming/Metron-MessageParsers/src/main/java/org/apache/metron/bolt/ParserBolt.java
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-MessageParsers/src/main/java/org/apache/metron/bolt/ParserBolt.java b/metron-streaming/Metron-MessageParsers/src/main/java/org/apache/metron/bolt/ParserBolt.java
index 27294ef..abdb207 100644
--- a/metron-streaming/Metron-MessageParsers/src/main/java/org/apache/metron/bolt/ParserBolt.java
+++ b/metron-streaming/Metron-MessageParsers/src/main/java/org/apache/metron/bolt/ParserBolt.java
@@ -38,12 +38,12 @@ public class ParserBolt extends ConfiguredBolt {
   private MessageParser<JSONObject> parser;
   private MessageFilter<JSONObject> filter = new GenericMessageFilter();
   private MessageWriter<JSONObject> writer;
-  private String sourceType;
+  private String sensorType;
 
-  public ParserBolt(String zookeeperUrl, String sourceType, MessageParser<JSONObject> parser, MessageWriter<JSONObject> writer) {
+  public ParserBolt(String zookeeperUrl, String sensorType, MessageParser<JSONObject> parser, MessageWriter<JSONObject> writer) {
     super(zookeeperUrl);
     this.parser = parser;
-    this.sourceType = sourceType;
+    this.sensorType = sensorType;
     this.writer = writer;
   }
 
@@ -70,8 +70,8 @@ public class ParserBolt extends ConfiguredBolt {
       for(JSONObject message: messages) {
         if (parser.validate(message)) {
           if (filter != null && filter.emitTuple(message)) {
-            message.put(Constants.SOURCE_TYPE, sourceType);
-            writer.write(sourceType, configurations.get(sourceType), tuple, message);
+            message.put(Constants.SENSOR_TYPE, sensorType);
+            writer.write(sensorType, configurations, tuple, message);
           }
         }
       }

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/e59b1a31/metron-streaming/Metron-MessageParsers/src/main/java/org/apache/metron/writer/KafkaWriter.java
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-MessageParsers/src/main/java/org/apache/metron/writer/KafkaWriter.java b/metron-streaming/Metron-MessageParsers/src/main/java/org/apache/metron/writer/KafkaWriter.java
index 8372e14..ec323d6 100644
--- a/metron-streaming/Metron-MessageParsers/src/main/java/org/apache/metron/writer/KafkaWriter.java
+++ b/metron-streaming/Metron-MessageParsers/src/main/java/org/apache/metron/writer/KafkaWriter.java
@@ -21,7 +21,7 @@ import backtype.storm.tuple.Tuple;
 import org.apache.kafka.clients.producer.KafkaProducer;
 import org.apache.kafka.clients.producer.ProducerRecord;
 import org.apache.metron.Constants;
-import org.apache.metron.domain.SourceConfig;
+import org.apache.metron.domain.Configurations;
 import org.apache.metron.writer.interfaces.MessageWriter;
 import org.json.simple.JSONObject;
 
@@ -68,7 +68,7 @@ public class KafkaWriter implements MessageWriter<JSONObject>, Serializable {
 
   @SuppressWarnings("unchecked")
   @Override
-  public void write(String sourceType, SourceConfig configuration, Tuple tuple, JSONObject message) throws Exception {
+  public void write(String sourceType, Configurations configurations, Tuple tuple, JSONObject message) throws Exception {
     kafkaProducer.send(new ProducerRecord<String, String>(Constants.ENRICHMENT_TOPIC, message.toJSONString()));
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/e59b1a31/metron-streaming/Metron-Solr/pom.xml
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-Solr/pom.xml b/metron-streaming/Metron-Solr/pom.xml
new file mode 100644
index 0000000..cbb7395
--- /dev/null
+++ b/metron-streaming/Metron-Solr/pom.xml
@@ -0,0 +1,204 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  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.
+  -->
+
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <modelVersion>4.0.0</modelVersion>
+    <parent>
+        <groupId>org.apache.metron</groupId>
+        <artifactId>Metron-Streaming</artifactId>
+        <version>0.1BETA</version>
+    </parent>
+    <artifactId>Metron-Solr</artifactId>
+    <properties>
+        <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
+        <project.reporting.outputEncoding>UTF-8</project.reporting.outputEncoding>
+    </properties>
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.metron</groupId>
+            <artifactId>Metron-Common</artifactId>
+            <version>${project.parent.version}</version>
+            <exclusions>
+                <exclusion>
+                    <artifactId>servlet-api</artifactId>
+                    <groupId>javax.servlet</groupId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.solr</groupId>
+            <artifactId>solr-solrj</artifactId>
+            <version>${global_solr_version}</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.solr</groupId>
+            <artifactId>solr-test-framework</artifactId>
+            <version>${global_solr_version}</version>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.storm</groupId>
+            <artifactId>storm-core</artifactId>
+            <version>${global_storm_version}</version>
+            <scope>provided</scope>
+            <exclusions>
+                <exclusion>
+                    <artifactId>servlet-api</artifactId>
+                    <groupId>javax.servlet</groupId>
+                </exclusion>
+                <exclusion>
+                    <artifactId>log4j-over-slf4j</artifactId>
+                    <groupId>org.slf4j</groupId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.metron</groupId>
+            <artifactId>Metron-Testing</artifactId>
+            <version>${project.parent.version}</version>
+            <scope>test</scope>
+            <exclusions>
+                <exclusion>
+                    <artifactId>servlet-api</artifactId>
+                    <groupId>javax.servlet</groupId>
+                </exclusion>
+                <exclusion>
+                    <groupId>org.apache.httpcomponents</groupId>
+                    <artifactId>httpclient</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.metron</groupId>
+            <artifactId>Metron-Topologies</artifactId>
+            <version>${project.parent.version}</version>
+        </dependency>
+
+        <dependency>
+            <groupId>org.mockito</groupId>
+            <artifactId>mockito-all</artifactId>
+            <version>${global_mockito_version}</version>
+            <scope>test</scope>
+        </dependency>
+
+    </dependencies>
+
+    <build>
+        <plugins>
+            <plugin>
+                <!-- Separates the unit tests from the integration tests. -->
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-surefire-plugin</artifactId>
+                <version>2.12.4</version>
+                <configuration>
+                    <!-- Skip the default running of this plug-in (or everything is run twice...see below) -->
+                    <argLine>-Xmx2048m -XX:MaxPermSize=256m</argLine>
+                    <skip>true</skip>
+                    <!-- Show 100% of the lines from the stack trace (doesn't work) -->
+                    <trimStackTrace>false</trimStackTrace>
+                </configuration>
+                <executions>
+                    <execution>
+                        <id>unit-tests</id>
+                        <phase>test</phase>
+                        <goals>
+                            <goal>test</goal>
+                        </goals>
+                        <configuration>
+                            <!-- Never skip running the tests when the test phase is invoked -->
+                            <skip>false</skip>
+                            <includes>
+                                <!-- Include unit tests within integration-test phase. -->
+                                <include>**/*Test.java</include>
+                            </includes>
+                            <excludes>
+                                <!-- Exclude integration tests within (unit) test phase. -->
+                                <exclude>**/*IntegrationTest.java</exclude>
+                            </excludes>
+                        </configuration>
+                    </execution>
+                    <execution>
+                        <id>integration-tests</id>
+                        <phase>integration-test</phase>
+                        <goals>
+                            <goal>test</goal>
+                        </goals>
+                        <configuration>
+                            <!-- Never skip running the tests when the integration-test phase is invoked -->
+                            <skip>false</skip>
+                            <includes>
+                                <!-- Include integration tests within integration-test phase. -->
+                                <include>**/*IntegrationTest.java</include>
+                            </includes>
+                        </configuration>
+                    </execution>
+                </executions>
+            </plugin>
+            <plugin>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-shade-plugin</artifactId>
+                <version>2.3</version>
+                <configuration>
+                    <createDependencyReducedPom>true</createDependencyReducedPom>
+                </configuration>
+                <executions>
+                    <execution>
+                        <phase>package</phase>
+                        <goals>
+                            <goal>shade</goal>
+                        </goals>
+                        <configuration>
+                            <artifactSet>
+                                <excludes>
+                                    <exclude>storm:storm-core:*</exclude>
+                                    <exclude>storm:storm-lib:*</exclude>
+                                    <exclude>org.slf4j.impl*</exclude>
+                                    <exclude>org.slf4j:slf4j-log4j*</exclude>
+                                </excludes>
+                            </artifactSet>
+                            <transformers>
+                                <transformer
+                                        implementation="org.apache.maven.plugins.shade.resource.DontIncludeResourceTransformer">
+                                    <resource>.yaml</resource>
+                                </transformer>
+                                <transformer
+                                        implementation="org.apache.maven.plugins.shade.resource.ServicesResourceTransformer"/>
+                                <transformer
+                                        implementation="org.apache.maven.plugins.shade.resource.ManifestResourceTransformer">
+                                    <mainClass></mainClass>
+                                </transformer>
+                            </transformers>
+                        </configuration>
+                    </execution>
+                </executions>
+            </plugin>
+            <plugin>
+                <artifactId>maven-assembly-plugin</artifactId>
+                <configuration>
+                    <descriptor>src/main/assembly/assembly.xml</descriptor>
+                </configuration>
+                <executions>
+                    <execution>
+                        <id>make-assembly</id> <!-- this is used for inheritance merges -->
+                        <phase>package</phase> <!-- bind to the packaging phase -->
+                        <goals>
+                            <goal>single</goal>
+                        </goals>
+                    </execution>
+                </executions>
+            </plugin>
+        </plugins>
+    </build>
+</project>

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/e59b1a31/metron-streaming/Metron-Solr/src/main/assembly/assembly.xml
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-Solr/src/main/assembly/assembly.xml b/metron-streaming/Metron-Solr/src/main/assembly/assembly.xml
new file mode 100644
index 0000000..35cbcc3
--- /dev/null
+++ b/metron-streaming/Metron-Solr/src/main/assembly/assembly.xml
@@ -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.
+  -->
+
+<assembly>
+  <id>archive</id>
+  <formats>
+    <format>tar.gz</format>
+  </formats>
+  <includeBaseDirectory>false</includeBaseDirectory>
+  <fileSets>
+    <fileSet>
+      <directory>${project.basedir}/src/main/resources/Metron_Configs/etc</directory>
+      <outputDirectory>/config/etc</outputDirectory>
+      <useDefaultExcludes>true</useDefaultExcludes>
+      <excludes>
+        <exclude>**/*.formatted</exclude>
+        <exclude>**/*.filtered</exclude>
+      </excludes>
+      <fileMode>0644</fileMode>
+      <lineEnding>unix</lineEnding>
+    </fileSet>
+    <fileSet>
+      <directory>${project.basedir}/target</directory>
+      <includes>
+        <include>${project.artifactId}-${project.version}.jar</include>
+      </includes>
+      <outputDirectory>/lib</outputDirectory>
+      <useDefaultExcludes>true</useDefaultExcludes>
+    </fileSet>
+  </fileSets>
+</assembly>

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/e59b1a31/metron-streaming/Metron-Solr/src/main/java/org/apache/metron/solr/SolrConstants.java
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-Solr/src/main/java/org/apache/metron/solr/SolrConstants.java b/metron-streaming/Metron-Solr/src/main/java/org/apache/metron/solr/SolrConstants.java
new file mode 100644
index 0000000..d5dc7a0
--- /dev/null
+++ b/metron-streaming/Metron-Solr/src/main/java/org/apache/metron/solr/SolrConstants.java
@@ -0,0 +1,29 @@
+/**
+ * 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.solr;
+
+public class SolrConstants {
+
+  public static final String REQUEST_ACTION = "action";
+  public static final String REQUEST_NAME = "name";
+  public static final String REQUEST_NUM_SHARDS = "numShards";
+  public static final String REQUEST_REPLICATION_FACTOR = "replicationFactor";
+  public static final String REQUEST_COLLECTION_CONFIG_NAME = "collection.configName";
+  public static final String REQUEST_COLLECTIONS_PATH = "/admin/collections";
+  public static final String RESPONSE_COLLECTIONS = "collections";
+}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/e59b1a31/metron-streaming/Metron-Solr/src/main/java/org/apache/metron/writer/solr/MetronSolrClient.java
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-Solr/src/main/java/org/apache/metron/writer/solr/MetronSolrClient.java b/metron-streaming/Metron-Solr/src/main/java/org/apache/metron/writer/solr/MetronSolrClient.java
new file mode 100644
index 0000000..e0485ab
--- /dev/null
+++ b/metron-streaming/Metron-Solr/src/main/java/org/apache/metron/writer/solr/MetronSolrClient.java
@@ -0,0 +1,72 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.metron.writer.solr;
+
+import org.apache.log4j.Logger;
+import org.apache.metron.solr.SolrConstants;
+import org.apache.solr.client.solrj.SolrServerException;
+import org.apache.solr.client.solrj.impl.CloudSolrClient;
+import org.apache.solr.client.solrj.request.QueryRequest;
+import org.apache.solr.common.params.CollectionParams;
+import org.apache.solr.common.params.ModifiableSolrParams;
+import org.apache.solr.common.util.NamedList;
+
+import java.io.IOException;
+import java.util.List;
+
+public class MetronSolrClient extends CloudSolrClient {
+
+  private static final Logger LOG = Logger.getLogger(MetronSolrClient.class);
+
+
+  public MetronSolrClient(String zkHost) {
+    super(zkHost);
+  }
+
+  public void createCollection(String name, int numShards, int replicationFactor) throws IOException, SolrServerException {
+    if (!listCollections().contains(name)) {
+      request(getCreateCollectionsRequest(name, numShards, replicationFactor));
+    }
+  }
+
+  public QueryRequest getCreateCollectionsRequest(String name, int numShards, int replicationFactor) {
+    ModifiableSolrParams params = new ModifiableSolrParams();
+    params.set(SolrConstants.REQUEST_ACTION, CollectionParams.CollectionAction.CREATE.name());
+    params.set(SolrConstants.REQUEST_NAME, name);
+    params.set(SolrConstants.REQUEST_NUM_SHARDS, numShards);
+    params.set(SolrConstants.REQUEST_REPLICATION_FACTOR, replicationFactor);
+    params.set(SolrConstants.REQUEST_COLLECTION_CONFIG_NAME, name);
+    QueryRequest request = new QueryRequest(params);
+    request.setPath(SolrConstants.REQUEST_COLLECTIONS_PATH);
+    return request;
+  }
+
+  @SuppressWarnings("unchecked")
+  public List<String> listCollections() throws IOException, SolrServerException {
+    NamedList<Object> response = request(getListCollectionsRequest(), null);
+    return (List<String>) response.get(SolrConstants.RESPONSE_COLLECTIONS);
+  }
+
+  public QueryRequest getListCollectionsRequest() {
+    ModifiableSolrParams params = new ModifiableSolrParams();
+    params.set(SolrConstants.REQUEST_ACTION, CollectionParams.CollectionAction.LIST.name());
+    QueryRequest request = new QueryRequest(params);
+    request.setPath(SolrConstants.REQUEST_COLLECTIONS_PATH);
+    return request;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/e59b1a31/metron-streaming/Metron-Solr/src/main/java/org/apache/metron/writer/solr/SolrWriter.java
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-Solr/src/main/java/org/apache/metron/writer/solr/SolrWriter.java b/metron-streaming/Metron-Solr/src/main/java/org/apache/metron/writer/solr/SolrWriter.java
new file mode 100644
index 0000000..68303ea
--- /dev/null
+++ b/metron-streaming/Metron-Solr/src/main/java/org/apache/metron/writer/solr/SolrWriter.java
@@ -0,0 +1,108 @@
+/**
+ * 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.writer.solr;
+
+import backtype.storm.tuple.Tuple;
+import org.apache.log4j.Logger;
+import org.apache.metron.domain.Configurations;
+import org.apache.metron.writer.interfaces.BulkMessageWriter;
+import org.apache.solr.client.solrj.SolrServerException;
+import org.apache.solr.client.solrj.response.UpdateResponse;
+import org.apache.solr.common.SolrInputDocument;
+import org.json.simple.JSONObject;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.List;
+import java.util.Map;
+
+public class SolrWriter implements BulkMessageWriter<JSONObject>, Serializable {
+
+  public static final String DEFAULT_COLLECTION = "metron";
+
+  private static final Logger LOG = Logger.getLogger(SolrWriter.class);
+
+  private boolean shouldCommit = false;
+  private MetronSolrClient solr;
+
+  public SolrWriter withShouldCommit(boolean shouldCommit) {
+    this.shouldCommit = shouldCommit;
+    return this;
+  }
+
+  public SolrWriter withMetronSolrClient(MetronSolrClient solr) {
+    this.solr = solr;
+    return this;
+  }
+
+  @Override
+  public void init(Map stormConf, Configurations 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);
+    solr.createCollection(collection, (Integer) globalConfiguration.get("solr.numShards"), (Integer) globalConfiguration.get("solr.replicationFactor"));
+    solr.setDefaultCollection(collection);
+  }
+
+  @Override
+  public void write(String sourceType, Configurations configurations, List<Tuple> tuples, List<JSONObject> messages) throws Exception {
+    for(JSONObject message: messages) {
+      SolrInputDocument document = new SolrInputDocument();
+      document.addField("id", getIdValue(message));
+      document.addField("sensorType", sourceType);
+      for(Object key: message.keySet()) {
+        Object value = message.get(key);
+        document.addField(getFieldName(key, value), value);
+      }
+      UpdateResponse response = solr.add(document);
+    }
+    if (shouldCommit) {
+      solr.commit(getCollection(configurations));
+    }
+  }
+
+  protected String getCollection(Configurations configurations) {
+    String collection = (String) configurations.getGlobalConfig().get("solr.collection");
+    return collection != null ? collection : DEFAULT_COLLECTION;
+  }
+
+  private int getIdValue(JSONObject message) {
+    return message.toJSONString().hashCode();
+  }
+
+  protected String getFieldName(Object key, Object value) {
+    String field;
+    if (value instanceof Integer) {
+      field = key + "_i";
+    } else if (value instanceof Long) {
+      field = key + "_l";
+    } else if (value instanceof Float) {
+      field = key + "_f";
+    } else if (value instanceof Double) {
+      field = key + "_d";
+    } else {
+      field = key + "_s";
+    }
+    return field;
+  }
+
+  @Override
+  public void close() throws Exception {
+    solr.close();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/e59b1a31/metron-streaming/Metron-Solr/src/main/resources/Metron_Configs/etc/env/solr.properties
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-Solr/src/main/resources/Metron_Configs/etc/env/solr.properties b/metron-streaming/Metron-Solr/src/main/resources/Metron_Configs/etc/env/solr.properties
new file mode 100644
index 0000000..df25506
--- /dev/null
+++ b/metron-streaming/Metron-Solr/src/main/resources/Metron_Configs/etc/env/solr.properties
@@ -0,0 +1,109 @@
+#  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
+spout.kafka.topic.asa=asa
+spout.kafka.topic.bro=bro
+spout.kafka.topic.fireeye=fireeye
+spout.kafka.topic.ise=ise
+spout.kafka.topic.lancope=lancope
+spout.kafka.topic.paloalto=paloalto
+spout.kafka.topic.pcap=pcap
+spout.kafka.topic.snort=snort
+spout.kafka.topic.yaf=yaf
+
+##### Indexing #####
+writer.class.name=org.apache.metron.writer.solr.SolrWriter
+
+##### ElasticSearch #####
+
+es.ip=10.22.0.214
+es.port=9300
+es.clustername=elasticsearch
+
+##### MySQL #####
+
+mysql.ip=10.22.0.214
+mysql.port=3306
+mysql.username=root
+mysql.password=hadoop123
+
+##### Metrics #####
+
+#reporters
+org.apache.metron.metrics.reporter.graphite=true
+org.apache.metron.metrics.reporter.console=false
+org.apache.metron.metrics.reporter.jmx=false
+
+#Graphite Addresses
+
+org.apache.metron.metrics.graphite.address=localhost
+org.apache.metron.metrics.graphite.port=2023
+
+#TelemetryParserBolt
+org.apache.metron.metrics.TelemetryParserBolt.acks=true
+org.apache.metron.metrics.TelemetryParserBolt.emits=true
+org.apache.metron.metrics.TelemetryParserBolt.fails=true
+
+
+#GenericEnrichmentBolt
+org.apache.metron.metrics.GenericEnrichmentBolt.acks=true
+org.apache.metron.metrics.GenericEnrichmentBolt.emits=true
+org.apache.metron.metrics.GenericEnrichmentBolt.fails=true
+
+
+#TelemetryIndexingBolt
+org.apache.metron.metrics.TelemetryIndexingBolt.acks=true
+org.apache.metron.metrics.TelemetryIndexingBolt.emits=true
+org.apache.metron.metrics.TelemetryIndexingBolt.fails=true
+
+##### Host Enrichment #####
+
+org.apache.metron.enrichment.host.known_hosts=[{"ip":"10.1.128.236", "local":"YES", "type":"webserver", "asset_value" : "important"},\
+{"ip":"10.1.128.237", "local":"UNKNOWN", "type":"unknown", "asset_value" : "important"},\
+{"ip":"10.60.10.254", "local":"YES", "type":"printer", "asset_value" : "important"}]
+
+##### HDFS #####
+
+bolt.hdfs.batch.size=5000
+bolt.hdfs.field.delimiter=|
+bolt.hdfs.file.rotation.size.in.mb=5
+bolt.hdfs.file.system.url=hdfs://iot01.cloud.hortonworks.com:8020
+bolt.hdfs.wip.file.path=/paloalto/wip
+bolt.hdfs.finished.file.path=/paloalto/rotated
+bolt.hdfs.compression.codec.class=org.apache.hadoop.io.compress.SnappyCodec
+index.hdfs.output=/tmp/metron/enriched
+
+##### HBase #####
+bolt.hbase.table.name=pcap
+bolt.hbase.table.fields=t:value
+bolt.hbase.table.key.tuple.field.name=key
+bolt.hbase.table.timestamp.tuple.field.name=timestamp
+bolt.hbase.enable.batching=false
+bolt.hbase.write.buffer.size.in.bytes=2000000
+bolt.hbase.durability=SKIP_WAL
+bolt.hbase.partitioner.region.info.refresh.interval.mins=60
+
+##### Threat Intel #####
+
+threat.intel.tracker.table=
+threat.intel.tracker.cf=
+threat.intel.ip.table=
+threat.intel.ip.cf=

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/e59b1a31/metron-streaming/Metron-Solr/src/test/java/org/apache/metron/integration/SolrEnrichmentIntegrationTest.java
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-Solr/src/test/java/org/apache/metron/integration/SolrEnrichmentIntegrationTest.java b/metron-streaming/Metron-Solr/src/test/java/org/apache/metron/integration/SolrEnrichmentIntegrationTest.java
new file mode 100644
index 0000000..afeb56b
--- /dev/null
+++ b/metron-streaming/Metron-Solr/src/test/java/org/apache/metron/integration/SolrEnrichmentIntegrationTest.java
@@ -0,0 +1,107 @@
+/**
+ * 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;
+
+import com.google.common.base.Function;
+import org.apache.metron.domain.Configurations;
+import org.apache.metron.integration.util.integration.ComponentRunner;
+import org.apache.metron.integration.util.integration.InMemoryComponent;
+import org.apache.metron.integration.util.integration.Processor;
+import org.apache.metron.integration.util.integration.ReadinessState;
+import org.apache.metron.integration.util.integration.components.SolrComponent;
+import org.apache.metron.util.SampleUtil;
+import org.apache.metron.utils.ConfigurationsUtils;
+import org.apache.metron.utils.JSONUtils;
+
+import javax.annotation.Nullable;
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+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 {
+    SolrComponent solrComponent = new SolrComponent.Builder()
+            .addCollection(collection, "../Metron-Solr/src/test/resources/solr/conf")
+            .withPostStartCallback(new Function<SolrComponent, Void>() {
+              @Nullable
+              @Override
+              public Void apply(@Nullable SolrComponent solrComponent) {
+                topologyProperties.setProperty("solr.zk", solrComponent.getZookeeperUrl());
+                try {
+                  String testZookeeperUrl = topologyProperties.getProperty("kafka.zk");
+                  Configurations configurations = SampleUtil.getSampleConfigs();
+                  Map<String, Object> globalConfig = configurations.getGlobalConfig();
+                  globalConfig.put("solr.zookeeper", solrComponent.getZookeeperUrl());
+                  ConfigurationsUtils.writerGlobalConfigToZookeeper(JSONUtils.INSTANCE.toJSON(globalConfig), testZookeeperUrl);
+                } catch (Exception e) {
+                  e.printStackTrace();
+                }
+                return null;
+              }
+            })
+            .build();
+    return solrComponent;
+  }
+
+  @Override
+  Processor<List<Map<String, Object>>> getProcessor(final List<byte[]> inputMessages) {
+    return new Processor<List<Map<String, Object>>>() {
+      List<Map<String, Object>> docs = null;
+      public ReadinessState process(ComponentRunner runner) {
+        SolrComponent solrComponent = runner.getComponent("search", SolrComponent.class);
+        if (solrComponent.hasCollection(collection)) {
+          List<Map<String, Object>> docsFromDisk;
+          try {
+            docs = solrComponent.getAllIndexedDocs(collection);
+            docsFromDisk = EnrichmentIntegrationTest.readDocsFromDisk(hdfsDir);
+            System.out.println(docs.size() + " vs " + inputMessages.size() + " vs " + docsFromDisk.size());
+          } catch (IOException e) {
+            throw new IllegalStateException("Unable to retrieve indexed documents.", e);
+          }
+          if (docs.size() < inputMessages.size() || docs.size() != docsFromDisk.size()) {
+            return ReadinessState.NOT_READY;
+          } else {
+            return ReadinessState.READY;
+          }
+        } else {
+          return ReadinessState.NOT_READY;
+        }
+      }
+
+      public List<Map<String, Object>> getResult() {
+        return docs;
+      }
+    };
+  }
+
+  @Override
+  void setAdditionalProperties(Properties topologyProperties) {
+    topologyProperties.setProperty("writer.class.name", "org.apache.metron.writer.solr.SolrWriter");
+  }
+
+  @Override
+  public String cleanField(String field) {
+    return field.replaceFirst("_[dfils]$", "");
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/e59b1a31/metron-streaming/Metron-Solr/src/test/java/org/apache/metron/integration/util/integration/components/SolrComponent.java
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-Solr/src/test/java/org/apache/metron/integration/util/integration/components/SolrComponent.java b/metron-streaming/Metron-Solr/src/test/java/org/apache/metron/integration/util/integration/components/SolrComponent.java
new file mode 100644
index 0000000..f2b9748
--- /dev/null
+++ b/metron-streaming/Metron-Solr/src/test/java/org/apache/metron/integration/util/integration/components/SolrComponent.java
@@ -0,0 +1,153 @@
+/**
+ * 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.util.integration.components;
+
+import com.google.common.base.Function;
+import org.apache.metron.integration.util.integration.InMemoryComponent;
+import org.apache.metron.integration.util.integration.UnableToStartException;
+import org.apache.metron.writer.solr.MetronSolrClient;
+import org.apache.solr.client.solrj.SolrQuery;
+import org.apache.solr.client.solrj.SolrServerException;
+import org.apache.solr.client.solrj.embedded.JettyConfig;
+import org.apache.solr.client.solrj.impl.CloudSolrClient;
+import org.apache.solr.client.solrj.response.QueryResponse;
+import org.apache.solr.cloud.MiniSolrCloudCluster;
+import org.apache.solr.common.SolrDocument;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+public class SolrComponent implements InMemoryComponent {
+
+  public static class Builder {
+    private int port = 8983;
+    private String solrXmlPath = "../Metron-Solr/src/test/resources/solr/solr.xml";
+    private Map<String, String> collections = new HashMap<>();
+    private Function<SolrComponent, Void> postStartCallback;
+
+    public Builder withPort(int port) {
+      this.port = port;
+      return this;
+    }
+
+    public Builder withSolrXmlPath(String solrXmlPath) {
+      this.solrXmlPath = solrXmlPath;
+      return this;
+    }
+
+    public Builder addCollection(String name, String configPath) {
+      collections.put(name, configPath);
+      return this;
+    }
+
+    public Builder withPostStartCallback(Function<SolrComponent, Void> f) {
+      postStartCallback = f;
+      return this;
+    }
+
+    public SolrComponent build() throws Exception {
+      if (collections.isEmpty()) throw new Exception("Must add at least 1 collection");
+      return new SolrComponent(port, solrXmlPath, collections, postStartCallback);
+    }
+  }
+
+  private int port;
+  private String solrXmlPath;
+  private Map<String, String> collections;
+  private MiniSolrCloudCluster miniSolrCloudCluster;
+  private Function<SolrComponent, Void> postStartCallback;
+
+  private SolrComponent(int port, String solrXmlPath, Map<String, String> collections, Function<SolrComponent, Void> postStartCallback) throws Exception {
+    this.port = port;
+    this.solrXmlPath = solrXmlPath;
+    this.collections = collections;
+    this.postStartCallback = postStartCallback;
+  }
+
+  @Override
+  public void start() throws UnableToStartException {
+    try {
+      File baseDir = Files.createTempDirectory("solrcomponent").toFile();
+      baseDir.deleteOnExit();
+      miniSolrCloudCluster = new MiniSolrCloudCluster(1, baseDir, new File(solrXmlPath), JettyConfig.builder().setPort(port).build());
+      for(String name: collections.keySet()) {
+        String configPath = collections.get(name);
+        miniSolrCloudCluster.uploadConfigDir(new File(configPath), name);
+      }
+      miniSolrCloudCluster.createCollection("metron", 1, 1, "metron", new HashMap<String, String>());
+      if (postStartCallback != null) postStartCallback.apply(this);
+    } catch(Exception e) {
+      throw new UnableToStartException(e.getMessage(), e);
+    }
+  }
+
+  @Override
+  public void stop() {
+    try {
+      miniSolrCloudCluster.shutdown();
+    } catch (Exception e) {
+      e.printStackTrace();
+    }
+  }
+
+  public MetronSolrClient getSolrClient() {
+    return new MetronSolrClient(getZookeeperUrl());
+  }
+
+  public MiniSolrCloudCluster getMiniSolrCloudCluster() {
+    return this.miniSolrCloudCluster;
+  }
+
+  public String getZookeeperUrl() {
+    return miniSolrCloudCluster.getZkServer().getZkAddress();
+  }
+
+  public boolean hasCollection(String collection) {
+    MetronSolrClient solr = getSolrClient();
+    boolean collectionFound = false;
+    try {
+      collectionFound = solr.listCollections().contains(collection);
+    } catch(Exception e) {
+      e.printStackTrace();
+    }
+    return collectionFound;
+  }
+
+  public List<Map<String, Object>> getAllIndexedDocs(String collection) {
+    List<Map<String, Object>> docs = new ArrayList<>();
+    CloudSolrClient solr = miniSolrCloudCluster.getSolrClient();
+    solr.setDefaultCollection(collection);
+    SolrQuery parameters = new SolrQuery();
+    parameters.set("q", "*:*");
+    try {
+      solr.commit();
+      QueryResponse response = solr.query(parameters);
+      for (SolrDocument solrDocument : response.getResults()) {
+        docs.add(solrDocument);
+      }
+    } catch (SolrServerException | IOException e) {
+      e.printStackTrace();
+    }
+    return docs;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/e59b1a31/metron-streaming/Metron-Solr/src/test/java/org/apache/metron/writer/solr/MetronSolrClientTest.java
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-Solr/src/test/java/org/apache/metron/writer/solr/MetronSolrClientTest.java b/metron-streaming/Metron-Solr/src/test/java/org/apache/metron/writer/solr/MetronSolrClientTest.java
new file mode 100644
index 0000000..7bd3ac6
--- /dev/null
+++ b/metron-streaming/Metron-Solr/src/test/java/org/apache/metron/writer/solr/MetronSolrClientTest.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
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * 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.writer.solr;
+
+import org.apache.solr.client.solrj.request.QueryRequest;
+import org.apache.solr.common.params.CollectionParams;
+import org.apache.solr.common.util.NamedList;
+import org.hamcrest.Description;
+import org.junit.Test;
+import org.mockito.ArgumentMatcher;
+import org.mockito.Mockito;
+
+import java.util.ArrayList;
+
+import static org.mockito.Matchers.argThat;
+import static org.mockito.Matchers.isNull;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+
+public class MetronSolrClientTest {
+
+  class CollectionRequestMatcher extends ArgumentMatcher<QueryRequest> {
+
+    private String name;
+
+    public CollectionRequestMatcher(String name) {
+      this.name = name;
+    }
+
+    @Override
+    public boolean matches(Object o) {
+      QueryRequest queryRequest = (QueryRequest) o;
+      return name.equals(queryRequest.getParams().get("action"));
+    }
+
+    @Override
+    public void describeTo(Description description) {
+      description.appendText(name);
+    }
+  }
+
+  @Test
+  public void testClient() throws Exception {
+
+    final String collection = "metron";
+    String zookeeperUrl = "zookeeperUrl";
+    MetronSolrClient metronSolrClient = Mockito.spy(new MetronSolrClient(zookeeperUrl));
+
+    Mockito.doReturn(new NamedList<Object>() {{
+      add("collections", new ArrayList<String>() {{
+        add(collection);
+      }});
+    }}).when(metronSolrClient).request(argThat(new CollectionRequestMatcher(CollectionParams.CollectionAction.LIST.name())), (String) isNull());
+    metronSolrClient.createCollection(collection, 1, 1);
+    verify(metronSolrClient, times(1)).request(argThat(new CollectionRequestMatcher(CollectionParams.CollectionAction.LIST.name())), (String) isNull());
+    verify(metronSolrClient, times(0)).request(argThat(new CollectionRequestMatcher(CollectionParams.CollectionAction.CREATE.name())), (String) isNull());
+
+    metronSolrClient = Mockito.spy(new MetronSolrClient(zookeeperUrl));
+    Mockito.doReturn(new NamedList<Object>() {{
+      add("collections", new ArrayList<String>());
+    }}).when(metronSolrClient).request(argThat(new CollectionRequestMatcher(CollectionParams.CollectionAction.LIST.name())), (String) isNull());
+    Mockito.doReturn(new NamedList<>()).when(metronSolrClient).request(argThat(new CollectionRequestMatcher(CollectionParams.CollectionAction.CREATE.name())), (String) isNull());
+    metronSolrClient.createCollection(collection, 1, 1);
+    verify(metronSolrClient, times(1)).request(argThat(new CollectionRequestMatcher(CollectionParams.CollectionAction.LIST.name())), (String) isNull());
+    verify(metronSolrClient, times(1)).request(argThat(new CollectionRequestMatcher(CollectionParams.CollectionAction.CREATE.name())), (String) isNull());
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/e59b1a31/metron-streaming/Metron-Solr/src/test/java/org/apache/metron/writer/solr/SolrWriterTest.java
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-Solr/src/test/java/org/apache/metron/writer/solr/SolrWriterTest.java b/metron-streaming/Metron-Solr/src/test/java/org/apache/metron/writer/solr/SolrWriterTest.java
new file mode 100644
index 0000000..7c720ea
--- /dev/null
+++ b/metron-streaming/Metron-Solr/src/test/java/org/apache/metron/writer/solr/SolrWriterTest.java
@@ -0,0 +1,139 @@
+/**
+ * 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
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * 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.writer.solr;
+
+import backtype.storm.tuple.Tuple;
+import org.apache.metron.domain.Configurations;
+import org.apache.metron.util.SampleUtil;
+import org.apache.solr.client.solrj.request.QueryRequest;
+import org.apache.solr.common.SolrInputDocument;
+import org.hamcrest.Description;
+import org.json.simple.JSONObject;
+import org.junit.Test;
+import org.mockito.ArgumentMatcher;
+import org.mockito.Mockito;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+import static org.mockito.Mockito.argThat;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+
+
+public class SolrWriterTest {
+
+  class CollectionRequestMatcher extends ArgumentMatcher<QueryRequest> {
+
+    private String name;
+
+    public CollectionRequestMatcher(String name) {
+      this.name = name;
+    }
+
+    @Override
+    public boolean matches(Object o) {
+      QueryRequest queryRequest = (QueryRequest) o;
+      return name.equals(queryRequest.getParams().get("action"));
+    }
+
+    @Override
+    public void describeTo(Description description) {
+      description.appendText(name);
+    }
+  }
+
+  class SolrInputDocumentMatcher extends ArgumentMatcher<SolrInputDocument> {
+
+    private int expectedId;
+    private String expectedSourceType;
+    private int expectedInt;
+    private double expectedDouble;
+
+    public SolrInputDocumentMatcher(int expectedId, String expectedSourceType, int expectedInt, double expectedDouble) {
+      this.expectedId = expectedId;
+      this.expectedSourceType = expectedSourceType;
+      this.expectedInt = expectedInt;
+      this.expectedDouble = expectedDouble;
+    }
+
+    @Override
+    public boolean matches(Object o) {
+      SolrInputDocument solrInputDocument = (SolrInputDocument) o;
+      int actualId = (Integer) solrInputDocument.get("id").getValue();
+      String actualName = (String) solrInputDocument.get("sensorType").getValue();
+      int actualInt = (Integer) solrInputDocument.get("intField_i").getValue();
+      double actualDouble = (Double) solrInputDocument.get("doubleField_d").getValue();
+      return expectedId == actualId && expectedSourceType.equals(actualName) && expectedInt == actualInt && expectedDouble == actualDouble;
+    }
+
+    @Override
+    public void describeTo(Description description) {
+      description.appendText(String.format("fields: [id=%d, doubleField_d=%f, name=%s, intField_i=%d]", expectedId, expectedDouble, expectedSourceType, expectedInt));
+    }
+
+  }
+
+  @Test
+  public void testWriter() throws Exception {
+    Configurations configurations = SampleUtil.getSampleConfigs();
+    JSONObject message1 = new JSONObject();
+    message1.put("intField", 100);
+    message1.put("doubleField", 100.0);
+    JSONObject message2 = new JSONObject();
+    message2.put("intField", 200);
+    message2.put("doubleField", 200.0);
+    List<JSONObject> messages = new ArrayList<>();
+    messages.add(message1);
+    messages.add(message2);
+
+    String collection = "metron";
+    MetronSolrClient solr = Mockito.mock(MetronSolrClient.class);
+    SolrWriter writer = new SolrWriter().withMetronSolrClient(solr);
+    writer.init(null, configurations);
+    verify(solr, times(1)).createCollection(collection, 1, 1);
+    verify(solr, times(1)).setDefaultCollection(collection);
+
+    collection = "metron2";
+    int numShards = 4;
+    int replicationFactor = 2;
+    Map<String, Object> globalConfig = configurations.getGlobalConfig();
+    globalConfig.put("solr.collection", collection);
+    globalConfig.put("solr.numShards", numShards);
+    globalConfig.put("solr.replicationFactor", replicationFactor);
+    configurations.updateGlobalConfig(globalConfig);
+    writer = new SolrWriter().withMetronSolrClient(solr);
+    writer.init(null, configurations);
+    verify(solr, times(1)).createCollection(collection, numShards, replicationFactor);
+    verify(solr, times(1)).setDefaultCollection(collection);
+
+    writer.write("test", configurations, new ArrayList<Tuple>(), messages);
+    verify(solr, times(1)).add(argThat(new SolrInputDocumentMatcher(message1.toJSONString().hashCode(), "test", 100, 100.0)));
+    verify(solr, times(1)).add(argThat(new SolrInputDocumentMatcher(message2.toJSONString().hashCode(), "test", 200, 200.0)));
+    verify(solr, times(0)).commit(collection);
+
+    writer = new SolrWriter().withMetronSolrClient(solr).withShouldCommit(true);
+    writer.init(null, configurations);
+    writer.write("test", configurations, new ArrayList<Tuple>(), messages);
+    verify(solr, times(2)).add(argThat(new SolrInputDocumentMatcher(message1.toJSONString().hashCode(), "test", 100, 100.0)));
+    verify(solr, times(2)).add(argThat(new SolrInputDocumentMatcher(message2.toJSONString().hashCode(), "test", 200, 200.0)));
+    verify(solr, times(1)).commit(collection);
+
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/e59b1a31/metron-streaming/Metron-Solr/src/test/resources/solr/conf/_rest_managed.json
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-Solr/src/test/resources/solr/conf/_rest_managed.json b/metron-streaming/Metron-Solr/src/test/resources/solr/conf/_rest_managed.json
new file mode 100644
index 0000000..6a4aec3
--- /dev/null
+++ b/metron-streaming/Metron-Solr/src/test/resources/solr/conf/_rest_managed.json
@@ -0,0 +1 @@
+{"initArgs":{},"managedList":[]}


[04/15] incubator-metron git commit: METRON 86: Adding Solr indexing support (merrimanr via cestella) closes apache/incubator-metron#67

Posted by ce...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/e59b1a31/metron-streaming/Metron-Topologies/src/main/resources/SampleInput/FireeyeExampleOutput
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-Topologies/src/main/resources/SampleInput/FireeyeExampleOutput b/metron-streaming/Metron-Topologies/src/main/resources/SampleInput/FireeyeExampleOutput
deleted file mode 100644
index 0210010..0000000
--- a/metron-streaming/Metron-Topologies/src/main/resources/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



[10/15] incubator-metron git commit: METRON 86: Adding Solr indexing support (merrimanr via cestella) closes apache/incubator-metron#67

Posted by ce...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/e59b1a31/metron-streaming/Metron-Testing/src/main/resources/sample/data/SampleInput/BroExampleOutput
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-Testing/src/main/resources/sample/data/SampleInput/BroExampleOutput b/metron-streaming/Metron-Testing/src/main/resources/sample/data/SampleInput/BroExampleOutput
new file mode 100644
index 0000000..6ded8a8
--- /dev/null
+++ b/metron-streaming/Metron-Testing/src/main/resources/sample/data/SampleInput/BroExampleOutput
@@ -0,0 +1,23411 @@
+{"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":1,"method":"GET","host":"www.cisco.

<TRUNCATED>


[14/15] incubator-metron git commit: METRON 86: Adding Solr indexing support (merrimanr via cestella) closes apache/incubator-metron#67

Posted by ce...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/e59b1a31/metron-streaming/Metron-Common/src/main/java/org/apache/metron/bolt/ConfiguredBolt.java
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-Common/src/main/java/org/apache/metron/bolt/ConfiguredBolt.java b/metron-streaming/Metron-Common/src/main/java/org/apache/metron/bolt/ConfiguredBolt.java
index 30c8e23..a832ebb 100644
--- a/metron-streaming/Metron-Common/src/main/java/org/apache/metron/bolt/ConfiguredBolt.java
+++ b/metron-streaming/Metron-Common/src/main/java/org/apache/metron/bolt/ConfiguredBolt.java
@@ -23,17 +23,16 @@ import backtype.storm.topology.base.BaseRichBolt;
 import org.apache.curator.RetryPolicy;
 import org.apache.curator.framework.CuratorFramework;
 import org.apache.curator.framework.CuratorFrameworkFactory;
-import org.apache.curator.framework.recipes.cache.PathChildrenCache;
-import org.apache.curator.framework.recipes.cache.PathChildrenCacheEvent;
-import org.apache.curator.framework.recipes.cache.PathChildrenCacheListener;
+import org.apache.curator.framework.recipes.cache.TreeCache;
+import org.apache.curator.framework.recipes.cache.TreeCacheEvent;
+import org.apache.curator.framework.recipes.cache.TreeCacheListener;
 import org.apache.curator.retry.ExponentialBackoffRetry;
 import org.apache.log4j.Logger;
 import org.apache.metron.Constants;
-import org.apache.metron.domain.SourceConfig;
+import org.apache.metron.domain.Configurations;
+import org.apache.metron.utils.ConfigurationsUtils;
 
 import java.io.IOException;
-import java.util.Collections;
-import java.util.HashMap;
 import java.util.Map;
 
 public abstract class ConfiguredBolt extends BaseRichBolt {
@@ -41,51 +40,62 @@ public abstract class ConfiguredBolt extends BaseRichBolt {
   private static final Logger LOG = Logger.getLogger(ConfiguredBolt.class);
 
   private String zookeeperUrl;
+  private long timeout = Constants.DEFAULT_CONFIGURED_BOLT_TIMEOUT;
 
-  protected Map<String, SourceConfig> configurations = Collections.synchronizedMap(new HashMap<String, SourceConfig>());
+  protected final Configurations configurations = new Configurations();
   private CuratorFramework client;
-  private PathChildrenCache cache;
+  private TreeCache cache;
 
   public ConfiguredBolt(String zookeeperUrl) {
     this.zookeeperUrl = zookeeperUrl;
   }
 
+  public ConfiguredBolt withTimeout(long timeout) {
+    this.timeout = timeout;
+    return this;
+  }
+
   @Override
   public void prepare(Map stormConf, TopologyContext context, OutputCollector collector) {
     RetryPolicy retryPolicy = new ExponentialBackoffRetry(1000, 3);
     client = CuratorFrameworkFactory.newClient(zookeeperUrl, retryPolicy);
     client.start();
-    cache = new PathChildrenCache(client, Constants.ZOOKEEPER_TOPOLOGY_ROOT, true);
-    PathChildrenCacheListener listener = new PathChildrenCacheListener() {
-      @Override
-      public void childEvent(CuratorFramework client, PathChildrenCacheEvent event) throws Exception {
-        if (event.getType().equals(PathChildrenCacheEvent.Type.CHILD_ADDED) || event.getType().equals(PathChildrenCacheEvent.Type.CHILD_UPDATED)) {
-          byte[] data = event.getData().getData();
-          if (data != null) {
-            SourceConfig temp = SourceConfig.load(data);
-            if (temp != null) {
-              String[] path = event.getData().getPath().split("/");
-              configurations.put(path[path.length - 1], temp);
-            }
+    try {
+      ConfigurationsUtils.updateConfigsFromZookeeper(configurations, client);
+      cache = new TreeCache(client, Constants.ZOOKEEPER_TOPOLOGY_ROOT);
+      TreeCacheListener listener = new TreeCacheListener() {
+        @Override
+        public void childEvent(CuratorFramework client, TreeCacheEvent event) throws Exception {
+          if (event.getType().equals(TreeCacheEvent.Type.NODE_ADDED) || event.getType().equals(TreeCacheEvent.Type.NODE_UPDATED)) {
+            String path = event.getData().getPath();
+            byte[] data = event.getData().getData();
+            updateConfig(path, data);
           }
         }
-      }
-    };
-    cache.getListenable().addListener(listener);
-    try {
+      };
+      cache.getListenable().addListener(listener);
       cache.start();
     } catch (Exception e) {
       throw new RuntimeException(e);
     }
   }
 
+  public void updateConfig(String path, byte[] data) throws IOException {
+    if (data.length != 0 && path != null) {
+      String name = path.substring(path.lastIndexOf("/") + 1);
+      if (path.startsWith(Constants.ZOOKEEPER_SENSOR_ROOT)) {
+        configurations.updateSensorEnrichmentConfig(name, data);
+      } else if (Constants.ZOOKEEPER_GLOBAL_ROOT.equals(path)) {
+        configurations.updateGlobalConfig(data);
+      } else {
+        configurations.updateConfig(name, data);
+      }
+    }
+  }
+
   @Override
   public void cleanup() {
-    try {
-      cache.close();
-      client.close();
-    } catch (IOException e) {
-      LOG.error(e.getMessage(), e);
-    }
+    cache.close();
+    client.close();
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/e59b1a31/metron-streaming/Metron-Common/src/main/java/org/apache/metron/domain/Configurations.java
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-Common/src/main/java/org/apache/metron/domain/Configurations.java b/metron-streaming/Metron-Common/src/main/java/org/apache/metron/domain/Configurations.java
new file mode 100644
index 0000000..d93cc5f
--- /dev/null
+++ b/metron-streaming/Metron-Common/src/main/java/org/apache/metron/domain/Configurations.java
@@ -0,0 +1,92 @@
+/**
+ * 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.domain;
+
+import com.fasterxml.jackson.core.type.TypeReference;
+import org.apache.log4j.Logger;
+import org.apache.metron.utils.JSONUtils;
+
+import java.io.ByteArrayInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.Serializable;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+
+public class Configurations implements Serializable {
+
+  private static final Logger LOG = Logger.getLogger(Configurations.class);
+
+  public static final String GLOBAL_CONFIG_NAME = "global";
+
+  private ConcurrentMap<String, Object> configurations = new ConcurrentHashMap<>();
+
+  @SuppressWarnings("unchecked")
+  public Map<String, Object> getGlobalConfig() {
+    return (Map<String, Object>) configurations.get(GLOBAL_CONFIG_NAME);
+  }
+
+  public void updateGlobalConfig(byte[] data) throws IOException {
+    updateGlobalConfig(new ByteArrayInputStream(data));
+  }
+
+  public void updateGlobalConfig(InputStream io) throws IOException {
+    Map<String, Object> globalConfig = JSONUtils.INSTANCE.load(io, new TypeReference<Map<String, Object>>() {});
+    updateGlobalConfig(globalConfig);
+  }
+
+  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) {
+    try {
+      Map<String, Object> config = JSONUtils.INSTANCE.load(new ByteArrayInputStream(data), new TypeReference<Map<String, Object>>(){});
+      updateConfig(name, config);
+    } catch (IOException e) {
+      throw new IllegalStateException(e);
+    }
+  }
+
+  public void updateConfig(String name, Map<String, Object> config) {
+    configurations.put(name, config);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/e59b1a31/metron-streaming/Metron-Common/src/main/java/org/apache/metron/domain/SensorEnrichmentConfig.java
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-Common/src/main/java/org/apache/metron/domain/SensorEnrichmentConfig.java b/metron-streaming/Metron-Common/src/main/java/org/apache/metron/domain/SensorEnrichmentConfig.java
new file mode 100644
index 0000000..b24e8a8
--- /dev/null
+++ b/metron-streaming/Metron-Common/src/main/java/org/apache/metron/domain/SensorEnrichmentConfig.java
@@ -0,0 +1,62 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.metron.domain;
+
+import java.util.List;
+import java.util.Map;
+
+public class SensorEnrichmentConfig {
+
+  private String index;
+  private Map<String, List<String>> enrichmentFieldMap;
+  private Map<String, List<String>> threatIntelFieldMap;
+  private int batchSize;
+
+  public String getIndex() {
+    return index;
+  }
+
+  public void setIndex(String index) {
+    this.index = index;
+  }
+
+  public Map<String, List<String>> getEnrichmentFieldMap() {
+    return enrichmentFieldMap;
+  }
+
+  public void setEnrichmentFieldMap(Map<String, List<String>> enrichmentFieldMap) {
+    this.enrichmentFieldMap = enrichmentFieldMap;
+  }
+
+  public Map<String, List<String>> getThreatIntelFieldMap() {
+    return threatIntelFieldMap;
+  }
+
+  public void setThreatIntelFieldMap(Map<String, List<String>> threatIntelFieldMap) {
+    this.threatIntelFieldMap = threatIntelFieldMap;
+  }
+
+  public int getBatchSize() {
+    return batchSize;
+  }
+
+  public void setBatchSize(int batchSize) {
+    this.batchSize = batchSize;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/e59b1a31/metron-streaming/Metron-Common/src/main/java/org/apache/metron/domain/SourceConfig.java
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-Common/src/main/java/org/apache/metron/domain/SourceConfig.java b/metron-streaming/Metron-Common/src/main/java/org/apache/metron/domain/SourceConfig.java
deleted file mode 100644
index 8e1a960..0000000
--- a/metron-streaming/Metron-Common/src/main/java/org/apache/metron/domain/SourceConfig.java
+++ /dev/null
@@ -1,88 +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.domain;
-
-import org.codehaus.jackson.map.ObjectMapper;
-import org.yaml.snakeyaml.TypeDescription;
-import org.yaml.snakeyaml.Yaml;
-import org.yaml.snakeyaml.constructor.Constructor;
-
-import java.io.ByteArrayInputStream;
-import java.io.IOException;
-import java.io.InputStream;
-import java.nio.charset.Charset;
-import java.util.List;
-import java.util.Map;
-
-public class SourceConfig {
-
-  final static ObjectMapper _mapper = new ObjectMapper();
-
-  private String index;
-  private Map<String, List<String>> enrichmentFieldMap;
-  private Map<String, List<String>> threatIntelFieldMap;
-  private int batchSize;
-
-  public String getIndex() {
-    return index;
-  }
-
-  public void setIndex(String index) {
-    this.index = index;
-  }
-
-  public Map<String, List<String>> getEnrichmentFieldMap() {
-    return enrichmentFieldMap;
-  }
-
-  public void setEnrichmentFieldMap(Map<String, List<String>> enrichmentFieldMap) {
-    this.enrichmentFieldMap = enrichmentFieldMap;
-  }
-
-  public Map<String, List<String>> getThreatIntelFieldMap() {
-    return threatIntelFieldMap;
-  }
-
-  public void setThreatIntelFieldMap(Map<String, List<String>> threatIntelFieldMap) {
-    this.threatIntelFieldMap = threatIntelFieldMap;
-  }
-
-  public int getBatchSize() {
-    return batchSize;
-  }
-
-  public void setBatchSize(int batchSize) {
-    this.batchSize = batchSize;
-  }
-
-  public static synchronized SourceConfig load(InputStream is) throws IOException {
-    SourceConfig ret = _mapper.readValue(is, SourceConfig.class);
-    return ret;
-  }
-
-  public static synchronized SourceConfig load(byte[] data) throws IOException {
-    return load( new ByteArrayInputStream(data));
-  }
-
-  public static synchronized SourceConfig load(String s, Charset c) throws IOException {
-    return load( s.getBytes(c));
-  }
-  public static synchronized SourceConfig load(String s) throws IOException {
-    return load( s, Charset.defaultCharset());
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/e59b1a31/metron-streaming/Metron-Common/src/main/java/org/apache/metron/helpers/topology/ErrorUtils.java
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-Common/src/main/java/org/apache/metron/helpers/topology/ErrorUtils.java b/metron-streaming/Metron-Common/src/main/java/org/apache/metron/helpers/topology/ErrorUtils.java
index b02cbaf..a1b7ccc 100644
--- a/metron-streaming/Metron-Common/src/main/java/org/apache/metron/helpers/topology/ErrorUtils.java
+++ b/metron-streaming/Metron-Common/src/main/java/org/apache/metron/helpers/topology/ErrorUtils.java
@@ -49,7 +49,7 @@ public class ErrorUtils {
 		}
 		
 		error_message.put("message", message);
-		error_message.put(Constants.SOURCE_TYPE, "error");
+		error_message.put(Constants.SENSOR_TYPE, "error");
 		error_message.put("exception", exception);
 		error_message.put("stack", stackTrace);
 		

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/e59b1a31/metron-streaming/Metron-Common/src/main/java/org/apache/metron/pcap/PcapUtils.java
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-Common/src/main/java/org/apache/metron/pcap/PcapUtils.java b/metron-streaming/Metron-Common/src/main/java/org/apache/metron/pcap/PcapUtils.java
index a046801..10ab03d 100644
--- a/metron-streaming/Metron-Common/src/main/java/org/apache/metron/pcap/PcapUtils.java
+++ b/metron-streaming/Metron-Common/src/main/java/org/apache/metron/pcap/PcapUtils.java
@@ -17,10 +17,13 @@
  */
 package org.apache.metron.pcap;
 
+import java.util.ArrayList;
 import java.util.HashMap;
+import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
 
+import com.google.common.base.Joiner;
 import org.apache.commons.lang.StringUtils;
 
 import com.google.common.collect.BiMap;
@@ -210,6 +213,15 @@ public class PcapUtils {
 
   }
 
+  public static String convertHexToIpv4Ip(String hex) {
+    List<Integer> ipSegments = new ArrayList<>();
+    for(int i = 0; i < hex.length(); i += 2) {
+      String segment = hex.substring(i, i + 2);
+      ipSegments.add(Integer.parseInt(segment, 16));
+    }
+    return Joiner.on(".").join(ipSegments);
+  }
+
   /**
    * Gets the session key.
    * 

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/e59b1a31/metron-streaming/Metron-Common/src/main/java/org/apache/metron/topology/TopologyUtils.java
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-Common/src/main/java/org/apache/metron/topology/TopologyUtils.java b/metron-streaming/Metron-Common/src/main/java/org/apache/metron/topology/TopologyUtils.java
index 581d74f..78371d8 100644
--- a/metron-streaming/Metron-Common/src/main/java/org/apache/metron/topology/TopologyUtils.java
+++ b/metron-streaming/Metron-Common/src/main/java/org/apache/metron/topology/TopologyUtils.java
@@ -22,7 +22,7 @@ import org.json.simple.JSONObject;
 
 public class TopologyUtils {
 
-  public static String getSourceType(JSONObject message) {
-    return (String) message.get(Constants.SOURCE_TYPE);
+  public static String getSensorType(JSONObject message) {
+    return (String) message.get(Constants.SENSOR_TYPE);
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/e59b1a31/metron-streaming/Metron-Common/src/main/java/org/apache/metron/utils/ConfigurationsUtils.java
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-Common/src/main/java/org/apache/metron/utils/ConfigurationsUtils.java b/metron-streaming/Metron-Common/src/main/java/org/apache/metron/utils/ConfigurationsUtils.java
new file mode 100644
index 0000000..20026b2
--- /dev/null
+++ b/metron-streaming/Metron-Common/src/main/java/org/apache/metron/utils/ConfigurationsUtils.java
@@ -0,0 +1,179 @@
+/**
+ * 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.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.commons.io.FilenameUtils;
+import org.apache.curator.RetryPolicy;
+import org.apache.curator.framework.CuratorFramework;
+import org.apache.curator.framework.CuratorFrameworkFactory;
+import org.apache.curator.retry.ExponentialBackoffRetry;
+import org.apache.metron.Constants;
+import org.apache.metron.domain.Configurations;
+import org.apache.zookeeper.KeeperException;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.Paths;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+public class ConfigurationsUtils {
+
+  public static CuratorFramework getClient(String zookeeperUrl) {
+    RetryPolicy retryPolicy = new ExponentialBackoffRetry(1000, 3);
+    return CuratorFrameworkFactory.newClient(zookeeperUrl, retryPolicy);
+  }
+
+  public static void writeToZookeeperFromFile(String path, String filePath, String zookeeperUrl) throws Exception {
+    writeToZookeeper(path, Files.readAllBytes(Paths.get(filePath)), zookeeperUrl);
+  }
+
+  public static void writerGlobalConfigToZookeeper(byte[] configData, String zookeeperUrl) throws Exception {
+    writeToZookeeper(Constants.ZOOKEEPER_GLOBAL_ROOT, configData, zookeeperUrl);
+  }
+
+  public static void writeSensorEnrichmentConfigToZookeeper(String sensorType, byte[] configData, String zookeeperUrl) throws Exception {
+    writeToZookeeper(Constants.ZOOKEEPER_SENSOR_ROOT + "/" + sensorType, configData, zookeeperUrl);
+  }
+
+  public static void writeToZookeeper(String path, byte[] configData, String zookeeperUrl) throws Exception {
+    CuratorFramework client = getClient(zookeeperUrl);
+    client.start();
+    try {
+      client.setData().forPath(path, configData);
+    } catch (KeeperException.NoNodeException e) {
+      client.create().creatingParentsIfNeeded().forPath(path, configData);
+    }
+    client.close();
+  }
+
+  public static void updateConfigsFromZookeeper(Configurations configurations, CuratorFramework client) throws Exception {
+    configurations.updateGlobalConfig(readGlobalConfigBytesFromZookeeper(client));
+    List<String> sensorTypes = client.getChildren().forPath(Constants.ZOOKEEPER_SENSOR_ROOT);
+    for(String sensorType: sensorTypes) {
+      configurations.updateSensorEnrichmentConfig(sensorType, readSensorEnrichmentConfigBytesFromZookeeper(sensorType, client));
+    }
+  }
+
+  public static byte[] readGlobalConfigBytesFromZookeeper(CuratorFramework client) throws Exception {
+    return readConfigBytesFromZookeeper(Constants.ZOOKEEPER_GLOBAL_ROOT, client);
+  }
+
+  public static byte[] readSensorEnrichmentConfigBytesFromZookeeper(String sensorType, CuratorFramework client) throws Exception {
+    return readConfigBytesFromZookeeper(Constants.ZOOKEEPER_SENSOR_ROOT + "/" + sensorType, client);
+  }
+
+  public static byte[] readConfigBytesFromZookeeper(String path, CuratorFramework client) throws Exception {
+    return client.getData().forPath(path);
+  }
+
+  public static void uploadConfigsToZookeeper(String rootFilePath, String zookeeperUrl) throws Exception {
+    ConfigurationsUtils.writerGlobalConfigToZookeeper(readGlobalConfigFromFile(rootFilePath), zookeeperUrl);
+    Map<String, byte[]> sensorEnrichmentConfigs = readSensorEnrichmentConfigsFromFile(rootFilePath);
+    for(String sensorType: sensorEnrichmentConfigs.keySet()) {
+      ConfigurationsUtils.writeSensorEnrichmentConfigToZookeeper(sensorType, sensorEnrichmentConfigs.get(sensorType), zookeeperUrl);
+    }
+  }
+
+  public static byte[] readGlobalConfigFromFile(String rootFilePath) throws IOException {
+    return Files.readAllBytes(Paths.get(rootFilePath, Constants.GLOBAL_CONFIG_NAME + ".json"));
+  }
+
+  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()) {
+      sensorEnrichmentConfigs.put(FilenameUtils.removeExtension(file.getName()), Files.readAllBytes(file.toPath()));
+    }
+    return sensorEnrichmentConfigs;
+  }
+
+  public static void dumpConfigs(String zookeeperUrl) throws Exception {
+    CuratorFramework client = getClient(zookeeperUrl);
+    client.start();
+    List<String> children = client.getChildren().forPath(Constants.ZOOKEEPER_TOPOLOGY_ROOT);
+    for (String child : children) {
+      byte[] data = client.getData().forPath(Constants.ZOOKEEPER_TOPOLOGY_ROOT + "/" + child);
+      System.out.println("Config for source " + child);
+      System.out.println(new String(data));
+      System.out.println();
+    }
+    client.close();
+  }
+
+  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("p", "config_files", true, "Path to the source config files.  Must be named like \"$source\"-config.json");
+      o.setArgName("DIR_NAME");
+      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);
+    }
+
+    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("ConfigurationsUtils", null, options, null, true);
+        System.exit(-1);
+      }
+      if (cmd.hasOption("h")) {
+        final HelpFormatter usageFormatter = new HelpFormatter();
+        usageFormatter.printHelp("ConfigurationsUtils", null, options, null, true);
+        System.exit(0);
+      }
+
+      String zkQuorum = cmd.getOptionValue("z");
+      if (cmd.hasOption("p")) {
+        String sourcePath = cmd.getOptionValue("p");
+        uploadConfigsToZookeeper(sourcePath, zkQuorum);
+      }
+
+      ConfigurationsUtils.dumpConfigs(zkQuorum);
+
+    } catch (Exception e) {
+      e.printStackTrace();
+      System.exit(-1);
+    }
+
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/e59b1a31/metron-streaming/Metron-Common/src/main/java/org/apache/metron/utils/JSONUtils.java
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-Common/src/main/java/org/apache/metron/utils/JSONUtils.java b/metron-streaming/Metron-Common/src/main/java/org/apache/metron/utils/JSONUtils.java
index 93b0a58..cffcd68 100644
--- a/metron-streaming/Metron-Common/src/main/java/org/apache/metron/utils/JSONUtils.java
+++ b/metron-streaming/Metron-Common/src/main/java/org/apache/metron/utils/JSONUtils.java
@@ -75,4 +75,8 @@ public enum JSONUtils {
       return _mapper.get().writeValueAsString(o);
     }
   }
+
+  public byte[] toJSON(Object config) throws JsonProcessingException {
+    return _mapper.get().writeValueAsBytes(config);
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/e59b1a31/metron-streaming/Metron-Common/src/main/java/org/apache/metron/writer/HBaseWriter.java
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-Common/src/main/java/org/apache/metron/writer/HBaseWriter.java b/metron-streaming/Metron-Common/src/main/java/org/apache/metron/writer/HBaseWriter.java
index b257b24..291b849 100644
--- a/metron-streaming/Metron-Common/src/main/java/org/apache/metron/writer/HBaseWriter.java
+++ b/metron-streaming/Metron-Common/src/main/java/org/apache/metron/writer/HBaseWriter.java
@@ -23,7 +23,7 @@ import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.client.HTableInterface;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.metron.domain.SourceConfig;
+import org.apache.metron.domain.Configurations;
 import org.apache.metron.hbase.HTableProvider;
 import org.apache.metron.hbase.TableProvider;
 import org.apache.metron.utils.ConfigUtils;
@@ -62,7 +62,7 @@ public abstract class HBaseWriter implements MessageWriter<JSONObject>, Serializ
   }
 
   @Override
-  public void write(String sourceType, SourceConfig configuration, Tuple tuple, JSONObject message) throws Exception {
+  public void write(String sourceType, Configurations configurations, Tuple tuple, JSONObject message) throws Exception {
     Put put = new Put(getKey(tuple, message));
     Map<String, byte[]> values = getValues(tuple, message);
     for(String column: values.keySet()) {

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/e59b1a31/metron-streaming/Metron-Common/src/main/java/org/apache/metron/writer/interfaces/BulkMessageWriter.java
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-Common/src/main/java/org/apache/metron/writer/interfaces/BulkMessageWriter.java b/metron-streaming/Metron-Common/src/main/java/org/apache/metron/writer/interfaces/BulkMessageWriter.java
index 9b627e6..c3a930c 100644
--- a/metron-streaming/Metron-Common/src/main/java/org/apache/metron/writer/interfaces/BulkMessageWriter.java
+++ b/metron-streaming/Metron-Common/src/main/java/org/apache/metron/writer/interfaces/BulkMessageWriter.java
@@ -18,14 +18,14 @@
 package org.apache.metron.writer.interfaces;
 
 import backtype.storm.tuple.Tuple;
-import org.apache.metron.domain.SourceConfig;
+import org.apache.metron.domain.Configurations;
 
 import java.util.List;
 import java.util.Map;
 
 public interface BulkMessageWriter<T> extends AutoCloseable {
 
-  void init(Map stormConf);
-  void write(String sourceType, SourceConfig configuration, List<Tuple> tuples, List<T> messages) throws Exception;
+  void init(Map stormConf, Configurations configuration) throws Exception;
+  void write(String sensorType, Configurations configurations, List<Tuple> tuples, List<T> messages) throws Exception;
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/e59b1a31/metron-streaming/Metron-Common/src/main/java/org/apache/metron/writer/interfaces/MessageWriter.java
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-Common/src/main/java/org/apache/metron/writer/interfaces/MessageWriter.java b/metron-streaming/Metron-Common/src/main/java/org/apache/metron/writer/interfaces/MessageWriter.java
index 12de836..25c8a5a 100644
--- a/metron-streaming/Metron-Common/src/main/java/org/apache/metron/writer/interfaces/MessageWriter.java
+++ b/metron-streaming/Metron-Common/src/main/java/org/apache/metron/writer/interfaces/MessageWriter.java
@@ -18,10 +18,10 @@
 package org.apache.metron.writer.interfaces;
 
 import backtype.storm.tuple.Tuple;
-import org.apache.metron.domain.SourceConfig;
+import org.apache.metron.domain.Configurations;
 
 public interface MessageWriter<T> extends AutoCloseable {
 
   void init();
-  void write(String sourceType, SourceConfig configuration, Tuple tuple, T message) throws Exception;
+  void write(String sensorType, Configurations configurations, Tuple tuple, T message) throws Exception;
 }

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/e59b1a31/metron-streaming/Metron-Common/src/test/java/org/apache/metron/pcap/PcapUtilsTest.java
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-Common/src/test/java/org/apache/metron/pcap/PcapUtilsTest.java b/metron-streaming/Metron-Common/src/test/java/org/apache/metron/pcap/PcapUtilsTest.java
new file mode 100644
index 0000000..fa2385c
--- /dev/null
+++ b/metron-streaming/Metron-Common/src/test/java/org/apache/metron/pcap/PcapUtilsTest.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.pcap;
+
+import junit.framework.Assert;
+import org.junit.Test;
+
+public class PcapUtilsTest {
+
+  @Test
+  public void testConvertHexToIpv4Ip() {
+    String hex = "c0a88a9e";
+    String ipAddress = PcapUtils.convertHexToIpv4Ip(hex);
+    Assert.assertEquals("192.168.138.158", ipAddress);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/e59b1a31/metron-streaming/Metron-Common/src/test/resources/config/source/bro-config.json
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-Common/src/test/resources/config/source/bro-config.json b/metron-streaming/Metron-Common/src/test/resources/config/source/bro-config.json
deleted file mode 100644
index 34109b8..0000000
--- a/metron-streaming/Metron-Common/src/test/resources/config/source/bro-config.json
+++ /dev/null
@@ -1,14 +0,0 @@
-{
-  "index": "bro",
-  "batchSize": 5,
-  "enrichmentFieldMap":
-  {
-    "geo": ["ip_dst_addr", "ip_src_addr"],
-    "host": ["host"]
-  },
-  "threatIntelFieldMap":
-  {
-    "ip": ["ip_dst_addr", "ip_src_addr"]
-  }
-}
-

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/e59b1a31/metron-streaming/Metron-Common/src/test/resources/config/source/pcap-config.json
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-Common/src/test/resources/config/source/pcap-config.json b/metron-streaming/Metron-Common/src/test/resources/config/source/pcap-config.json
deleted file mode 100644
index 82c7c5e..0000000
--- a/metron-streaming/Metron-Common/src/test/resources/config/source/pcap-config.json
+++ /dev/null
@@ -1,13 +0,0 @@
-{
-  "index": "pcap",
-  "batchSize": 5,
-  "enrichmentFieldMap":
-  {
-    "geo": ["ip_src_addr", "ip_dst_addr"],
-    "host": ["ip_src_addr", "ip_dst_addr"]
-  },
-  "threatIntelFieldMap":
-  {
-    "ip": ["ip_src_addr", "ip_dst_addr"]
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/e59b1a31/metron-streaming/Metron-Common/src/test/resources/config/source/snort-config.json
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-Common/src/test/resources/config/source/snort-config.json b/metron-streaming/Metron-Common/src/test/resources/config/source/snort-config.json
deleted file mode 100644
index 1208637..0000000
--- a/metron-streaming/Metron-Common/src/test/resources/config/source/snort-config.json
+++ /dev/null
@@ -1,14 +0,0 @@
-{
-  "index": "snort",
-  "batchSize": 1,
-  "enrichmentFieldMap":
-  {
-    "geo": ["ip_dst_addr", "ip_src_addr"],
-    "host": ["host"]
-  },
-  "threatIntelFieldMap":
-  {
-    "ip": ["ip_dst_addr", "ip_src_addr"]
-  }
-}
-

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/e59b1a31/metron-streaming/Metron-Common/src/test/resources/config/source/yaf-config.json
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-Common/src/test/resources/config/source/yaf-config.json b/metron-streaming/Metron-Common/src/test/resources/config/source/yaf-config.json
deleted file mode 100644
index 65de961..0000000
--- a/metron-streaming/Metron-Common/src/test/resources/config/source/yaf-config.json
+++ /dev/null
@@ -1,14 +0,0 @@
-{
-  "index": "yaf",
-  "batchSize": 5,
-  "enrichmentFieldMap":
-  {
-    "geo": ["ip_dst_addr", "ip_src_addr"],
-    "host": ["host"]
-  },
-  "threatIntelFieldMap":
-  {
-    "ip": ["ip_dst_addr", "ip_src_addr"]
-  }
-}
-

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/e59b1a31/metron-streaming/Metron-Elasticsearch/pom.xml
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-Elasticsearch/pom.xml b/metron-streaming/Metron-Elasticsearch/pom.xml
new file mode 100644
index 0000000..ab9242a
--- /dev/null
+++ b/metron-streaming/Metron-Elasticsearch/pom.xml
@@ -0,0 +1,202 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  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.
+  -->
+
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <modelVersion>4.0.0</modelVersion>
+    <parent>
+        <groupId>org.apache.metron</groupId>
+        <artifactId>Metron-Streaming</artifactId>
+        <version>0.1BETA</version>
+    </parent>
+    <artifactId>Metron-Elasticsearch</artifactId>
+    <properties>
+        <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
+        <project.reporting.outputEncoding>UTF-8</project.reporting.outputEncoding>
+    </properties>
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.metron</groupId>
+            <artifactId>Metron-Common</artifactId>
+            <version>${project.parent.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>org.elasticsearch</groupId>
+            <artifactId>elasticsearch</artifactId>
+            <version>${global_elasticsearch_version}</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.storm</groupId>
+            <artifactId>storm-core</artifactId>
+            <version>${global_storm_version}</version>
+            <scope>provided</scope>
+            <exclusions>
+                <exclusion>
+                    <artifactId>servlet-api</artifactId>
+                    <groupId>javax.servlet</groupId>
+                </exclusion>
+                <exclusion>
+                    <artifactId>log4j-over-slf4j</artifactId>
+                    <groupId>org.slf4j</groupId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.metron</groupId>
+            <artifactId>Metron-Testing</artifactId>
+            <version>${project.parent.version}</version>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.metron</groupId>
+            <artifactId>Metron-Topologies</artifactId>
+            <version>${project.parent.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>org.mockito</groupId>
+            <artifactId>mockito-all</artifactId>
+            <version>${global_mockito_version}</version>
+            <scope>test</scope>
+        </dependency>
+
+    </dependencies>
+    <reporting>
+        <plugins>
+            <!-- Normally, dependency report takes time, skip it -->
+            <plugin>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-project-info-reports-plugin</artifactId>
+                <version>2.7</version>
+
+                <configuration>
+                    <dependencyLocationsEnabled>false</dependencyLocationsEnabled>
+                </configuration>
+            </plugin>
+
+            <plugin>
+                <groupId>org.codehaus.mojo</groupId>
+                <artifactId>emma-maven-plugin</artifactId>
+                <version>1.0-alpha-3</version>
+                <inherited>true</inherited>
+            </plugin>
+        </plugins>
+    </reporting>
+
+    <build>
+        <plugins>
+            <plugin>
+                <!-- Separates the unit tests from the integration tests. -->
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-surefire-plugin</artifactId>
+                <version>2.12.4</version>
+                <configuration>
+                    <!-- Skip the default running of this plug-in (or everything is run twice...see below) -->
+                    <argLine>-Xmx2048m -XX:MaxPermSize=256m</argLine>
+                    <skip>true</skip>
+                    <!-- Show 100% of the lines from the stack trace (doesn't work) -->
+                    <trimStackTrace>false</trimStackTrace>
+                </configuration>
+                <executions>
+                    <execution>
+                        <id>unit-tests</id>
+                        <phase>test</phase>
+                        <goals>
+                            <goal>test</goal>
+                        </goals>
+                        <configuration>
+                            <!-- Never skip running the tests when the test phase is invoked -->
+                            <skip>false</skip>
+                            <includes>
+                                <!-- Include unit tests within integration-test phase. -->
+                                <include>**/*Test.java</include>
+                            </includes>
+                            <excludes>
+                                <!-- Exclude integration tests within (unit) test phase. -->
+                                <exclude>**/*IntegrationTest.java</exclude>
+                            </excludes>
+                        </configuration>
+                    </execution>
+                    <execution>
+                        <id>integration-tests</id>
+                        <phase>integration-test</phase>
+                        <goals>
+                            <goal>test</goal>
+                        </goals>
+                        <configuration>
+                            <!-- Never skip running the tests when the integration-test phase is invoked -->
+                            <skip>false</skip>
+                            <includes>
+                                <!-- Include integration tests within integration-test phase. -->
+                                <include>**/*IntegrationTest.java</include>
+                            </includes>
+                        </configuration>
+                    </execution>
+                </executions>
+            </plugin>
+            <plugin>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-shade-plugin</artifactId>
+                <version>2.3</version>
+                <configuration>
+                    <createDependencyReducedPom>true</createDependencyReducedPom>
+                </configuration>
+                <executions>
+                    <execution>
+                        <phase>package</phase>
+                        <goals>
+                            <goal>shade</goal>
+                        </goals>
+                        <configuration>
+                            <artifactSet>
+                                <excludes>
+                                    <exclude>storm:storm-core:*</exclude>
+                                    <exclude>storm:storm-lib:*</exclude>
+                                    <exclude>org.slf4j.impl*</exclude>
+                                    <exclude>org.slf4j:slf4j-log4j*</exclude>
+                                </excludes>
+                            </artifactSet>
+                            <transformers>
+                                <transformer
+                                        implementation="org.apache.maven.plugins.shade.resource.DontIncludeResourceTransformer">
+                                    <resource>.yaml</resource>
+                                </transformer>
+                                <transformer
+                                        implementation="org.apache.maven.plugins.shade.resource.ServicesResourceTransformer"/>
+                                <transformer
+                                        implementation="org.apache.maven.plugins.shade.resource.ManifestResourceTransformer">
+                                    <mainClass></mainClass>
+                                </transformer>
+                            </transformers>
+                        </configuration>
+                    </execution>
+                </executions>
+            </plugin>
+            <plugin>
+                <artifactId>maven-assembly-plugin</artifactId>
+                <configuration>
+                    <descriptor>src/main/assembly/assembly.xml</descriptor>
+                </configuration>
+                <executions>
+                    <execution>
+                        <id>make-assembly</id> <!-- this is used for inheritance merges -->
+                        <phase>package</phase> <!-- bind to the packaging phase -->
+                        <goals>
+                            <goal>single</goal>
+                        </goals>
+                    </execution>
+                </executions>
+            </plugin>
+        </plugins>
+    </build>
+</project>

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/e59b1a31/metron-streaming/Metron-Elasticsearch/src/main/assembly/assembly.xml
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-Elasticsearch/src/main/assembly/assembly.xml b/metron-streaming/Metron-Elasticsearch/src/main/assembly/assembly.xml
new file mode 100644
index 0000000..35cbcc3
--- /dev/null
+++ b/metron-streaming/Metron-Elasticsearch/src/main/assembly/assembly.xml
@@ -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.
+  -->
+
+<assembly>
+  <id>archive</id>
+  <formats>
+    <format>tar.gz</format>
+  </formats>
+  <includeBaseDirectory>false</includeBaseDirectory>
+  <fileSets>
+    <fileSet>
+      <directory>${project.basedir}/src/main/resources/Metron_Configs/etc</directory>
+      <outputDirectory>/config/etc</outputDirectory>
+      <useDefaultExcludes>true</useDefaultExcludes>
+      <excludes>
+        <exclude>**/*.formatted</exclude>
+        <exclude>**/*.filtered</exclude>
+      </excludes>
+      <fileMode>0644</fileMode>
+      <lineEnding>unix</lineEnding>
+    </fileSet>
+    <fileSet>
+      <directory>${project.basedir}/target</directory>
+      <includes>
+        <include>${project.artifactId}-${project.version}.jar</include>
+      </includes>
+      <outputDirectory>/lib</outputDirectory>
+      <useDefaultExcludes>true</useDefaultExcludes>
+    </fileSet>
+  </fileSets>
+</assembly>

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/e59b1a31/metron-streaming/Metron-Elasticsearch/src/main/java/org/apache/metron/writer/ElasticsearchWriter.java
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-Elasticsearch/src/main/java/org/apache/metron/writer/ElasticsearchWriter.java b/metron-streaming/Metron-Elasticsearch/src/main/java/org/apache/metron/writer/ElasticsearchWriter.java
new file mode 100644
index 0000000..45631f2
--- /dev/null
+++ b/metron-streaming/Metron-Elasticsearch/src/main/java/org/apache/metron/writer/ElasticsearchWriter.java
@@ -0,0 +1,94 @@
+/**
+ * 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.writer;
+
+import backtype.storm.tuple.Tuple;
+import org.apache.metron.domain.Configurations;
+import org.apache.metron.domain.SensorEnrichmentConfig;
+import org.apache.metron.writer.interfaces.BulkMessageWriter;
+import org.elasticsearch.action.bulk.BulkRequestBuilder;
+import org.elasticsearch.action.bulk.BulkResponse;
+import org.elasticsearch.action.index.IndexRequestBuilder;
+import org.elasticsearch.client.transport.TransportClient;
+import org.elasticsearch.common.settings.ImmutableSettings;
+import org.elasticsearch.common.transport.InetSocketTransportAddress;
+import org.json.simple.JSONObject;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Serializable;
+import java.text.SimpleDateFormat;
+import java.util.Date;
+import java.util.List;
+import java.util.Map;
+
+public class ElasticsearchWriter implements BulkMessageWriter<JSONObject>, Serializable {
+
+  private Map<String, String> optionalSettings;
+  private transient TransportClient client;
+  private SimpleDateFormat dateFormat;
+  private static final Logger LOG = LoggerFactory
+          .getLogger(ElasticsearchWriter.class);
+
+  public ElasticsearchWriter withOptionalSettings(Map<String, String> optionalSettings) {
+    this.optionalSettings = optionalSettings;
+    return this;
+  }
+
+  @Override
+  public void init(Map stormConf, Configurations configurations) {
+    Map<String, Object> globalConfiguration = configurations.getGlobalConfig();
+    ImmutableSettings.Builder builder = ImmutableSettings.settingsBuilder();
+    builder.put("cluster.name", globalConfiguration.get("es.clustername"));
+    builder.put("client.transport.ping_timeout","500s");
+    if (optionalSettings != null) {
+      builder.put(optionalSettings);
+    }
+    client = new TransportClient(builder.build())
+            .addTransportAddress(new InetSocketTransportAddress((String) globalConfiguration.get("es.ip"), (Integer) globalConfiguration.get("es.port")));
+    dateFormat = new SimpleDateFormat((String) globalConfiguration.get("es.date.format"));
+
+  }
+
+  @Override
+  public void write(String sensorType, Configurations configurations, List<Tuple> tuples, List<JSONObject> messages) throws Exception {
+    SensorEnrichmentConfig sensorEnrichmentConfig = configurations.getSensorEnrichmentConfig(sensorType);
+    String indexPostfix = dateFormat.format(new Date());
+    BulkRequestBuilder bulkRequest = client.prepareBulk();
+    for(JSONObject message: messages) {
+      String indexName = sensorType;
+      if (sensorEnrichmentConfig != null) {
+        indexName = sensorEnrichmentConfig.getIndex();
+      }
+      IndexRequestBuilder indexRequestBuilder = client.prepareIndex(indexName + "_index_" + indexPostfix,
+              sensorType);
+
+      indexRequestBuilder.setSource(message.toJSONString());
+      bulkRequest.add(indexRequestBuilder);
+    }
+    BulkResponse resp = bulkRequest.execute().actionGet();
+    if (resp.hasFailures()) {
+      throw new Exception(resp.buildFailureMessage());
+    }
+  }
+
+  @Override
+  public void close() throws Exception {
+    client.close();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/e59b1a31/metron-streaming/Metron-Elasticsearch/src/main/resources/Metron_Configs/etc/env/elasticsearch.properties
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-Elasticsearch/src/main/resources/Metron_Configs/etc/env/elasticsearch.properties b/metron-streaming/Metron-Elasticsearch/src/main/resources/Metron_Configs/etc/env/elasticsearch.properties
new file mode 100644
index 0000000..1381b49
--- /dev/null
+++ b/metron-streaming/Metron-Elasticsearch/src/main/resources/Metron_Configs/etc/env/elasticsearch.properties
@@ -0,0 +1,109 @@
+#  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
+spout.kafka.topic.asa=asa
+spout.kafka.topic.bro=bro
+spout.kafka.topic.fireeye=fireeye
+spout.kafka.topic.ise=ise
+spout.kafka.topic.lancope=lancope
+spout.kafka.topic.paloalto=paloalto
+spout.kafka.topic.pcap=pcap
+spout.kafka.topic.snort=snort
+spout.kafka.topic.yaf=yaf
+
+##### Indexing #####
+writer.class.name=org.apache.metron.writer.ElasticsearchWriter
+
+##### ElasticSearch #####
+
+es.ip=10.22.0.214
+es.port=9300
+es.clustername=elasticsearch
+
+##### MySQL #####
+
+mysql.ip=10.22.0.214
+mysql.port=3306
+mysql.username=root
+mysql.password=hadoop123
+
+##### Metrics #####
+
+#reporters
+org.apache.metron.metrics.reporter.graphite=true
+org.apache.metron.metrics.reporter.console=false
+org.apache.metron.metrics.reporter.jmx=false
+
+#Graphite Addresses
+
+org.apache.metron.metrics.graphite.address=localhost
+org.apache.metron.metrics.graphite.port=2023
+
+#TelemetryParserBolt
+org.apache.metron.metrics.TelemetryParserBolt.acks=true
+org.apache.metron.metrics.TelemetryParserBolt.emits=true
+org.apache.metron.metrics.TelemetryParserBolt.fails=true
+
+
+#GenericEnrichmentBolt
+org.apache.metron.metrics.GenericEnrichmentBolt.acks=true
+org.apache.metron.metrics.GenericEnrichmentBolt.emits=true
+org.apache.metron.metrics.GenericEnrichmentBolt.fails=true
+
+
+#TelemetryIndexingBolt
+org.apache.metron.metrics.TelemetryIndexingBolt.acks=true
+org.apache.metron.metrics.TelemetryIndexingBolt.emits=true
+org.apache.metron.metrics.TelemetryIndexingBolt.fails=true
+
+##### Host Enrichment #####
+
+org.apache.metron.enrichment.host.known_hosts=[{"ip":"10.1.128.236", "local":"YES", "type":"webserver", "asset_value" : "important"},\
+{"ip":"10.1.128.237", "local":"UNKNOWN", "type":"unknown", "asset_value" : "important"},\
+{"ip":"10.60.10.254", "local":"YES", "type":"printer", "asset_value" : "important"}]
+
+##### HDFS #####
+
+bolt.hdfs.batch.size=5000
+bolt.hdfs.field.delimiter=|
+bolt.hdfs.file.rotation.size.in.mb=5
+bolt.hdfs.file.system.url=hdfs://iot01.cloud.hortonworks.com:8020
+bolt.hdfs.wip.file.path=/paloalto/wip
+bolt.hdfs.finished.file.path=/paloalto/rotated
+bolt.hdfs.compression.codec.class=org.apache.hadoop.io.compress.SnappyCodec
+index.hdfs.output=/tmp/metron/enriched
+
+##### HBase #####
+bolt.hbase.table.name=pcap
+bolt.hbase.table.fields=t:value
+bolt.hbase.table.key.tuple.field.name=key
+bolt.hbase.table.timestamp.tuple.field.name=timestamp
+bolt.hbase.enable.batching=false
+bolt.hbase.write.buffer.size.in.bytes=2000000
+bolt.hbase.durability=SKIP_WAL
+bolt.hbase.partitioner.region.info.refresh.interval.mins=60
+
+##### Threat Intel #####
+
+threat.intel.tracker.table=
+threat.intel.tracker.cf=
+threat.intel.ip.table=
+threat.intel.ip.cf=

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/e59b1a31/metron-streaming/Metron-Elasticsearch/src/test/java/org/apache/metron/integration/ElasticsearchEnrichmentIntegrationTest.java
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-Elasticsearch/src/test/java/org/apache/metron/integration/ElasticsearchEnrichmentIntegrationTest.java b/metron-streaming/Metron-Elasticsearch/src/test/java/org/apache/metron/integration/ElasticsearchEnrichmentIntegrationTest.java
new file mode 100644
index 0000000..2765c25
--- /dev/null
+++ b/metron-streaming/Metron-Elasticsearch/src/test/java/org/apache/metron/integration/ElasticsearchEnrichmentIntegrationTest.java
@@ -0,0 +1,88 @@
+/**
+ * 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;
+
+import org.apache.metron.integration.util.integration.ComponentRunner;
+import org.apache.metron.integration.util.integration.InMemoryComponent;
+import org.apache.metron.integration.util.integration.Processor;
+import org.apache.metron.integration.util.integration.ReadinessState;
+import org.apache.metron.integration.util.integration.components.ElasticSearchComponent;
+
+import java.io.File;
+import java.io.IOException;
+import java.text.SimpleDateFormat;
+import java.util.Date;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+
+public class ElasticsearchEnrichmentIntegrationTest extends EnrichmentIntegrationTest {
+
+  private String indexDir = "target/elasticsearch";
+  private String dateFormat = "yyyy.MM.dd.hh";
+  private String index = "yaf_index_" + new SimpleDateFormat(dateFormat).format(new Date());
+
+  @Override
+  InMemoryComponent getSearchComponent(final Properties topologyProperties) {
+    return new ElasticSearchComponent.Builder()
+            .withHttpPort(9211)
+            .withIndexDir(new File(indexDir))
+            .build();
+  }
+
+  @Override
+  Processor<List<Map<String, Object>>> getProcessor(final List<byte[]> inputMessages) {
+    return new Processor<List<Map<String, Object>>>() {
+      List<Map<String, Object>> docs = null;
+      public ReadinessState process(ComponentRunner runner) {
+        ElasticSearchComponent elasticSearchComponent = runner.getComponent("search", ElasticSearchComponent.class);
+        if (elasticSearchComponent.hasIndex(index)) {
+          List<Map<String, Object>> docsFromDisk;
+          try {
+            docs = elasticSearchComponent.getAllIndexedDocs(index, "yaf");
+            docsFromDisk = readDocsFromDisk(hdfsDir);
+            System.out.println(docs.size() + " vs " + inputMessages.size() + " vs " + docsFromDisk.size());
+          } catch (IOException e) {
+            throw new IllegalStateException("Unable to retrieve indexed documents.", e);
+          }
+          if (docs.size() < inputMessages.size() || docs.size() != docsFromDisk.size()) {
+            return ReadinessState.NOT_READY;
+          } else {
+            return ReadinessState.READY;
+          }
+        } else {
+          return ReadinessState.NOT_READY;
+        }
+      }
+
+      public List<Map<String, Object>> getResult() {
+        return docs;
+      }
+    };
+  }
+
+  @Override
+  void setAdditionalProperties(Properties topologyProperties) {
+    topologyProperties.setProperty("writer.class.name", "org.apache.metron.writer.ElasticsearchWriter");
+  }
+
+  @Override
+  public String cleanField(String field) {
+    return field;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/e59b1a31/metron-streaming/Metron-Elasticsearch/src/test/java/org/apache/metron/integration/util/integration/components/ElasticSearchComponent.java
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-Elasticsearch/src/test/java/org/apache/metron/integration/util/integration/components/ElasticSearchComponent.java b/metron-streaming/Metron-Elasticsearch/src/test/java/org/apache/metron/integration/util/integration/components/ElasticSearchComponent.java
new file mode 100644
index 0000000..671c4f5
--- /dev/null
+++ b/metron-streaming/Metron-Elasticsearch/src/test/java/org/apache/metron/integration/util/integration/components/ElasticSearchComponent.java
@@ -0,0 +1,186 @@
+/**
+ * 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.util.integration.components;
+
+import org.apache.commons.io.FileUtils;
+import org.apache.metron.integration.util.integration.InMemoryComponent;
+import org.apache.metron.integration.util.integration.UnableToStartException;
+import org.elasticsearch.ElasticsearchTimeoutException;
+import org.elasticsearch.action.admin.cluster.health.ClusterHealthAction;
+import org.elasticsearch.action.admin.cluster.health.ClusterHealthRequest;
+import org.elasticsearch.action.admin.cluster.health.ClusterHealthResponse;
+import org.elasticsearch.action.admin.cluster.health.ClusterHealthStatus;
+import org.elasticsearch.action.admin.indices.refresh.RefreshRequest;
+import org.elasticsearch.action.admin.indices.stats.IndicesStatsRequest;
+import org.elasticsearch.action.search.SearchResponse;
+import org.elasticsearch.client.Client;
+import org.elasticsearch.client.ElasticsearchClient;
+import org.elasticsearch.client.transport.TransportClient;
+import org.elasticsearch.common.settings.ImmutableSettings;
+import org.elasticsearch.common.settings.Settings;
+import org.elasticsearch.common.transport.InetSocketTransportAddress;
+import org.elasticsearch.common.unit.TimeValue;
+import org.elasticsearch.node.Node;
+import org.elasticsearch.node.NodeBuilder;
+import org.elasticsearch.search.SearchHit;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+public class ElasticSearchComponent implements InMemoryComponent {
+
+    public static class Builder{
+        private int httpPort;
+        private File indexDir;
+        private Map<String, String> extraElasticSearchSettings = null;
+        public Builder withHttpPort(int httpPort) {
+            this.httpPort = httpPort;
+            return this;
+        }
+        public Builder withIndexDir(File indexDir) {
+            this.indexDir = indexDir;
+            return this;
+        }
+        public Builder withExtraElasticSearchSettings(Map<String, String> extraElasticSearchSettings) {
+            this.extraElasticSearchSettings = extraElasticSearchSettings;
+            return this;
+        }
+        public ElasticSearchComponent build() {
+            return new ElasticSearchComponent(httpPort, indexDir, extraElasticSearchSettings);
+        }
+    }
+
+    private Client client;
+    private Node node;
+    private int httpPort;
+    private File indexDir;
+    private Map<String, String> extraElasticSearchSettings;
+
+    public ElasticSearchComponent(int httpPort, File indexDir) {
+        this(httpPort, indexDir, null);
+    }
+    public ElasticSearchComponent(int httpPort, File indexDir, Map<String, String> extraElasticSearchSettings) {
+        this.httpPort = httpPort;
+        this.indexDir = indexDir;
+        this.extraElasticSearchSettings = extraElasticSearchSettings;
+    }
+    public Client getClient() {
+        return client;
+    }
+
+    private void cleanDir(File dir) throws IOException {
+        if(dir.exists()) {
+            FileUtils.deleteDirectory(dir);
+        }
+        dir.mkdirs();
+    }
+    public void start() throws UnableToStartException {
+        File logDir= new File(indexDir, "/logs");
+        File dataDir= new File(indexDir, "/data");
+        try {
+            cleanDir(logDir);
+            cleanDir(dataDir);
+
+        } catch (IOException e) {
+            throw new UnableToStartException("Unable to clean log or data directories", e);
+        }
+        ImmutableSettings.Builder immutableSettings = ImmutableSettings.settingsBuilder()
+                .put("node.http.enabled", true)
+                .put("http.port", httpPort)
+                .put("cluster.name", "metron")
+                .put("path.logs",logDir.getAbsolutePath())
+                .put("path.data",dataDir.getAbsolutePath())
+                .put("gateway.type", "none")
+                .put("index.store.type", "memory")
+                .put("index.number_of_shards", 1)
+                .put("node.mode", "network")
+                .put("index.number_of_replicas", 1);
+        if(extraElasticSearchSettings != null) {
+            immutableSettings = immutableSettings.put(extraElasticSearchSettings);
+        }
+        Settings settings = immutableSettings.build();
+        node = NodeBuilder.nodeBuilder().settings(settings).node();
+        node.start();
+        settings = ImmutableSettings.settingsBuilder()
+					.put("cluster.name", "metron").build();
+		client = new TransportClient(settings)
+					.addTransportAddress(new InetSocketTransportAddress("localhost",
+							9300));
+
+        waitForCluster(client, ClusterHealthStatus.YELLOW, new TimeValue(60000));
+    }
+
+    public static void waitForCluster(ElasticsearchClient client, ClusterHealthStatus status, TimeValue timeout) throws UnableToStartException {
+        try {
+            ClusterHealthResponse healthResponse =
+                    (ClusterHealthResponse)client.execute(ClusterHealthAction.INSTANCE, new ClusterHealthRequest().waitForStatus(status).timeout(timeout)).actionGet();
+            if (healthResponse != null && healthResponse.isTimedOut()) {
+                throw new UnableToStartException("cluster state is " + healthResponse.getStatus().name()
+                        + " and not " + status.name()
+                        + ", from here on, everything will fail!");
+            }
+        } catch (ElasticsearchTimeoutException e) {
+            throw new UnableToStartException("timeout, cluster does not respond to health request, cowardly refusing to continue with operations");
+        }
+    }
+
+    public List<Map<String, Object>> getAllIndexedDocs(String index, String sourceType) throws IOException {
+       return getAllIndexedDocs(index, sourceType, null);
+    }
+    public List<Map<String, Object>> getAllIndexedDocs(String index, String sourceType, String subMessage) throws IOException {
+        getClient().admin().indices().refresh(new RefreshRequest());
+        SearchResponse response = getClient().prepareSearch(index)
+                .setTypes(sourceType)
+                .setSource("message")
+                .setFrom(0)
+                .setSize(1000)
+                .execute().actionGet();
+        List<Map<String, Object>> ret = new ArrayList<Map<String, Object>>();
+        for (SearchHit hit : response.getHits()) {
+            Object o = null;
+            if(subMessage == null) {
+                o = hit.getSource();
+            }
+            else {
+                o = hit.getSource().get(subMessage);
+            }
+            ret.add((Map<String, Object>)(o));
+        }
+        return ret;
+    }
+    public boolean hasIndex(String indexName) {
+        Set<String> indices = getClient().admin()
+                                    .indices()
+                                    .stats(new IndicesStatsRequest())
+                                    .actionGet()
+                                    .getIndices()
+                                    .keySet();
+        return indices.contains(indexName);
+
+    }
+
+    public void stop() {
+        node.stop();
+        node = null;
+        client = null;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/e59b1a31/metron-streaming/Metron-EnrichmentAdapters/src/main/java/org/apache/metron/enrichment/bolt/EnrichmentJoinBolt.java
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-EnrichmentAdapters/src/main/java/org/apache/metron/enrichment/bolt/EnrichmentJoinBolt.java b/metron-streaming/Metron-EnrichmentAdapters/src/main/java/org/apache/metron/enrichment/bolt/EnrichmentJoinBolt.java
index a2cec5a..bfb4d91 100644
--- a/metron-streaming/Metron-EnrichmentAdapters/src/main/java/org/apache/metron/enrichment/bolt/EnrichmentJoinBolt.java
+++ b/metron-streaming/Metron-EnrichmentAdapters/src/main/java/org/apache/metron/enrichment/bolt/EnrichmentJoinBolt.java
@@ -25,7 +25,11 @@ import org.json.simple.JSONObject;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.util.*;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
 
 public class EnrichmentJoinBolt extends JoinBolt<JSONObject> {
 
@@ -51,7 +55,7 @@ public class EnrichmentJoinBolt extends JoinBolt<JSONObject> {
   @Override
   public Set<String> getStreamIds(JSONObject message) {
     Set<String> streamIds = new HashSet<>();
-    String sourceType = TopologyUtils.getSourceType(message);
+    String sourceType = TopologyUtils.getSensorType(message);
     for (String enrichmentType : getFieldMap(sourceType).keySet()) {
       streamIds.add(enrichmentType);
     }
@@ -81,7 +85,7 @@ public class EnrichmentJoinBolt extends JoinBolt<JSONObject> {
     return message;
   }
 
-  public Map<String, List<String>> getFieldMap(String sourceType) {
-    return configurations.get(sourceType).getEnrichmentFieldMap();
+  protected Map<String, List<String>> getFieldMap(String sensorType) {
+    return configurations.getSensorEnrichmentConfig(sensorType).getEnrichmentFieldMap();
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/e59b1a31/metron-streaming/Metron-EnrichmentAdapters/src/main/java/org/apache/metron/enrichment/bolt/EnrichmentSplitterBolt.java
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-EnrichmentAdapters/src/main/java/org/apache/metron/enrichment/bolt/EnrichmentSplitterBolt.java b/metron-streaming/Metron-EnrichmentAdapters/src/main/java/org/apache/metron/enrichment/bolt/EnrichmentSplitterBolt.java
index 7970674..c37133d 100644
--- a/metron-streaming/Metron-EnrichmentAdapters/src/main/java/org/apache/metron/enrichment/bolt/EnrichmentSplitterBolt.java
+++ b/metron-streaming/Metron-EnrichmentAdapters/src/main/java/org/apache/metron/enrichment/bolt/EnrichmentSplitterBolt.java
@@ -105,10 +105,9 @@ public class EnrichmentSplitterBolt extends SplitBolt<JSONObject> {
     @SuppressWarnings("unchecked")
     @Override
     public Map<String, JSONObject> splitMessage(JSONObject message) {
-
         Map<String, JSONObject> streamMessageMap = new HashMap<>();
-        String sourceType = TopologyUtils.getSourceType(message);
-        Map<String, List<String>> enrichmentFieldMap = getFieldMap(sourceType);
+        String sensorType = TopologyUtils.getSensorType(message);
+        Map<String, List<String>> enrichmentFieldMap = getFieldMap(sensorType);
         for (String enrichmentType : enrichmentFieldMap.keySet()) {
             List<String> fields = enrichmentFieldMap.get(enrichmentType);
             JSONObject enrichmentObject = new JSONObject();
@@ -116,7 +115,7 @@ public class EnrichmentSplitterBolt extends SplitBolt<JSONObject> {
                 for (String field : fields) {
                     enrichmentObject.put(getKeyName(enrichmentType, field), message.get(field));
                 }
-                enrichmentObject.put(Constants.SOURCE_TYPE, sourceType);
+                enrichmentObject.put(Constants.SENSOR_TYPE, sensorType);
                 streamMessageMap.put(enrichmentType, enrichmentObject);
             }
         }
@@ -124,8 +123,8 @@ public class EnrichmentSplitterBolt extends SplitBolt<JSONObject> {
         return streamMessageMap;
     }
 
-    protected Map<String, List<String>> getFieldMap(String sourceType) {
-        return configurations.get(sourceType).getEnrichmentFieldMap();
+    protected Map<String, List<String>> getFieldMap(String sensorType) {
+        return configurations.getSensorEnrichmentConfig(sensorType).getEnrichmentFieldMap();
     }
 
     protected String getKeyName(String type, String field) {

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/e59b1a31/metron-streaming/Metron-EnrichmentAdapters/src/main/java/org/apache/metron/enrichment/bolt/GenericEnrichmentBolt.java
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-EnrichmentAdapters/src/main/java/org/apache/metron/enrichment/bolt/GenericEnrichmentBolt.java b/metron-streaming/Metron-EnrichmentAdapters/src/main/java/org/apache/metron/enrichment/bolt/GenericEnrichmentBolt.java
index b5c4c44..08b223c 100644
--- a/metron-streaming/Metron-EnrichmentAdapters/src/main/java/org/apache/metron/enrichment/bolt/GenericEnrichmentBolt.java
+++ b/metron-streaming/Metron-EnrichmentAdapters/src/main/java/org/apache/metron/enrichment/bolt/GenericEnrichmentBolt.java
@@ -18,29 +18,26 @@
 
 package org.apache.metron.enrichment.bolt;
 
-import java.util.Map;
-import java.util.concurrent.TimeUnit;
-
-import com.google.common.base.Splitter;
+import backtype.storm.task.OutputCollector;
+import backtype.storm.task.TopologyContext;
+import backtype.storm.topology.OutputFieldsDeclarer;
+import backtype.storm.tuple.Fields;
+import backtype.storm.tuple.Tuple;
+import backtype.storm.tuple.Values;
 import com.google.common.cache.CacheBuilder;
 import com.google.common.cache.CacheLoader;
 import com.google.common.cache.LoadingCache;
-import com.google.common.collect.Iterables;
 import org.apache.metron.Constants;
 import org.apache.metron.bolt.ConfiguredBolt;
 import org.apache.metron.domain.Enrichment;
 import org.apache.metron.enrichment.interfaces.EnrichmentAdapter;
+import org.apache.metron.helpers.topology.ErrorUtils;
 import org.json.simple.JSONObject;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import backtype.storm.task.OutputCollector;
-import backtype.storm.task.TopologyContext;
-import backtype.storm.topology.OutputFieldsDeclarer;
-import backtype.storm.tuple.Fields;
-import backtype.storm.tuple.Tuple;
-import backtype.storm.tuple.Values;
-import org.apache.metron.helpers.topology.ErrorUtils;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
 
 /**
  * Uses an adapter to enrich telemetry messages with additional metadata
@@ -157,8 +154,8 @@ public class GenericEnrichmentBolt extends ConfiguredBolt {
       for (Object o : rawMessage.keySet()) {
         String field = (String) o;
         String value = (String) rawMessage.get(field);
-        if (field.equals(Constants.SOURCE_TYPE)) {
-          enrichedMessage.put(Constants.SOURCE_TYPE, value);
+        if (field.equals(Constants.SENSOR_TYPE)) {
+          enrichedMessage.put(Constants.SENSOR_TYPE, value);
         } else {
           JSONObject enrichedField = new JSONObject();
           if (value != null && value.length() != 0) {

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/e59b1a31/metron-streaming/Metron-EnrichmentAdapters/src/main/java/org/apache/metron/enrichment/bolt/ThreatIntelJoinBolt.java
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-EnrichmentAdapters/src/main/java/org/apache/metron/enrichment/bolt/ThreatIntelJoinBolt.java b/metron-streaming/Metron-EnrichmentAdapters/src/main/java/org/apache/metron/enrichment/bolt/ThreatIntelJoinBolt.java
index 3516ee0..014e0a9 100644
--- a/metron-streaming/Metron-EnrichmentAdapters/src/main/java/org/apache/metron/enrichment/bolt/ThreatIntelJoinBolt.java
+++ b/metron-streaming/Metron-EnrichmentAdapters/src/main/java/org/apache/metron/enrichment/bolt/ThreatIntelJoinBolt.java
@@ -34,8 +34,8 @@ public class ThreatIntelJoinBolt extends EnrichmentJoinBolt {
   }
 
   @Override
-  public Map<String, List<String>> getFieldMap(String sourceType) {
-    return configurations.get(sourceType).getThreatIntelFieldMap();
+  public Map<String, List<String>> getFieldMap(String sensorType) {
+    return configurations.getSensorEnrichmentConfig(sensorType).getThreatIntelFieldMap();
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/e59b1a31/metron-streaming/Metron-EnrichmentAdapters/src/main/java/org/apache/metron/enrichment/bolt/ThreatIntelSplitterBolt.java
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-EnrichmentAdapters/src/main/java/org/apache/metron/enrichment/bolt/ThreatIntelSplitterBolt.java b/metron-streaming/Metron-EnrichmentAdapters/src/main/java/org/apache/metron/enrichment/bolt/ThreatIntelSplitterBolt.java
index a43360e..692c327 100644
--- a/metron-streaming/Metron-EnrichmentAdapters/src/main/java/org/apache/metron/enrichment/bolt/ThreatIntelSplitterBolt.java
+++ b/metron-streaming/Metron-EnrichmentAdapters/src/main/java/org/apache/metron/enrichment/bolt/ThreatIntelSplitterBolt.java
@@ -29,8 +29,8 @@ public class ThreatIntelSplitterBolt extends EnrichmentSplitterBolt {
   }
 
   @Override
-  protected Map<String, List<String>> getFieldMap(String sourceType) {
-    return configurations.get(sourceType).getThreatIntelFieldMap();
+  protected Map<String, List<String>> getFieldMap(String sensorType) {
+    return configurations.getSensorEnrichmentConfig(sensorType).getThreatIntelFieldMap();
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/e59b1a31/metron-streaming/Metron-Indexing/pom.xml
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-Indexing/pom.xml b/metron-streaming/Metron-Indexing/pom.xml
index 1f5d04d..d32fc07 100644
--- a/metron-streaming/Metron-Indexing/pom.xml
+++ b/metron-streaming/Metron-Indexing/pom.xml
@@ -71,11 +71,6 @@
             </exclusions>
         </dependency>
         <dependency>
-            <groupId>org.elasticsearch</groupId>
-            <artifactId>elasticsearch</artifactId>
-            <version>${global_elasticsearch_version}</version>
-        </dependency>
-        <dependency>
             <groupId>org.apache.httpcomponents</groupId>
             <artifactId>httpclient</artifactId>
             <version>${http.client.version}</version>

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/e59b1a31/metron-streaming/Metron-Indexing/src/main/java/org/apache/metron/indexing/AbstractIndexingBolt.java
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-Indexing/src/main/java/org/apache/metron/indexing/AbstractIndexingBolt.java b/metron-streaming/Metron-Indexing/src/main/java/org/apache/metron/indexing/AbstractIndexingBolt.java
deleted file mode 100644
index 423a5c2..0000000
--- a/metron-streaming/Metron-Indexing/src/main/java/org/apache/metron/indexing/AbstractIndexingBolt.java
+++ /dev/null
@@ -1,110 +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.indexing;
-
-import java.io.IOException;
-import java.util.Map;
-
-import org.apache.metron.bolt.ConfiguredBolt;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import backtype.storm.task.OutputCollector;
-import backtype.storm.task.TopologyContext;
-import backtype.storm.topology.OutputFieldsDeclarer;
-import backtype.storm.topology.base.BaseRichBolt;
-
-import com.codahale.metrics.Counter;
-import org.apache.metron.index.interfaces.IndexAdapter;
-import org.apache.metron.metrics.MetricReporter;
-
-@SuppressWarnings("rawtypes")
-public abstract class AbstractIndexingBolt extends ConfiguredBolt {
-	/**
-	 * 
-	 */
-	private static final long serialVersionUID = -6710596708304282838L;
-
-	protected static final Logger LOG = LoggerFactory
-			.getLogger(AbstractIndexingBolt.class);
-
-	protected OutputCollector _collector;
-	protected IndexAdapter _adapter;
-	protected MetricReporter _reporter;
-
-	protected String _IndexIP;
-	protected int _IndexPort = 0;
-	protected String _ClusterName;
-	protected String _IndexName;
-	protected String _DocumentName;
-	protected int _BulkIndexNumber = 10;
-
-	protected Counter ackCounter, emitCounter, failCounter;
-
-	public AbstractIndexingBolt(String zookeeperUrl) {
-		super(zookeeperUrl);
-	}
-
-	protected void registerCounters() {
-
-		String ackString = _adapter.getClass().getSimpleName() + ".ack";
-
-		String emitString = _adapter.getClass().getSimpleName() + ".emit";
-
-		String failString = _adapter.getClass().getSimpleName() + ".fail";
-
-		ackCounter = _reporter.registerCounter(ackString);
-		emitCounter = _reporter.registerCounter(emitString);
-		failCounter = _reporter.registerCounter(failString);
-
-	}
-
-	public final void prepare(Map conf, TopologyContext topologyContext,
-			OutputCollector collector) {
-		_collector = collector;
-
-		if (this._IndexIP == null)
-			throw new IllegalStateException("_IndexIP must be specified");
-		if (this._IndexPort == 0)
-			throw new IllegalStateException("_IndexPort must be specified");
-		if (this._ClusterName == null)
-			throw new IllegalStateException("_ClusterName must be specified");
-		if (this._IndexName == null)
-			throw new IllegalStateException("_IndexName must be specified");
-		if (this._DocumentName == null)
-			throw new IllegalStateException("_DocumentName must be specified");
-		if (this._adapter == null)
-			throw new IllegalStateException("IndexAdapter must be specified");
-
-		try {
-			doPrepare(conf, topologyContext, collector);
-		} catch (IOException e) {
-			LOG.error("Counld not initialize...");
-			e.printStackTrace();
-		}
-	}
-
-	public void declareOutputFields(OutputFieldsDeclarer declearer) {
-		
-	}
-
-	abstract void doPrepare(Map conf, TopologyContext topologyContext,
-			OutputCollector collector) throws IOException;
-
-}


[08/15] incubator-metron git commit: METRON 86: Adding Solr indexing support (merrimanr via cestella) closes apache/incubator-metron#67

Posted by ce...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/e59b1a31/metron-streaming/Metron-Testing/src/main/resources/sample/data/SampleInput/ISESampleOutput
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-Testing/src/main/resources/sample/data/SampleInput/ISESampleOutput b/metron-streaming/Metron-Testing/src/main/resources/sample/data/SampleInput/ISESampleOutput
new file mode 100644
index 0000000..1cb0678
--- /dev/null
+++ b/metron-streaming/Metron-Testing/src/main/resources/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 Net

<TRUNCATED>


[07/15] incubator-metron git commit: METRON 86: Adding Solr indexing support (merrimanr via cestella) closes apache/incubator-metron#67

Posted by ce...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/e59b1a31/metron-streaming/Metron-Testing/src/main/resources/sample/data/SampleInput/LancopeExampleOutput
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-Testing/src/main/resources/sample/data/SampleInput/LancopeExampleOutput b/metron-streaming/Metron-Testing/src/main/resources/sample/data/SampleInput/LancopeExampleOutput
new file mode 100644
index 0000000..b1bccf9
--- /dev/null
+++ b/metron-streaming/Metron-Testing/src/main/resources/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/e59b1a31/metron-streaming/Metron-Testing/src/main/resources/sample/data/SampleInput/PCAPExampleOutput
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-Testing/src/main/resources/sample/data/SampleInput/PCAPExampleOutput b/metron-streaming/Metron-Testing/src/main/resources/sample/data/SampleInput/PCAPExampleOutput
new file mode 100644
index 0000000..e730181
Binary files /dev/null and b/metron-streaming/Metron-Testing/src/main/resources/sample/data/SampleInput/PCAPExampleOutput differ

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/e59b1a31/metron-streaming/Metron-Testing/src/main/resources/sample/data/SampleInput/PaloaltoOutput
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-Testing/src/main/resources/sample/data/SampleInput/PaloaltoOutput b/metron-streaming/Metron-Testing/src/main/resources/sample/data/SampleInput/PaloaltoOutput
new file mode 100644
index 0000000..16793a2
--- /dev/null
+++ b/metron-streaming/Metron-Testing/src/main/resources/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/e59b1a31/metron-streaming/Metron-Testing/src/main/resources/sample/data/SampleInput/SnortOutput
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-Testing/src/main/resources/sample/data/SampleInput/SnortOutput b/metron-streaming/Metron-Testing/src/main/resources/sample/data/SampleInput/SnortOutput
new file mode 100644
index 0000000..0497b0f
--- /dev/null
+++ b/metron-streaming/Metron-Testing/src/main/resources/sample/data/SampleInput/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/e59b1a31/metron-streaming/Metron-Testing/src/main/resources/sample/data/SampleInput/SourcefireExampleOutput
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-Testing/src/main/resources/sample/data/SampleInput/SourcefireExampleOutput b/metron-streaming/Metron-Testing/src/main/resources/sample/data/SampleInput/SourcefireExampleOutput
new file mode 100644
index 0000000..5f177df
--- /dev/null
+++ b/metron-streaming/Metron-Testing/src/main/resources/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

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/e59b1a31/metron-streaming/Metron-Testing/src/main/resources/sample/data/SampleInput/YafExampleOutput
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-Testing/src/main/resources/sample/data/SampleInput/YafExampleOutput b/metron-streaming/Metron-Testing/src/main/resources/sample/data/SampleInput/YafExampleOutput
new file mode 100644
index 0000000..8f3ff44
--- /dev/null
+++ b/metron-streaming/Metron-Testing/src/main/resources/sample/data/SampleInput/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/e59b1a31/metron-streaming/Metron-Testing/src/main/resources/sample/data/SampleParsed/SnortParsed
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-Testing/src/main/resources/sample/data/SampleParsed/SnortParsed b/metron-streaming/Metron-Testing/src/main/resources/sample/data/SampleParsed/SnortParsed
new file mode 100644
index 0000000..318b158
--- /dev/null
+++ b/metron-streaming/Metron-Testing/src/main/resources/sample/data/SampleParsed/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/e59b1a31/metron-streaming/Metron-Testing/src/main/resources/sample/data/SampleParsed/YafExampleParsed
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-Testing/src/main/resources/sample/data/SampleParsed/YafExampleParsed b/metron-streaming/Metron-Testing/src/main/resources/sample/data/SampleParsed/YafExampleParsed
new file mode 100644
index 0000000..6155e98
--- /dev/null
+++ b/metron-streaming/Metron-Testing/src/main/resources/sample/data/SampleParsed/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/e59b1a31/metron-streaming/Metron-Topologies/pom.xml
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-Topologies/pom.xml b/metron-streaming/Metron-Topologies/pom.xml
index 9077b82..ad60980 100644
--- a/metron-streaming/Metron-Topologies/pom.xml
+++ b/metron-streaming/Metron-Topologies/pom.xml
@@ -47,6 +47,12 @@
             <scope>test</scope>
         </dependency>
         <dependency>
+            <groupId>org.apache.commons</groupId>
+            <artifactId>commons-math3</artifactId>
+            <version>3.1.1</version>
+            <scope>compile</scope>
+        </dependency>
+        <dependency>
             <groupId>org.apache.metron</groupId>
             <artifactId>Metron-Common</artifactId>
             <version>${project.parent.version}</version>
@@ -137,6 +143,12 @@
             <groupId>org.apache.storm</groupId>
             <artifactId>storm-kafka</artifactId>
             <version>${global_storm_version}</version>
+            <exclusions>
+                <exclusion>
+                    <artifactId>org.apache.curator</artifactId>
+                    <groupId>curator-client</groupId>
+                </exclusion>
+            </exclusions>
         </dependency>
         <dependency>
             <groupId>org.apache.storm</groupId>
@@ -235,12 +247,12 @@
                         </configuration>
                     </execution>
                 </executions>
-            </plugin> 
-            
+            </plugin>
+
             <plugin>
                 <groupId>org.apache.maven.plugins</groupId>
                 <artifactId>maven-shade-plugin</artifactId>
-                <version>1.4</version>
+                <version>2.3</version>
                 <configuration>
                     <createDependencyReducedPom>true</createDependencyReducedPom>
                 </configuration>
@@ -265,7 +277,7 @@
                                     <resource>.yaml</resource>
                                 </transformer>
                                 <transformer
-                                        implementation="org.apache.maven.plugins.shade.resource.ServicesResourceTransformer" />
+                                        implementation="org.apache.maven.plugins.shade.resource.ServicesResourceTransformer"/>
                                 <transformer
                                         implementation="org.apache.maven.plugins.shade.resource.ManifestResourceTransformer">
                                     <mainClass></mainClass>

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/e59b1a31/metron-streaming/Metron-Topologies/src/main/assembly/assembly.xml
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-Topologies/src/main/assembly/assembly.xml b/metron-streaming/Metron-Topologies/src/main/assembly/assembly.xml
index fee1238..69556cb 100644
--- a/metron-streaming/Metron-Topologies/src/main/assembly/assembly.xml
+++ b/metron-streaming/Metron-Topologies/src/main/assembly/assembly.xml
@@ -19,7 +19,7 @@
   <includeBaseDirectory>false</includeBaseDirectory>
   <fileSets>
     <fileSet>
-      <directory>${project.basedir}/src/main/bash</directory>
+      <directory>${project.basedir}/../Metron-Topologies/src/main/bash</directory>
       <outputDirectory>/bin</outputDirectory>
       <useDefaultExcludes>true</useDefaultExcludes>
       <excludes>
@@ -30,7 +30,7 @@
       <lineEnding>unix</lineEnding>
     </fileSet>
     <fileSet>
-      <directory>${project.basedir}/src/main/resources/Metron_Configs/etc</directory>
+      <directory>${project.basedir}/../Metron-Topologies/src/main/resources/Metron_Configs/etc</directory>
       <outputDirectory>/config/etc</outputDirectory>
       <useDefaultExcludes>true</useDefaultExcludes>
       <excludes>
@@ -41,7 +41,7 @@
       <lineEnding>unix</lineEnding>
     </fileSet>
     <fileSet>
-      <directory>${project.basedir}/src/main/resources/Metron_Configs/topologies</directory>
+      <directory>${project.basedir}/../Metron-Topologies/src/main/resources/Metron_Configs/topologies</directory>
       <outputDirectory>/config/topologies</outputDirectory>
       <useDefaultExcludes>true</useDefaultExcludes>
       <excludes>

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/e59b1a31/metron-streaming/Metron-Topologies/src/main/bash/start_elasticsearch_topology.sh
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-Topologies/src/main/bash/start_elasticsearch_topology.sh b/metron-streaming/Metron-Topologies/src/main/bash/start_elasticsearch_topology.sh
new file mode 100755
index 0000000..68b4536
--- /dev/null
+++ b/metron-streaming/Metron-Topologies/src/main/bash/start_elasticsearch_topology.sh
@@ -0,0 +1,22 @@
+#!/bin/bash
+# 
+# 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.
+#
+METRON_VERSION=0.1BETA
+METRON_HOME=/usr/metron/$METRON_VERSION
+TOPOLOGY_JAR=Metron-Elasticsearch-$METRON_VERSION.jar
+storm jar $METRON_HOME/lib/$TOPOLOGY_JAR org.apache.storm.flux.Flux --remote $METRON_HOME/config/topologies/$1/remote.yaml --filter $METRON_HOME/config/etc/env/elasticsearch.properties

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/e59b1a31/metron-streaming/Metron-Topologies/src/main/bash/start_solr_topology.sh
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-Topologies/src/main/bash/start_solr_topology.sh b/metron-streaming/Metron-Topologies/src/main/bash/start_solr_topology.sh
new file mode 100755
index 0000000..472b0e1
--- /dev/null
+++ b/metron-streaming/Metron-Topologies/src/main/bash/start_solr_topology.sh
@@ -0,0 +1,22 @@
+#!/bin/bash
+# 
+# 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.
+#
+METRON_VERSION=0.1BETA
+METRON_HOME=/usr/metron/$METRON_VERSION
+TOPOLOGY_JAR=Metron-Solr-$METRON_VERSION.jar
+storm jar $METRON_HOME/lib/$TOPOLOGY_JAR org.apache.storm.flux.Flux --remote $METRON_HOME/config/topologies/$1/remote.yaml --filter $METRON_HOME/config/etc/env/solr.properties

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/e59b1a31/metron-streaming/Metron-Topologies/src/main/bash/start_topology.sh
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-Topologies/src/main/bash/start_topology.sh b/metron-streaming/Metron-Topologies/src/main/bash/start_topology.sh
deleted file mode 100755
index 21626c2..0000000
--- a/metron-streaming/Metron-Topologies/src/main/bash/start_topology.sh
+++ /dev/null
@@ -1,22 +0,0 @@
-#!/bin/bash
-# 
-# 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.
-#
-METRON_VERSION=0.1BETA
-METRON_HOME=/usr/metron/$METRON_VERSION
-TOPOLOGY_JAR=Metron-Topologies-$METRON_VERSION.jar
-storm jar $METRON_HOME/lib/$TOPOLOGY_JAR org.apache.storm.flux.Flux --remote $METRON_HOME/config/topologies/$1/remote.yaml --filter $METRON_HOME/config/etc/env/config.properties