You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@metron.apache.org by js...@apache.org on 2016/03/19 22:12:39 UTC

[1/3] incubator-metron git commit: METRON-72 Create unified enrichment topology (merrimanr via jsirota) closes apache/incubator-metron#50

Repository: incubator-metron
Updated Branches:
  refs/heads/master d28083701 -> 68aab6e9a


http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/c737aa9d/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
new file mode 100644
index 0000000..284e3c0
--- /dev/null
+++ b/metron-streaming/Metron-Topologies/src/test/java/org/apache/metron/integration/PcapParserIntegrationTest.java
@@ -0,0 +1,218 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.metron.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;
+import org.apache.hadoop.hbase.client.HTableInterface;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.ResultScanner;
+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.hbase.TableProvider;
+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.FluxTopologyComponent;
+import org.apache.metron.integration.util.integration.components.KafkaWithZKComponent;
+import org.apache.metron.integration.util.mock.MockHTable;
+import org.apache.metron.parsing.parsers.PcapParser;
+import org.apache.metron.pcap.PcapUtils;
+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.*;
+
+public class PcapParserIntegrationTest {
+
+  private static String BASE_DIR = "pcap";
+  private static String DATA_DIR = BASE_DIR + "/data_dir";
+  private static String QUERY_DIR = BASE_DIR + "/query";
+  private String topologiesDir = "src/main/resources/Metron_Configs/topologies";
+  private String targetDir = "target";
+
+  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);
+    }
+  }
+
+  private File getOutDir(String targetDir) {
+    File outDir = new File(new File(targetDir), DATA_DIR);
+    if (!outDir.exists()) {
+      outDir.mkdirs();
+    }
+
+    return outDir;
+  }
+
+  private File getQueryDir(String targetDir) {
+    File outDir = new File(new File(targetDir), QUERY_DIR);
+    if (!outDir.exists()) {
+      outDir.mkdirs();
+    }
+    return outDir;
+  }
+  private static void clearOutDir(File outDir) {
+    for(File f : outDir.listFiles()) {
+      f.delete();
+    }
+  }
+
+  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)) {
+      int keyInt = key.get();
+      byte[] valueBytes = value.copyBytes();
+      JSONObject message = parser.parse(valueBytes).get(0);
+      if (parser.validate(message)) {
+        ret.put(PcapUtils.getSessionKey(message), valueBytes);
+      }
+    }
+    return ret;
+  }
+
+  @Test
+  public void testTopology() throws Exception {
+    if (!new File(topologiesDir).exists()) {
+      topologiesDir = UnitTestHelper.findDir("topologies");
+    }
+    targetDir = UnitTestHelper.findDir("target");
+    final String kafkaTopic = "pcap";
+    final String tableName = "pcap";
+    final String columnFamily = "t";
+    final String columnIdentifier = "value";
+    final File outDir = getOutDir(targetDir);
+    final File queryDir = getQueryDir(targetDir);
+    clearOutDir(outDir);
+    clearOutDir(queryDir);
+
+    File baseDir = new File(new File(targetDir), BASE_DIR);
+    Assert.assertNotNull(topologiesDir);
+    Assert.assertNotNull(targetDir);
+    Path pcapFile = new Path(topologiesDir + "/../../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("spout.kafka.topic.pcap", kafkaTopic);
+      setProperty("bolt.hbase.table.name",tableName);
+      setProperty("bolt.hbase.table.fields", columnFamily + ":" + columnIdentifier);
+    }};
+    final KafkaWithZKComponent kafkaComponent = new KafkaWithZKComponent().withTopics(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");
+
+    FluxTopologyComponent fluxComponent = new FluxTopologyComponent.Builder()
+            .withTopologyLocation(new File(topologiesDir + "/pcap/test.yaml"))
+            .withTopologyName("pcap")
+            .withTopologyProperties(topologyProperties)
+            .build();
+
+    final MockHTable pcapTable = (MockHTable)MockHTable.Provider.addToCache(tableName, columnFamily);
+
+    UnitTestHelper.verboseLogging();
+    ComponentRunner runner = new ComponentRunner.Builder()
+            .withComponent("kafka", kafkaComponent)
+            .withComponent("storm", fluxComponent)
+            .withMaxTimeMS(60000)
+            .withMillisecondsBetweenAttempts(6000)
+            .withNumRetries(10)
+            .build();
+    try {
+      runner.start();
+      System.out.println("Components started...");
+      fluxComponent.submitTopology();
+      kafkaComponent.writeMessages(kafkaTopic, pcapEntries.values());
+      System.out.println("Sent pcap data: " + pcapEntries.size());
+      List<byte[]> messages = kafkaComponent.readMessages(kafkaTopic);
+      Assert.assertEquals(pcapEntries.size(), messages.size());
+      System.out.println("Wrote " + pcapEntries.size() + " to kafka");
+      runner.process(new Processor<Void>() {
+        @Override
+        public ReadinessState process(ComponentRunner runner) {
+          int hbaseCount = 0;
+          try {
+            ResultScanner resultScanner = pcapTable.getScanner(columnFamily.getBytes(), columnIdentifier.getBytes());
+            while(resultScanner.next() != null) hbaseCount++;
+          } catch (IOException e) {
+            e.printStackTrace();
+          }
+          if (hbaseCount == pcapEntries.size()) {
+            return ReadinessState.READY;
+          } else {
+            return ReadinessState.NOT_READY;
+          }
+        }
+
+        @Override
+        public Void getResult() {
+          return null;
+        }
+      });
+      ResultScanner resultScanner = pcapTable.getScanner(columnFamily.getBytes(), columnIdentifier.getBytes());
+      Result result;
+      int rowCount = 0;
+      while((result = resultScanner.next()) != null) {
+        String rowKey = new String(result.getRow());
+        byte[] hbaseValue = result.getValue(columnFamily.getBytes(), columnIdentifier.getBytes());
+        byte[] originalValue = pcapEntries.get(rowKey);
+        Assert.assertNotNull("Could not find pcap with key " + rowKey + " in sample data", originalValue);
+        Assert.assertArrayEquals("Raw values are different for key " + rowKey, originalValue, hbaseValue);
+        rowCount++;
+      }
+      Assert.assertEquals(pcapEntries.size(), rowCount);
+      System.out.println("Ended");
+    }
+    finally {
+      runner.stop();
+      clearOutDir(outDir);
+      clearOutDir(queryDir);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/c737aa9d/metron-ui/.gitignore
----------------------------------------------------------------------
diff --git a/metron-ui/.gitignore b/metron-ui/.gitignore
index c94c2a1..a6dfcf4 100644
--- a/metron-ui/.gitignore
+++ b/metron-ui/.gitignore
@@ -9,9 +9,6 @@ pids
 # Pcap files
 *.pcap
 
-# Config overrides
-config.json
-
 # Directory for instrumented libs generated by jscoverage/JSCover
 lib-cov
 

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/c737aa9d/metron-ui/config.json
----------------------------------------------------------------------
diff --git a/metron-ui/config.json b/metron-ui/config.json
new file mode 100644
index 0000000..dde17b2
--- /dev/null
+++ b/metron-ui/config.json
@@ -0,0 +1,6 @@
+{
+  "auth":false,
+  "secret":"secret",
+  "elasticsearch": { "url": "http://host:port" },
+  "pcap": { "url": "http://host:port/pcapGetter","mock": false }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/c737aa9d/metron-ui/lib/metron-ui.js
----------------------------------------------------------------------
diff --git a/metron-ui/lib/metron-ui.js b/metron-ui/lib/metron-ui.js
index c11d7a8..14aeac3 100644
--- a/metron-ui/lib/metron-ui.js
+++ b/metron-ui/lib/metron-ui.js
@@ -23,6 +23,7 @@ var path = require('path');
 var express = require('express');
 
 var connect = require('connect');
+var serveStatic = require('serve-static');
 var flash = require('connect-flash');
 
 var cookieParser = require('cookie-parser');
@@ -37,7 +38,7 @@ var login = require('./modules/login');
 var pcap = require('./modules/pcap');
 
 var app = express();
-var config = require('./config');
+var config = require('../config.json');
 
 
 try {
@@ -51,7 +52,7 @@ app.set('view engine', 'jade');
 app.set('views', path.join(__dirname, 'views/'));
 
 // Cookie middleware
-app.use(connect.logger('dev'));
+//app.use(connect.logger('dev'));
 app.use(flash());
 app.use(cookieParser());
 app.use(cookieSession({
@@ -110,7 +111,7 @@ pcap(app, config);
 login(app, config);
 
 // Serve static assets
-app.use(connect.static(path.join(__dirname, 'public')));
+app.use(serveStatic(path.join(__dirname, 'public')));
 
 
 // Start server

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/c737aa9d/metron-ui/lib/modules/es-proxy.js
----------------------------------------------------------------------
diff --git a/metron-ui/lib/modules/es-proxy.js b/metron-ui/lib/modules/es-proxy.js
index b9b5a63..b805fea 100644
--- a/metron-ui/lib/modules/es-proxy.js
+++ b/metron-ui/lib/modules/es-proxy.js
@@ -25,7 +25,7 @@ exports = module.exports = function(config) {
   });
 
   return function(req, res, next) {
-    if (!req.user) {
+    if (config.auth && !req.user) {
       res.send(403, 'Forbidden!');
       return;
     }

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/c737aa9d/metron-ui/lib/modules/login.js
----------------------------------------------------------------------
diff --git a/metron-ui/lib/modules/login.js b/metron-ui/lib/modules/login.js
index 7fa2c7b..2ad669a 100644
--- a/metron-ui/lib/modules/login.js
+++ b/metron-ui/lib/modules/login.js
@@ -20,7 +20,7 @@ exports = module.exports = function(app, config) {
   var passport = require('passport');
 
   app.get('/', function (req, res, next) {
-    if (!req.user) {
+    if (config.auth && !req.user ) {
       res.redirect('/login');
       return;
     }

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/c737aa9d/metron-ui/lib/modules/pcap.js
----------------------------------------------------------------------
diff --git a/metron-ui/lib/modules/pcap.js b/metron-ui/lib/modules/pcap.js
index 5b0a7fc..eaad29e 100644
--- a/metron-ui/lib/modules/pcap.js
+++ b/metron-ui/lib/modules/pcap.js
@@ -62,7 +62,7 @@ exports = module.exports = function(app, config) {
   }
 
   app.get('/pcap/:command', function(req, res) {
-    if (!req.user || !req.user.permissions.pcap) {
+    if (config.auth && (!req.user || !req.user.permissions.pcap)) {
       res.send(403, 'Forbidden!');
       return;
     }
@@ -72,6 +72,16 @@ exports = module.exports = function(app, config) {
     pcapUrl += '?' + querystring.stringify(req.query);
 
     var curl = spawn('curl', ['-s', pcapUrl]);
+
+    if (true) {
+      res.set('Content-Type', 'application/cap');
+      var fileName = req.query.srcIp + "-" + req.query.dstIp + '-' + req.query.srcPort + '-' + req.query.dstPort + '-' + req.query.protocol + '-' + req.query.includeReverseTraffic;
+      fileName = fileName.replace(/\./g, '_');
+      res.set('Content-Disposition', 'attachment; filename="' + fileName + '.pcap"');
+      curl.stdout.pipe(res);
+      return;
+    }
+
     var tshark = spawn('tshark', ['-i', '-', '-T', 'pdml']);
     var xml = new XmlStream(tshark.stdout);
 

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/c737aa9d/metron-ui/lib/public/app/dashboards/default.json
----------------------------------------------------------------------
diff --git a/metron-ui/lib/public/app/dashboards/default.json b/metron-ui/lib/public/app/dashboards/default.json
index 9cd224e..4d3dd24 100644
--- a/metron-ui/lib/public/app/dashboards/default.json
+++ b/metron-ui/lib/public/app/dashboards/default.json
@@ -1,70 +1,70 @@
 {
-  "title": "New Dashboard",
+  "title": "Metron",
   "services": {
     "query": {
       "list": {
         "0": {
-          "query": "alerts.triggered.priority:1",
-          "alias": "",
-          "color": "#BF1B00",
           "id": 0,
+          "color": "#7EB26D",
+          "alias": "alerts",
           "pin": true,
           "type": "lucene",
-          "enable": true
+          "enable": true,
+          "query": "is_alert=true"
         },
         "1": {
           "id": 1,
-          "color": "#EAB839",
-          "alias": "",
+          "color": "#6ED0E0",
+          "alias": "Yaf",
           "pin": true,
           "type": "lucene",
           "enable": true,
-          "query": "alerts.triggered.priority:2"
+          "query": "_type:yaf_doc"
         },
         "2": {
           "id": 2,
-          "color": "#6ED0E0",
-          "alias": "",
+          "color": "#BA43A9",
+          "alias": "All Events",
           "pin": true,
           "type": "lucene",
           "enable": true,
-          "query": "alerts.triggered.priority:3"
+          "query": "_type:*_doc"
         },
         "3": {
           "id": 3,
-          "color": "#E5AC0E",
-          "alias": "Warning",
-          "pin": false,
+          "color": "#BA43A9",
+          "alias": "All Alerts",
+          "pin": true,
           "type": "lucene",
           "enable": true,
-          "query": ""
+          "query": "_type:*_alert"
         },
         "4": {
           "id": 4,
-          "color": "#E24D42",
-          "alias": "1. Alert",
+          "color": "#1F78C1",
+          "alias": "Bro Events",
           "pin": true,
           "type": "lucene",
           "enable": true,
-          "query": "alerts.triggered.type:alert"
+          "query": "_type:bro_doc"
         },
         "5": {
           "id": 5,
-          "color": "#F2C96D",
-          "alias": "3. Error",
+          "color": "#EF843C",
+          "alias": "Bro Alerts",
           "pin": true,
           "type": "lucene",
           "enable": true,
-          "query": "alerts.triggered.type:error"
+          "query": "_type:bro_alert"
         },
         "6": {
           "id": 6,
-          "color": "#F9934E",
-          "alias": "2. Warning",
+          "color": "#BA43A9",
+          "alias": "Snort Events",
           "pin": true,
           "type": "lucene",
           "enable": true,
-          "query": "alerts.triggered.type:warning"
+          "query": "_type:snort_doc"
         }
       },
       "ids": [
@@ -73,145 +73,192 @@
         2,
         3,
         4,
-        6,
-        5
+        5,
+        6
       ]
     },
     "filter": {
       "list": {
         "0": {
-          "type": "field",
-          "field": "_type",
-          "query": "\"pcap\"",
-          "mandate": "mustNot",
-          "active": true,
-          "alias": "",
-          "id": 0
-        },
-        "1": {
           "type": "time",
-          "field": "message.timestamp",
-          "from": "now-12h",
+          "field": "timestamp",
+          "from": "now-24h",
           "to": "now",
           "mandate": "must",
           "active": true,
           "alias": "",
-          "id": 1
+          "id": 0
         }
       },
       "ids": [
-        0,
-        1
+        0
       ]
     }
   },
   "rows": [
     {
-      "title": "Fixed Overview",
-      "height": "250px",
+      "title": "Histogram",
+      "height": "150px",
       "editable": true,
       "collapse": false,
       "collapsable": true,
       "panels": [
         {
-          "error": false,
-          "span": 3,
+          "span": 6,
           "editable": true,
-          "type": "terms",
+          "type": "histogram",
           "loadingEditor": false,
-          "field": "alerts.triggered.type",
-          "exclude": [],
-          "missing": false,
-          "other": false,
-          "size": 10,
-          "order": "count",
-          "style": {
-            "font-size": "10pt"
+          "mode": "count",
+          "time_field": "timestamp",
+          "value_field": null,
+          "x-axis": true,
+          "y-axis": true,
+          "scale": 1,
+          "y_format": "short",
+          "grid": {
+            "max": null,
+            "min": 0
           },
-          "donut": false,
-          "tilt": false,
-          "labels": true,
-          "arrangement": "horizontal",
-          "chart": "bar",
-          "counter_pos": "above",
-          "spyable": true,
           "queries": {
-            "mode": "all",
+            "mode": "selected",
             "ids": [
-              0,
-              1,
-              2,
-              3,
               4,
-              5,
-              6
+              5
             ]
           },
           "locked": false,
-          "tmode": "terms",
-          "tstat": "total",
-          "valuefield": "",
-          "title": "Alert Type"
+          "annotate": {
+            "enable": false,
+            "query": "*",
+            "size": 20,
+            "field": "_type",
+            "sort": [
+              "_score",
+              "desc"
+            ]
+          },
+          "auto_int": true,
+          "resolution": 100,
+          "interval": "10m",
+          "intervals": [
+            "auto",
+            "1s",
+            "1m",
+            "5m",
+            "10m",
+            "30m",
+            "1h",
+            "3h",
+            "12h",
+            "1d",
+            "1w",
+            "1y"
+          ],
+          "lines": false,
+          "fill": 0,
+          "linewidth": 3,
+          "points": false,
+          "pointradius": 5,
+          "bars": true,
+          "stack": false,
+          "spyable": true,
+          "zoomlinks": true,
+          "options": true,
+          "legend": true,
+          "show_query": true,
+          "interactive": true,
+          "legend_counts": true,
+          "timezone": "browser",
+          "percentage": false,
+          "zerofill": true,
+          "derivative": false,
+          "tooltip": {
+            "value_type": "individual",
+            "query_as_alias": true
+          },
+          "title": "Bro Data"
         },
         {
-          "error": false,
-          "span": 4,
+          "span": 6,
           "editable": true,
-          "type": "terms",
+          "type": "histogram",
           "loadingEditor": false,
-          "field": "_type",
-          "exclude": [
-            "pcap"
-          ],
-          "missing": false,
-          "other": false,
-          "size": 10,
-          "order": "count",
-          "style": {
-            "font-size": "10pt"
+          "mode": "count",
+          "time_field": "timestamp",
+          "value_field": null,
+          "x-axis": true,
+          "y-axis": true,
+          "scale": 1,
+          "y_format": "none",
+          "grid": {
+            "max": null,
+            "min": 0
           },
-          "donut": false,
-          "tilt": false,
-          "labels": true,
-          "arrangement": "horizontal",
-          "chart": "pie",
-          "counter_pos": "above",
-          "spyable": true,
           "queries": {
-            "mode": "all",
+            "mode": "selected",
             "ids": [
-              0,
-              1,
-              2,
-              3,
-              4,
-              5,
-              6
+              1
             ]
           },
           "locked": false,
-          "tmode": "terms",
-          "tstat": "total",
-          "valuefield": "",
-          "title": "Alert Source"
-        }
-      ],
-      "notice": false
-    },
-    {
-      "title": "",
-      "height": "150px",
-      "editable": true,
-      "collapse": false,
-      "collapsable": true,
-      "panels": [
+          "annotate": {
+            "enable": false,
+            "query": "*",
+            "size": 20,
+            "field": "_type",
+            "sort": [
+              "_score",
+              "desc"
+            ]
+          },
+          "auto_int": true,
+          "resolution": 100,
+          "interval": "10m",
+          "intervals": [
+            "auto",
+            "1s",
+            "1m",
+            "5m",
+            "10m",
+            "30m",
+            "1h",
+            "3h",
+            "12h",
+            "1d",
+            "1w",
+            "1y"
+          ],
+          "lines": false,
+          "fill": 0,
+          "linewidth": 3,
+          "points": false,
+          "pointradius": 5,
+          "bars": true,
+          "stack": true,
+          "spyable": true,
+          "zoomlinks": true,
+          "options": true,
+          "legend": true,
+          "show_query": true,
+          "interactive": true,
+          "legend_counts": true,
+          "timezone": "browser",
+          "percentage": false,
+          "zerofill": true,
+          "derivative": false,
+          "tooltip": {
+            "value_type": "cumulative",
+            "query_as_alias": true
+          },
+          "title": "Yaf Data"
+        },
         {
-          "span": 8,
+          "span": 12,
           "editable": true,
           "type": "histogram",
           "loadingEditor": false,
           "mode": "count",
-          "time_field": "message.timestamp",
+          "time_field": "timestamp",
           "value_field": null,
           "x-axis": true,
           "y-axis": true,
@@ -224,8 +271,6 @@
           "queries": {
             "mode": "selected",
             "ids": [
-              4,
-              5,
               6
             ]
           },
@@ -242,7 +287,7 @@
           },
           "auto_int": true,
           "resolution": 100,
-          "interval": "5m",
+          "interval": "10m",
           "intervals": [
             "auto",
             "1s",
@@ -279,56 +324,71 @@
             "value_type": "cumulative",
             "query_as_alias": true
           },
-          "title": "Alert History Timeline"
-        },
+          "title": "Snort Data"
+        }
+      ],
+      "notice": false
+    },
+    {
+      "title": "Alerts",
+      "height": "150px",
+      "editable": true,
+      "collapse": false,
+      "collapsable": true,
+      "panels": [
         {
           "error": false,
-          "span": 4,
+          "span": 12,
           "editable": true,
-          "type": "terms",
+          "type": "table",
           "loadingEditor": false,
-          "field": "message.protocol",
-          "exclude": [],
-          "missing": false,
-          "other": false,
           "size": 10,
-          "order": "count",
-          "style": {
-            "font-size": "10pt"
-          },
-          "donut": false,
-          "tilt": false,
-          "labels": true,
-          "arrangement": "horizontal",
-          "chart": "table",
-          "counter_pos": "above",
+          "pages": 100,
+          "offset": 0,
+          "sort": [
+            "timestamp",
+            "desc"
+          ],
+          "overflow": "min-height",
+          "fields": [
+            "_type",
+            "msg",
+            "ip_src_addr",
+            "ip_src_port",
+            "ip_dst_addr",
+            "ip_dst_port"
+          ],
+          "highlight": [],
+          "sortable": true,
+          "header": true,
+          "paging": true,
+          "field_list": false,
+          "all_fields": false,
+          "trimFactor": 400,
+          "localTime": true,
+          "timeField": "timestamp",
           "spyable": true,
           "queries": {
-            "mode": "all",
+            "mode": "selected",
             "ids": [
-              0,
-              1,
-              2,
-              3,
-              4,
-              5,
-              6
+              0
             ]
           },
           "locked": false,
-          "tmode": "terms",
-          "tstat": "total",
-          "valuefield": "",
-          "title": "Protocol"
+          "style": {
+            "font-size": "9pt"
+          },
+          "normTimes": true,
+          "title": "Alerts"
         }
       ],
       "notice": false
     },
     {
-      "title": "",
+      "title": "Events",
       "height": "150px",
       "editable": true,
-      "collapse": true,
+      "collapse": false,
       "collapsable": true,
       "panels": [
         {
@@ -337,36 +397,38 @@
           "editable": true,
           "type": "table",
           "loadingEditor": false,
-          "size": 100,
-          "pages": 5,
+          "size": 10,
+          "pages": 100,
           "offset": 0,
           "sort": [
-            "alerts.triggered.priority",
+            "timestamp",
             "desc"
           ],
           "overflow": "min-height",
           "fields": [
             "_type",
-            "_index",
-            "alerts.triggered.priority"
+            "timestamp",
+            "ip_src_addr",
+            "ip_src_port",
+            "ip_dst_addr",
+            "ip_dst_port",
+            "protocol",
+            "original_string"
           ],
           "highlight": [],
           "sortable": true,
           "header": true,
           "paging": true,
-          "field_list": true,
+          "field_list": false,
           "all_fields": false,
-          "trimFactor": 300,
-          "localTime": false,
-          "timeField": "@timestamp",
+          "trimFactor": 400,
+          "localTime": true,
+          "timeField": "timestamp",
           "spyable": true,
           "queries": {
-            "mode": "all",
+            "mode": "selected",
             "ids": [
-              0,
-              1,
-              2,
-              3
+              4
             ]
           },
           "locked": false,
@@ -374,13 +436,13 @@
             "font-size": "9pt"
           },
           "normTimes": true,
-          "title": "all"
+          "title": "Bro"
         }
       ],
       "notice": false
     },
     {
-      "title": "",
+      "title": "PCAP Data",
       "height": "150px",
       "editable": true,
       "collapse": false,
@@ -392,8 +454,8 @@
           "editable": true,
           "type": "table",
           "loadingEditor": false,
-          "size": 10,
-          "pages": 5000,
+          "size": 25,
+          "pages": 5,
           "offset": 0,
           "sort": [
             "_score",
@@ -401,45 +463,41 @@
           ],
           "overflow": "min-height",
           "fields": [
-            "alerts.triggered.type",
-            "alerts.triggered.priority",
-            "alerts.triggered.title",
-            "alerts.triggered.body",
-            "_type",
-            "message.ip_dst_port",
-            "message.ip_dst_addr",
-            "message.timestamp",
-            "message.protocol",
-            "message.ip_src_addr"
+            "ip_src_addr",
+            "ip_src_port",
+            "ip_dst_addr",
+            "ip_dst_port",
+            "original_string"
           ],
           "highlight": [],
           "sortable": true,
           "header": true,
           "paging": true,
-          "field_list": true,
+          "field_list": false,
           "all_fields": false,
           "trimFactor": 300,
           "localTime": false,
           "timeField": "@timestamp",
           "spyable": true,
           "queries": {
-            "mode": "all",
+            "mode": "selected",
             "ids": [
-              0,
-              1,
-              2,
-              3,
-              4,
-              5,
-              6
+              1
             ]
           },
-          "locked": true,
           "style": {
             "font-size": "9pt"
           },
           "normTimes": true,
-          "title": "Top Alerts"
+          "title": "Yaf"
+        },
+        {
+          "error": false,
+          "span": 12,
+          "editable": true,
+          "type": "pcap",
+          "loadingEditor": false,
+          "title": "PCAP Data"
         }
       ],
       "notice": false
@@ -450,7 +508,7 @@
   "index": {
     "interval": "none",
     "pattern": "[logstash-]YYYY.MM.DD",
-    "default": "_all",
+    "default": "*alert*,*_index*",
     "warm_fields": false
   },
   "style": "dark",
@@ -465,19 +523,22 @@
       "query": "*",
       "pinned": true,
       "history": [
-        "alerts.triggered.type:warning",
-        "alerts.triggered.type:alert",
-        "",
-        "alerts.triggered.priority:3",
-        "alerts.triggered.priority:2",
-        "alerts.triggered.priority:1",
+        "_type:snort_doc",
+        "_type:bro_alert",
+        "_type:bro_doc",
+        "_type:*_alert",
+        "_type:*_doc",
+        "_type:yaf_doc",
+        "is_alert=true",
+        "_type:sourcefire_doc",
+        "_type:sourcefire_alert",
         "*"
       ],
       "remember": 10
     },
     {
       "type": "filtering",
-      "collapse": false,
+      "collapse": true,
       "notice": true,
       "enable": true
     }
@@ -512,9 +573,9 @@
         "2h",
         "1d"
       ],
-      "timefield": "message.timestamp",
+      "timefield": "timestamp",
       "now": true,
-      "filter_id": 1
+      "filter_id": 0
     }
   ],
   "loader": {

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/c737aa9d/metron-ui/lib/public/app/panels/pcap/module.html
----------------------------------------------------------------------
diff --git a/metron-ui/lib/public/app/panels/pcap/module.html b/metron-ui/lib/public/app/panels/pcap/module.html
index 893f871..17c57e0 100755
--- a/metron-ui/lib/public/app/panels/pcap/module.html
+++ b/metron-ui/lib/public/app/panels/pcap/module.html
@@ -100,108 +100,8 @@
 
 <div class="table-container">
     <div class="pcap-editor" pcap-settings></div>
-    <div class="pcap-editor">
-      <h6>PCAP Search Results</h6>
-      <table class="table table-bordered table-condensed table-details table-striped" ng-style="panel.style">
-      <thead>
-        <tr>
-          <td>PCAP ID</td>
-          <td>Source Port</td>
-          <td>Destination Port</td>
-          <td>Source IP</td>
-          <td>Destination IP</td>
-          <td>Protocol</td>
-        <tr>
-      </thead>
-
-      <tbody bindonce ng-repeat="doc in results.hits.hits">
-          <tr ng-click="get_pcap(doc._source.message.pcap_id)" ng-class="">
-            <td>{{ doc._source.message.pcap_id }}</td>
-            <td>{{ doc._source.message.ip_src_port }}</td>
-            <td>{{ doc._source.message.dst_port }}</td>
-            <td>{{ doc._source.message.ip_src_addr }}</td>
-            <td>{{ doc._source.message.ip_dst_addr }}</td>
-            <td>{{ doc._source.message.ip_protocol }}</td>
-          </tr>
-      </tbody>
-      </table>
-    </div>
-
-    <table class="table table-bordered table-condensed table-details table-striped" ng-style="panel.style">
-    <thead>
-      <tr>
-        <td>Packet No.</td>
-        <td>Source</td>
-        <td>Destination</td>
-        <td>Info</td>
-      <tr>
-    </thead>
-    <tbody bindonce ng-repeat="packet in packet_data.pdml.packet">
-        <tr ng-click="$parent.drilldown = packet; setSelected($index);" ng-class="{highlight : $index === selectedValue}">
-          <td><span>{{$index}}</span></td>
-          <td>{{packet.proto[3].field[10].$.show}}</td>
-          <td>{{packet.proto[3].field[14].$.show}}</td>
-          <td>{{packet.proto[4].field[0].$.showname}}</td>
-        </tr>
-    </tbody>
-    </table>
-
-      <table class="">
-       <thead></thead>
-       <!-- TODO: Fix repeater bug -->
-        <tbody bindonce ng-repeat="event in data| slice:panel.offset:panel.offset+panel.size" ng-class-odd="'odd'" class="extra-row" ng-show="activePosition == $index">
-          <tr bindonce ng-repeat="prot in event.proto" class="main-row">
-            <td><div ng-click="prot.isVisible = !prot.isVisible">{{protShowname(prot.$)}}</div>
-            <div>
-              <ul ng-show="prot.isVisible">
-                <li ng-repeat="fieldval in prot.field" ng-click="fieldBytes(fieldval)">
-                    <p>{{fieldvalShowname(fieldval)}}</p>
-                </li>
-            </ul>
-            </div>
-            </td>
-          </tr>
-       </tbody>
-      </table>
-
-    <div packet="drilldown"></div>
 </div>
 
-  <script type="text/ng-template" id="packet.html">
-    <div ng-repeat="proto in packet.proto">
-      <span ng-class="{'icon-chevron-down': proto.expanded, 'icon-chevron-right': !proto.expanded}" ng-click="proto.expanded = select(proto) || !proto.expanded"></span>
-      <span ng-class="{highlight: (selectedData.uid === proto.$.uid) || (selectedData2 === proto.$.name )}" ng-click="proto.expanded = select(proto) || !proto.expanded">
-        {{ proto.$.showname }}
-      </span>
-
-      <div ng-if="proto.expanded" style="margin-left:50px">
-        <div ng-repeat="f in proto.field">
-          <span ng-class="{highlight: ($parent.selectedData.uid === f.$.uid) || (selectedData2 === f.$)}" ng-click="select_field(f)">
-            {{ f.$.showname }}
-          </span>
-        </div>
-      </div>
-    </div>
-    <div hex-bytes bytes="packet.hexPacket" selected-bytes="selectedBytes" selected-data="selectedData"></div>
-  </script>
-
-  <script type="text/ng-template" id="hexBytes.html">
-    <br/>
-    <span ng-repeat="b in bytes track by $index">
-      <span class="byte" ng-class="{highlight: selectedBytes.pos <= $index && (selectedBytes.pos|num)+(selectedBytes.size|num) > $index, newline:$index%16==0, padright:($index+8)%16==0}" ng-click="selectByte($index)" style="float:left; padding:4px;">
-        {{ b }}
-      </span>
-    </span>
-    <div style="clear:left">
-    <br/>
-    Position: {{selectedBytes.pos}}<br/>
-    Size: {{selectedBytes.size}}<br/>
-    Selected Field: {{selectedBytes.showname}} ({{selectedBytes.name}})</br>
-    selectedBytes :: {{$parent.selectedBytes}}</br>
-    selectedBytes.expand :: {{$parent.selectedBytes.expanded}}
-    </div>
-  </script>
-
   <!-- Filter PCAP Panel -->
   <script type="text/ng-template" id="pcapSettings.html">
     <div class="editor-row">
@@ -221,9 +121,11 @@
           <div class="editor-option">
               <label class="small">Protocol</label><input type="text" class="input-medium" ng-model="ip_protocol"></input>
           </div>
-
+          <div class="editor-option">
+            <label class="small">Include Reverse Traffic</label><input type="checkbox" value="false" class="input-medium" ng-model="include_reverse_traffic"></input>
+          </div>
           <div class="edit-option">
-          <button type="button" class="btn btn-success" ng-click="search()">Search</button>
+            <button type="button" class="btn btn-success" ng-click="getPcap()">Search</button>
           </div>
 
         </form>

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/c737aa9d/metron-ui/lib/public/app/panels/pcap/module.js
----------------------------------------------------------------------
diff --git a/metron-ui/lib/public/app/panels/pcap/module.js b/metron-ui/lib/public/app/panels/pcap/module.js
index 5991444..47c21f2 100755
--- a/metron-ui/lib/public/app/panels/pcap/module.js
+++ b/metron-ui/lib/public/app/panels/pcap/module.js
@@ -168,6 +168,11 @@ function (angular, app, _, require, kbn) {
         });
     }
 
+    $scope.getPcap = function() {
+      var url = '/pcap/getPcapsByIdentifiers?srcIp=' + $scope.ip_src_addr + '&dstIp=' + $scope.ip_dst_addr + '&protocol=' + $scope.ip_protocol + '&srcPort=' + $scope.ip_src_port + '&dstPort=' + $scope.dst_port + '&includeReverseTraffic=' + $scope.include_reverse_traffic;
+      window.location = url;
+    };
+
     // Query for PCAP IDS
     $scope.search = function() {
       var client = $scope.ejs.Request()

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/c737aa9d/metron-ui/lib/views/alerts.jade
----------------------------------------------------------------------
diff --git a/metron-ui/lib/views/alerts.jade b/metron-ui/lib/views/alerts.jade
index 0f75033..758e3b5 100644
--- a/metron-ui/lib/views/alerts.jade
+++ b/metron-ui/lib/views/alerts.jade
@@ -1,21 +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.
- */
- 
+//
+ 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.
+//
+
 doctype html
 html
   head

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/c737aa9d/metron-ui/lib/views/index.jade
----------------------------------------------------------------------
diff --git a/metron-ui/lib/views/index.jade b/metron-ui/lib/views/index.jade
index 67525e5..4246b58 100644
--- a/metron-ui/lib/views/index.jade
+++ b/metron-ui/lib/views/index.jade
@@ -1,20 +1,20 @@
-/**
- * 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.
- */
+//
+ 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.
+//
 
 doctype html
 html

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/c737aa9d/metron-ui/lib/views/login.jade
----------------------------------------------------------------------
diff --git a/metron-ui/lib/views/login.jade b/metron-ui/lib/views/login.jade
index d76ca9b..d2da35c 100644
--- a/metron-ui/lib/views/login.jade
+++ b/metron-ui/lib/views/login.jade
@@ -1,20 +1,20 @@
-/**
- * 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.
- */
+//
+ 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.
+//
 
 doctype html
 html

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/c737aa9d/metron-ui/package.json
----------------------------------------------------------------------
diff --git a/metron-ui/package.json b/metron-ui/package.json
index c904718..d142489 100644
--- a/metron-ui/package.json
+++ b/metron-ui/package.json
@@ -1,62 +1,26 @@
 {
   "name": "metron-ui",
   "version": "0.0.0",
-  "description": "Metron Portal",
-  "main": "index.js",
   "private": true,
   "scripts": {
-    "test": "make test",
-    "seed": "make seed"
+    "start": "node ./bin/www"
   },
-  "repository": {
-    "type": "git",
-    "url": "git://github.com/apache/incubator-metron.git"
-  },
-  "author": "",
-  "license": "Apache-2.0",
-  "bugs": {
-    "url": "https://issues.apache.org/jira/browse/METRON"
-  },
-  "homepage": "http://metron.incubator.apache.org",
   "dependencies": {
-    "async": "^0.9.0",
-    "bcrypt": "^0.7.8",
-    "bluebird": "^1.2.4",
-    "checkit": "^0.2.0-pre",
-    "connect": "^2.16.2",
-    "connect-flash": "^0.1.1",
-    "connect-redis": "^2.0.0",
-    "express": "^4.2.0",
-    "jade": "^1.3.1",
-    "kappa": "^0.14.3",
-    "lodash": "^2.4.1",
-    "lusca": "^1.0.0",
-    "passport": "^0.2.0",
-    "passport-ldapauth": "^0.2.0",
-    "passport-local": "^1.0.0",
-    "redis": "^0.10.2",
-    "ws": "^0.4.31",
-    "xxhashjs": "0.0.5",
-    "http-proxy": "1.3.0",
-    "cookie-parser": "~1.3.2",
-    "body-parser": "~1.6.5",
-    "cookie-session": "~1.0.2",
-    "xml-stream": "~0.4.4"
-  },
-  "devDependencies": {
-    "chai": "^1.9.1",
-    "chance": "^0.5.6",
-    "glob": "^3.2.9",
-    "grunt": "^0.4.4",
-    "grunt-bowercopy": "^1.0.0",
-    "istanbul": "^0.2.7",
-    "karma": "^0.12.14",
-    "karma-coverage": "^0.2.1",
-    "karma-mocha": "^0.1.3",
-    "karma-osx-reporter": "0.0.4",
-    "karma-sinon-chai": "^0.1.5",
-    "mocha": "^1.19.0",
-    "sinon": "^1.9.1",
-    "supertest": "^0.11.0"
+    "body-parser": "~1.13.2",
+    "cookie-parser": "~1.3.5",
+    "debug": "~2.2.0",
+    "express": "~4.13.1",
+    "jade": "~1.11.0",
+    "morgan": "~1.6.1",
+    "serve-favicon": "~2.3.0",
+    "lodash": "~4.6.1",
+    "connect": "3.4.1",
+    "connect-flash": "~0.1.1",
+    "cookie-session": "~2.0.0-alpha.1",
+    "passport": "~0.3.2",
+    "passport-ldapauth": "~0.5.0",
+    "http-proxy": "~1.13.2",
+    "xml-stream": "~0.4.5",
+    "serve-static": "~1.10.2"
   }
-}
+}
\ No newline at end of file



[3/3] incubator-metron git commit: METRON-76-Deployment Metron deploy has intermittent failures on EC2 closes apache/incubator-metron#51

Posted by js...@apache.org.
METRON-76-Deployment Metron deploy has intermittent failures on EC2 closes apache/incubator-metron#51


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

Branch: refs/heads/master
Commit: 68aab6e9adc42b3a21701d1fca2c1a3a81e14237
Parents: c737aa9
Author: dlyle65535 <dl...@gmail.com>
Authored: Sat Mar 19 14:11:38 2016 -0700
Committer: james-sirota <Ja...@yahoo.com>
Committed: Sat Mar 19 14:11:38 2016 -0700

----------------------------------------------------------------------
 deployment/amazon-ec2/playbook.yml              | 16 ++++++-
 deployment/amazon-ec2/tasks/check-hosts.yml     | 20 ++++++++
 deployment/amazon-ec2/tasks/create-hosts.yml    |  4 --
 deployment/amazon-ec2/tasks/expand-volume.yml   |  4 +-
 deployment/amazon-ec2/tasks/mount-volume.yml    |  9 +++-
 .../inventory/singlenode-vagrant/group_vars/all | 11 +++--
 deployment/playbooks/ambari_install.yml         |  1 +
 deployment/playbooks/metron_install.yml         |  4 --
 deployment/roles/ambari_common/meta/main.yml    | 22 +++++++++
 deployment/roles/ambari_common/tasks/main.yml   | 20 --------
 deployment/roles/ambari_config/meta/main.yml    | 21 +++++++++
 deployment/roles/ambari_config/tasks/main.yml   | 10 +++-
 .../ambari_config/tasks/start_services.yml      | 48 ++++++++++++++++++++
 .../vars/multi_vagrant_cluster.yml              |  2 +
 .../roles/ambari_config/vars/single_node_vm.yml |  1 +
 .../roles/ambari_config/vars/small_cluster.yml  |  2 +
 .../roles/ambari_gather_facts/meta/main.yml     | 21 +++++++++
 .../roles/ambari_gather_facts/tasks/main.yml    | 10 ----
 deployment/roles/ambari_master/tasks/main.yml   | 29 ++++++++----
 deployment/roles/ambari_slave/tasks/main.yml    |  8 +++-
 deployment/roles/bro/meta/main.yml              | 21 +++++++++
 deployment/roles/bro/tasks/dependencies.yml     |  5 +-
 deployment/roles/build-tools/meta/main.yml      | 19 ++++++++
 deployment/roles/build-tools/tasks/main.yml     | 34 ++++++++++++++
 .../elasticsearch/files/yaf_index.template      | 36 +++++++++++++++
 .../roles/elasticsearch/handlers/main.yml       | 19 --------
 deployment/roles/elasticsearch/meta/main.yml    | 24 ++++++++++
 .../elasticsearch/tasks/configure_index.yml     | 44 ++++++++++++++++++
 deployment/roles/elasticsearch/tasks/main.yml   | 17 +++----
 deployment/roles/epel/tasks/main.yml            | 30 ++++++++++++
 deployment/roles/flume/meta/main.yml            | 20 ++++++++
 deployment/roles/flume/tasks/main.yml           |  5 +-
 deployment/roles/hadoop_setup/meta/main.yml     | 20 ++++++++
 deployment/roles/httplib2/tasks/main.yml        | 20 ++++++++
 deployment/roles/java_jdk/defaults/main.yml     | 18 ++++++++
 deployment/roles/java_jdk/tasks/main.yml        | 34 ++++++++++++++
 .../roles/libselinux-python/tasks/main.yml      | 25 ++++++++++
 deployment/roles/metron_common/meta/main.yml    | 22 +++++++++
 deployment/roles/metron_common/tasks/main.yml   | 20 ++------
 .../roles/metron_pcapservice/meta/main.yml      | 19 ++++++++
 .../metron_pcapservice/tasks/config-java.yml    | 34 --------------
 .../roles/metron_pcapservice/tasks/main.yml     |  2 -
 deployment/roles/metron_streaming/meta/main.yml | 20 ++++++++
 .../metron_streaming/tasks/grok_upload.yml      | 37 +++++++++++++++
 .../metron_streaming/tasks/hdfs_filesystem.yml  | 41 +++++++++++++++++
 .../roles/metron_streaming/tasks/main.yml       | 27 +++--------
 deployment/roles/metron_ui/tasks/main.yml       |  9 +++-
 deployment/roles/mysql_server/tasks/main.yml    |  8 ++++
 deployment/roles/ntp/tasks/main.yml             | 31 +++++++++++++
 deployment/roles/pcap_replay/meta/main.yml      | 19 ++------
 deployment/roles/pcap_replay/tasks/main.yml     |  3 --
 .../roles/pcap_replay/tasks/tcpreplay.yml       |  7 ---
 deployment/roles/pycapa/meta/main.yml           | 21 +++++++++
 deployment/roles/pycapa/tasks/main.yml          | 22 +++++----
 deployment/roles/python-pip/tasks/main.yml      | 25 ++++++++++
 deployment/roles/snort/meta/main.yml            | 22 +++++++++
 deployment/roles/snort/tasks/daq.yml            |  8 +++-
 deployment/roles/snort/tasks/main.yml           | 16 -------
 deployment/roles/snort/tasks/snort.yml          | 28 +++++-------
 deployment/roles/tap_interface/tasks/main.yml   |  9 +++-
 deployment/roles/yaf/meta/main.yml              | 22 +++++++++
 deployment/roles/yaf/tasks/dependencies.yml     | 31 -------------
 deployment/roles/yaf/tasks/kafka-client.yml     |  9 ++--
 deployment/roles/yaf/tasks/main.yml             |  1 -
 deployment/roles/yum-update/tasks/main.yml      | 26 +++++++++++
 65 files changed, 926 insertions(+), 267 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/68aab6e9/deployment/amazon-ec2/playbook.yml
----------------------------------------------------------------------
diff --git a/deployment/amazon-ec2/playbook.yml b/deployment/amazon-ec2/playbook.yml
index d76ac4d..5914b98 100644
--- a/deployment/amazon-ec2/playbook.yml
+++ b/deployment/amazon-ec2/playbook.yml
@@ -27,15 +27,27 @@
     - include: tasks/create-open-inbound-security-group.yml
     - include: tasks/create-open-outbound-security-group.yml
     - include: tasks/create-hosts.yml host_count=1 host_type=sensors,ambari_master,metron,ec2
-    - include: tasks/create-hosts.yml host_count=5 host_type=ambari_slave,ec2
+    - include: tasks/create-hosts.yml host_count=4 host_type=ambari_slave,ec2
     - include: tasks/create-hosts.yml host_count=1 host_type=ambari_slave,hadoop_client,metron,ec2
     - include: tasks/create-hosts.yml host_count=1 host_type=ambari_slave,enrichment,metron,ec2
-    - include: tasks/create-hosts.yml host_count=3 host_type=search,metron,ec2
+    - include: tasks/create-hosts.yml host_count=2 host_type=search,metron,ec2
     - include: tasks/create-hosts.yml host_count=1 host_type=web,mysql,metron,ec2
   tags:
     - ec2
 
 #
+# wait for all ec2 hosts to come up
+#
+- hosts: ec2
+  become: True
+  vars_files:
+    - conf/defaults.yml
+  gather_facts: False
+  tasks:
+    - include: tasks/check-hosts.yml
+  tags:
+    - ec2
+#
 # mount additional data volumes on all ec2 hosts
 #
 - hosts: ec2

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/68aab6e9/deployment/amazon-ec2/tasks/check-hosts.yml
----------------------------------------------------------------------
diff --git a/deployment/amazon-ec2/tasks/check-hosts.yml b/deployment/amazon-ec2/tasks/check-hosts.yml
new file mode 100644
index 0000000..1a4b2c7
--- /dev/null
+++ b/deployment/amazon-ec2/tasks/check-hosts.yml
@@ -0,0 +1,20 @@
+#
+#  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: Wait for connectivity to host(s)
+  local_action: wait_for host={{ inventory_hostname }} state=started timeout=300 delay=10
+  become: False

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/68aab6e9/deployment/amazon-ec2/tasks/create-hosts.yml
----------------------------------------------------------------------
diff --git a/deployment/amazon-ec2/tasks/create-hosts.yml b/deployment/amazon-ec2/tasks/create-hosts.yml
index bea6b24..e1ff3e9 100644
--- a/deployment/amazon-ec2/tasks/create-hosts.yml
+++ b/deployment/amazon-ec2/tasks/create-hosts.yml
@@ -53,7 +53,3 @@
   with_items: ec2.tagged_instances
   when: item.public_dns_name is defined
 
-- name: Wait for connectivity to host(s)
-  wait_for: host={{ item.public_dns_name }} port=22 search_regex=OpenSSH
-  with_items: ec2.tagged_instances
-  when: item.public_dns_name is defined

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/68aab6e9/deployment/amazon-ec2/tasks/expand-volume.yml
----------------------------------------------------------------------
diff --git a/deployment/amazon-ec2/tasks/expand-volume.yml b/deployment/amazon-ec2/tasks/expand-volume.yml
index 41cd442..1e25e27 100644
--- a/deployment/amazon-ec2/tasks/expand-volume.yml
+++ b/deployment/amazon-ec2/tasks/expand-volume.yml
@@ -27,6 +27,4 @@
   poll: 0
   ignore_errors: True
 
-- name: Wait for connectivity to host(s)
-  local_action: wait_for host={{ inventory_hostname }} state=started
-  become: False
+- include: tasks/check-hosts.yml

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/68aab6e9/deployment/amazon-ec2/tasks/mount-volume.yml
----------------------------------------------------------------------
diff --git a/deployment/amazon-ec2/tasks/mount-volume.yml b/deployment/amazon-ec2/tasks/mount-volume.yml
index 9609816..11259a5 100644
--- a/deployment/amazon-ec2/tasks/mount-volume.yml
+++ b/deployment/amazon-ec2/tasks/mount-volume.yml
@@ -16,7 +16,14 @@
 #
 ---
 - name: Install xfsprogs
-  yum: name=xfsprogs
+  yum:
+    name: xfsprogs
+    state: present
+    update_cache: yes
+  register: result
+  until: result.rc == 0
+  retries: 5
+  delay: 10
 
 - name: Format data volume(s)
   filesystem: fstype=xfs dev={{ vol_src }}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/68aab6e9/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 06acaa9..a5e0af7 100644
--- a/deployment/inventory/singlenode-vagrant/group_vars/all
+++ b/deployment/inventory/singlenode-vagrant/group_vars/all
@@ -28,11 +28,6 @@ pcap_hbase_table: pcap
 tracker_hbase_table: access_tracker
 threatintel_ip_hbase_table: malicious_ip
 
-#elasticsearch
-elasticsearch_transport_port: 9300
-elasticsearch_network_interface: eth1
-elasticsearch_web_port: 9200
-
 # metron variables
 metron_version: 0.1BETA
 metron_directory: /usr/metron/{{ metron_version }}
@@ -78,3 +73,9 @@ storm_topologies:
     - "{{ metron_directory }}/config/topologies/snort/remote.yaml"
     - "{{ metron_directory }}/config/topologies/enrichment/remote.yaml"
 pcapservice_port: 8081
+
+#Search
+install_elasticsearch: True
+elasticsearch_transport_port: 9300
+elasticsearch_network_interface: eth1
+elasticsearch_web_port: 9200

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/68aab6e9/deployment/playbooks/ambari_install.yml
----------------------------------------------------------------------
diff --git a/deployment/playbooks/ambari_install.yml b/deployment/playbooks/ambari_install.yml
index e6d226b..685753c 100644
--- a/deployment/playbooks/ambari_install.yml
+++ b/deployment/playbooks/ambari_install.yml
@@ -47,6 +47,7 @@
     - hdp-install
 
 - hosts: ambari_master
+  become: true
   roles:
     - role: ambari_config
   tags:

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/68aab6e9/deployment/playbooks/metron_install.yml
----------------------------------------------------------------------
diff --git a/deployment/playbooks/metron_install.yml b/deployment/playbooks/metron_install.yml
index bfdde54..c8c181e 100644
--- a/deployment/playbooks/metron_install.yml
+++ b/deployment/playbooks/metron_install.yml
@@ -32,7 +32,6 @@
 - hosts: hadoop_client
   become: true
   roles:
-    - role: ambari_gather_facts
     - role: hadoop_setup
   tags:
     - metron-prereqs
@@ -63,7 +62,6 @@
 - hosts: sensors
   become: true
   roles:
-    - role: ambari_gather_facts
     - { role: tap_interface, when: install_tap | default(False) == True }
     - { role: pycapa, when: install_pycapa | default(True) == True }
     - { role: bro, when: install_bro | default(True) == True }
@@ -77,7 +75,6 @@
 - hosts: enrichment
   become: true
   roles:
-    - role: ambari_gather_facts
     - role: metron_streaming
   tags:
     - enrichment
@@ -85,7 +82,6 @@
 - hosts: web
   become: true
   roles:
-    - role: ambari_gather_facts
     - role: metron_ui
     - role: metron_pcapservice
   tags:

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/68aab6e9/deployment/roles/ambari_common/meta/main.yml
----------------------------------------------------------------------
diff --git a/deployment/roles/ambari_common/meta/main.yml b/deployment/roles/ambari_common/meta/main.yml
new file mode 100644
index 0000000..8992ac1
--- /dev/null
+++ b/deployment/roles/ambari_common/meta/main.yml
@@ -0,0 +1,22 @@
+#
+#  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:
+  - libselinux-python
+  - yum-update
+  - epel
+  - ntp

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/68aab6e9/deployment/roles/ambari_common/tasks/main.yml
----------------------------------------------------------------------
diff --git a/deployment/roles/ambari_common/tasks/main.yml b/deployment/roles/ambari_common/tasks/main.yml
index 35f3fce..b1b1659 100644
--- a/deployment/roles/ambari_common/tasks/main.yml
+++ b/deployment/roles/ambari_common/tasks/main.yml
@@ -19,20 +19,8 @@
   fail: msg="Ambari HDP deployment supports CentOS 6 only."
   when: (ansible_distribution != "CentOS" or ansible_distribution_major_version != "6")
 
-- name: Install libselinux-python
-  yum: name=libselinux-python
-
 - include: passwd_less_ssh.yml
 
-- name: upgrade all packages
-  yum: name=* state=latest
-
-- name: install ntp
-  yum: name=ntp state=present
-
-- name: Ensure ntp is running and enabled
-  service: name=ntpd state=started enabled=yes
-
 - name: Ensure iptables is stopped and is not running at boot time.
   ignore_errors: yes
   service: name=iptables state=stopped enabled=no
@@ -53,14 +41,6 @@
 - name: Add localhost to /etc/hosts
   lineinfile: dest=/etc/hosts line="127.0.0.1   localhost"
 
-- name: get epel-repo rpm
-  get_url: dest=/tmp/epel-release.rpm  url=http://dl.fedoraproject.org/pub/epel/6/x86_64/epel-release-6-8.noarch.rpm
-
-- name: install epel-repo rpm
-  yum: pkg=/tmp/epel-release.rpm state=installed
-
 - name: Download Ambari repo
   get_url: url="{{ rhel_ambari_install_url }}" dest=/etc/yum.repos.d/ambari.repo
 
-- name: Clean yum
-  shell: yum clean all

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/68aab6e9/deployment/roles/ambari_config/meta/main.yml
----------------------------------------------------------------------
diff --git a/deployment/roles/ambari_config/meta/main.yml b/deployment/roles/ambari_config/meta/main.yml
new file mode 100644
index 0000000..61197e3
--- /dev/null
+++ b/deployment/roles/ambari_config/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:
+  - epel
+  - python-pip
+  - httplib2

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/68aab6e9/deployment/roles/ambari_config/tasks/main.yml
----------------------------------------------------------------------
diff --git a/deployment/roles/ambari_config/tasks/main.yml b/deployment/roles/ambari_config/tasks/main.yml
index f98c33b..f44f929 100644
--- a/deployment/roles/ambari_config/tasks/main.yml
+++ b/deployment/roles/ambari_config/tasks/main.yml
@@ -18,7 +18,13 @@
 - include_vars: "{{ cluster_type }}.yml"
 
 - name: Install python-requests
-  yum: name=python-requests
+  yum:
+    name: python-requests
+    state: installed
+  register: result
+  until: result.rc == 0
+  retries: 5
+  delay: 10
 
 - name: Deploy cluster with Ambari; http://{{ groups.ambari_master[0] }}:{{ ambari_port }}
   ambari_cluster_state:
@@ -32,3 +38,5 @@
     configurations: "{{ configurations }}"
     wait_for_complete: True
     blueprint_var: "{{ blueprint }}"
+
+- include: start_services.yml

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/68aab6e9/deployment/roles/ambari_config/tasks/start_services.yml
----------------------------------------------------------------------
diff --git a/deployment/roles/ambari_config/tasks/start_services.yml b/deployment/roles/ambari_config/tasks/start_services.yml
new file mode 100644
index 0000000..7c6e0a9
--- /dev/null
+++ b/deployment/roles/ambari_config/tasks/start_services.yml
@@ -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.
+#
+---
+- name: Start All Hadoop Services {{ inventory_hostname }}
+  uri:
+    url: http://{{ inventory_hostname}}:{{ ambari_port}}/api/v1/clusters/{{ cluster_name }}/services/{{ item }}
+    HEADER_X-Requested-By: "{{ ambari_user }}"
+    method: PUT
+    body: "{ \"RequestInfo\": { \"context\": \"Start service via REST\" }, \"Body\": { \"ServiceInfo\": { \"state\": \"STARTED\" }}}"
+    body_format: json
+    status_code: 200,202
+    force_basic_auth: yes
+    user: "{{ ambari_user }}"
+    password: "{{ ambari_password }}"
+  with_items:
+    - "{{ metron_services }}"
+
+- name: Wait for Service Start
+  uri:
+    url: http://{{ inventory_hostname}}:{{ ambari_port}}/api/v1/clusters/{{ cluster_name }}/services/{{ item }}
+    HEADER_X-Requested-By: "{{ ambari_user }}"
+    method: GET
+    status_code: 200
+    force_basic_auth: yes
+    user: "{{ ambari_user }}"
+    password: "{{ ambari_password }}"
+    return_content: yes
+  with_items:
+    - "{{ metron_services }}"
+  register: result
+  until: result.content.find("STARTED") != -1
+  retries: 10
+  delay: 60
+

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/68aab6e9/deployment/roles/ambari_config/vars/multi_vagrant_cluster.yml
----------------------------------------------------------------------
diff --git a/deployment/roles/ambari_config/vars/multi_vagrant_cluster.yml b/deployment/roles/ambari_config/vars/multi_vagrant_cluster.yml
index 574268f..526661b 100644
--- a/deployment/roles/ambari_config/vars/multi_vagrant_cluster.yml
+++ b/deployment/roles/ambari_config/vars/multi_vagrant_cluster.yml
@@ -27,6 +27,8 @@ hbase_master: [HBASE_MASTER]
 hbase_slave: [HBASE_REGIONSERVER]
 hadoop_clients: [HDFS_CLIENT, YARN_CLIENT, MAPREDUCE2_CLIENT, SPARK_CLIENT, ZOOKEEPER_CLIENT, HBASE_CLIENT]
 
+metron_services: ["HDFS","YARN","MAPREDUCE2","ZOOKEEPER", "HBASE", "STORM", "KAFKA"]
+
 master_1_components: "{{ hadoop_master | union(hadoop_clients) }}"
 master_1_host:
   - "{{groups.ambari_slave[0]}}"

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/68aab6e9/deployment/roles/ambari_config/vars/single_node_vm.yml
----------------------------------------------------------------------
diff --git a/deployment/roles/ambari_config/vars/single_node_vm.yml b/deployment/roles/ambari_config/vars/single_node_vm.yml
index 6b18825..cb6fe4a 100644
--- a/deployment/roles/ambari_config/vars/single_node_vm.yml
+++ b/deployment/roles/ambari_config/vars/single_node_vm.yml
@@ -29,6 +29,7 @@ zookeeper_slave: [ZOOKEEPER_CLIENT]
 hbase_master: [HBASE_MASTER, HBASE_CLIENT]
 hbase_slave: [HBASE_REGIONSERVER]
 
+metron_services: ["HDFS","YARN","MAPREDUCE2","ZOOKEEPER", "HBASE", "STORM", "KAFKA"]
 metron_components: "{{ hadoop_master | union(zookeeper_master) | union(storm_master) | union(hbase_master) | union(hadoop_slave) | union(zookeeper_slave) | union(storm_slave) | union(kafka_broker) | union(hbase_slave) }}"
 
 cluster_name: "metron_cluster"

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/68aab6e9/deployment/roles/ambari_config/vars/small_cluster.yml
----------------------------------------------------------------------
diff --git a/deployment/roles/ambari_config/vars/small_cluster.yml b/deployment/roles/ambari_config/vars/small_cluster.yml
index ff985dc..a3792e1 100644
--- a/deployment/roles/ambari_config/vars/small_cluster.yml
+++ b/deployment/roles/ambari_config/vars/small_cluster.yml
@@ -27,6 +27,8 @@ hbase_master: [HBASE_MASTER]
 hbase_slave: [HBASE_REGIONSERVER]
 hadoop_clients: [HDFS_CLIENT, YARN_CLIENT, MAPREDUCE2_CLIENT, SPARK_CLIENT, ZOOKEEPER_CLIENT, HBASE_CLIENT]
 
+metron_services: ["HDFS","YARN","MAPREDUCE2","ZOOKEEPER", "HBASE", "STORM", "KAFKA"]
+
 master_1_components: "{{ hadoop_master | union(hadoop_clients) }}"
 master_1_host:
   - "{{groups.ambari_slave[0]}}"

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/68aab6e9/deployment/roles/ambari_gather_facts/meta/main.yml
----------------------------------------------------------------------
diff --git a/deployment/roles/ambari_gather_facts/meta/main.yml b/deployment/roles/ambari_gather_facts/meta/main.yml
new file mode 100644
index 0000000..61197e3
--- /dev/null
+++ b/deployment/roles/ambari_gather_facts/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:
+  - epel
+  - python-pip
+  - httplib2

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/68aab6e9/deployment/roles/ambari_gather_facts/tasks/main.yml
----------------------------------------------------------------------
diff --git a/deployment/roles/ambari_gather_facts/tasks/main.yml b/deployment/roles/ambari_gather_facts/tasks/main.yml
index 8ec7e40..db4927d 100644
--- a/deployment/roles/ambari_gather_facts/tasks/main.yml
+++ b/deployment/roles/ambari_gather_facts/tasks/main.yml
@@ -15,16 +15,6 @@
 #  limitations under the License.
 #
 ---
-- name: Install EPEL repository
-  yum: name=epel-release
-
-- name: Install pip
-  yum: name=python-pip
-
-- name: Install python httplib2 dependency
-  pip:
-    name: httplib2
-
 - name: Ambari rest get cluster name
   uri:
     url: "http://{{ groups.ambari_master[0] }}:{{ ambari_port }}/api/v1/clusters"

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/68aab6e9/deployment/roles/ambari_master/tasks/main.yml
----------------------------------------------------------------------
diff --git a/deployment/roles/ambari_master/tasks/main.yml b/deployment/roles/ambari_master/tasks/main.yml
index 021e377..25c3784 100644
--- a/deployment/roles/ambari_master/tasks/main.yml
+++ b/deployment/roles/ambari_master/tasks/main.yml
@@ -17,12 +17,14 @@
 ---
 # tasks file for ambari_master
 - name: Install ambari server
-  yum: name=ambari-server state=present
-
-- name: Setup ambari server
-  shell: ambari-server setup -s; touch /etc/ambari-server/configured creates=/etc/ambari-server/configured
-  register: ambari_server_setup
-  failed_when: ambari_server_setup.stderr
+  yum:
+    name: ambari-server
+    state: present
+    update_cache: yes
+  register: result
+  until: result.rc == 0
+  retries: 5
+  delay: 10
 
 - name: Set Ambari Server Max Memory
   replace:
@@ -31,8 +33,19 @@
     replace: " -Xmx{{ ambari_server_mem }}m "
     backup: no
 
+- name: Setup ambari server
+  shell: ambari-server setup -s && touch /etc/ambari-server/configured creates=/etc/ambari-server/configured
+  register: ambari_server_setup
+  failed_when: ambari_server_setup.stderr
+
 - name: start ambari server
-  service: name=ambari-server state=restarted
+  service:
+    name: ambari-server
+    state: restarted
 
 - name : check if ambari-server is up on {{ ambari_host }}:{{ambari_port}}
-  wait_for : host={{ ambari_host }} port={{ ambari_port }} delay=10 timeout=120
+  wait_for :
+    host: "{{ ambari_host }}"
+    port: "{{ ambari_port }}"
+    delay: 120
+    timeout: 300

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/68aab6e9/deployment/roles/ambari_slave/tasks/main.yml
----------------------------------------------------------------------
diff --git a/deployment/roles/ambari_slave/tasks/main.yml b/deployment/roles/ambari_slave/tasks/main.yml
index eac0610..fc068da 100644
--- a/deployment/roles/ambari_slave/tasks/main.yml
+++ b/deployment/roles/ambari_slave/tasks/main.yml
@@ -17,7 +17,13 @@
 ---
 # tasks file for ambari_slave
 - name: Install ambari-agent
-  yum: name=ambari-agent
+  yum:
+    name: ambari-agent
+    state: installed
+  register: result
+  until: result.rc == 0
+  retries: 5
+  delay: 10
 
 - name: Create ambari-agent hostname script
   template:

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/68aab6e9/deployment/roles/bro/meta/main.yml
----------------------------------------------------------------------
diff --git a/deployment/roles/bro/meta/main.yml b/deployment/roles/bro/meta/main.yml
new file mode 100644
index 0000000..4d10740
--- /dev/null
+++ b/deployment/roles/bro/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
+  - libselinux-python
+  - build-tools

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/68aab6e9/deployment/roles/bro/tasks/dependencies.yml
----------------------------------------------------------------------
diff --git a/deployment/roles/bro/tasks/dependencies.yml b/deployment/roles/bro/tasks/dependencies.yml
index 17e84df..431e861 100644
--- a/deployment/roles/bro/tasks/dependencies.yml
+++ b/deployment/roles/bro/tasks/dependencies.yml
@@ -18,7 +18,6 @@
 - name: Install prerequisites
   yum: name={{ item }}
   with_items:
-    - libselinux-python
     - cmake
     - make
     - gcc
@@ -32,3 +31,7 @@
     - swig
     - zlib-devel
     - perl
+  register: result
+  until: result.rc == 0
+  retries: 5
+  delay: 10

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/68aab6e9/deployment/roles/build-tools/meta/main.yml
----------------------------------------------------------------------
diff --git a/deployment/roles/build-tools/meta/main.yml b/deployment/roles/build-tools/meta/main.yml
new file mode 100644
index 0000000..ddf6aa9
--- /dev/null
+++ b/deployment/roles/build-tools/meta/main.yml
@@ -0,0 +1,19 @@
+#
+#  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:
+  - java_jdk

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/68aab6e9/deployment/roles/build-tools/tasks/main.yml
----------------------------------------------------------------------
diff --git a/deployment/roles/build-tools/tasks/main.yml b/deployment/roles/build-tools/tasks/main.yml
new file mode 100644
index 0000000..c47ef43
--- /dev/null
+++ b/deployment/roles/build-tools/tasks/main.yml
@@ -0,0 +1,34 @@
+#
+#  Licensed to the Apache Software Foundation (ASF) under one or more
+#  contributor license agreements.  See the NOTICE file distributed with
+#  this work for additional information regarding copyright ownership.
+#  The ASF licenses this file to You under the Apache License, Version 2.0
+#  (the "License"); you may not use this file except in compliance with
+#  the License.  You may obtain a copy of the License at
+#
+#      http://www.apache.org/licenses/LICENSE-2.0
+#
+#  Unless required by applicable law or agreed to in writing, software
+#  distributed under the License is distributed on an "AS IS" BASIS,
+#  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+#  See the License for the specific language governing permissions and
+#  limitations under the License.
+#
+---
+- name: Install Build Tools
+  yum: name={{ item }}
+  with_items:
+    - "@Development tools"
+    - libdnet-devel
+    - rpm-build
+    - libpcap
+    - libpcap-devel
+    - pcre
+    - pcre-devel
+    - zlib
+    - zlib-devel
+    - glib2-devel
+  register: result
+  until: result.rc == 0
+  retries: 5
+  delay: 10
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/68aab6e9/deployment/roles/elasticsearch/files/yaf_index.template
----------------------------------------------------------------------
diff --git a/deployment/roles/elasticsearch/files/yaf_index.template b/deployment/roles/elasticsearch/files/yaf_index.template
new file mode 100644
index 0000000..c8c1702
--- /dev/null
+++ b/deployment/roles/elasticsearch/files/yaf_index.template
@@ -0,0 +1,36 @@
+{
+    "template" : "yaf_index*",
+    "mappings" : {
+        "yaf_doc" : {
+            "properties": {
+                "message": {
+                    "properties": {
+                        "@timestamp":{"type":"date","format":"dateOptionalTime"},
+                        "end-time":{"type":"string"},
+                        "duration":{"type":"string"},
+                        "rtt":{"type":"string"},
+                        "proto":{"type":"string"},
+                        "sip":{"type":"string"},
+                        "sp":{"type":"string"},
+                        "dip":{"type":"string"},
+                        "dp":{"type":"string"},
+                        "iflags":{"type":"string"},
+                        "uflags":{"type":"string"},
+                        "riflags":{"type":"string"},
+                        "ruflags":{"type":"string"},
+                        "isn":{"type":"string"},
+                        "risn":{"type":"string"},
+                        "tag":{"type":"string"},
+                        "rtag":{"type":"string"},
+                        "pkt":{"type":"string"},
+                        "oct":{"type":"string"},
+                        "rpkt":{"type":"string"},
+                        "roct":{"type":"string"},
+                        "app":{"type":"string"},
+                        "end-reason":{"type":"string"}
+                    }
+                }
+            }
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/68aab6e9/deployment/roles/elasticsearch/handlers/main.yml
----------------------------------------------------------------------
diff --git a/deployment/roles/elasticsearch/handlers/main.yml b/deployment/roles/elasticsearch/handlers/main.yml
deleted file mode 100644
index 112c5ca..0000000
--- a/deployment/roles/elasticsearch/handlers/main.yml
+++ /dev/null
@@ -1,19 +0,0 @@
-#
-#  Licensed to the Apache Software Foundation (ASF) under one or more
-#  contributor license agreements.  See the NOTICE file distributed with
-#  this work for additional information regarding copyright ownership.
-#  The ASF licenses this file to You under the Apache License, Version 2.0
-#  (the "License"); you may not use this file except in compliance with
-#  the License.  You may obtain a copy of the License at
-#
-#      http://www.apache.org/licenses/LICENSE-2.0
-#
-#  Unless required by applicable law or agreed to in writing, software
-#  distributed under the License is distributed on an "AS IS" BASIS,
-#  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-#  See the License for the specific language governing permissions and
-#  limitations under the License.
-#
----
-- name: restart elasticsearch
-  service: name=elasticsearch state=restarted

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/68aab6e9/deployment/roles/elasticsearch/meta/main.yml
----------------------------------------------------------------------
diff --git a/deployment/roles/elasticsearch/meta/main.yml b/deployment/roles/elasticsearch/meta/main.yml
new file mode 100644
index 0000000..f5f059a
--- /dev/null
+++ b/deployment/roles/elasticsearch/meta/main.yml
@@ -0,0 +1,24 @@
+#
+#  Licensed to the Apache Software Foundation (ASF) under one or more
+#  contributor license agreements.  See the NOTICE file distributed with
+#  this work for additional information regarding copyright ownership.
+#  The ASF licenses this file to You under the Apache License, Version 2.0
+#  (the "License"); you may not use this file except in compliance with
+#  the License.  You may obtain a copy of the License at
+#
+#      http://www.apache.org/licenses/LICENSE-2.0
+#
+#  Unless required by applicable law or agreed to in writing, software
+#  distributed under the License is distributed on an "AS IS" BASIS,
+#  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+#  See the License for the specific language governing permissions and
+#  limitations under the License.
+#
+---
+dependencies:
+  - java_jdk
+  - epel
+  - python-pip
+  - httplib2
+  - libselinux-python
+

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/68aab6e9/deployment/roles/elasticsearch/tasks/configure_index.yml
----------------------------------------------------------------------
diff --git a/deployment/roles/elasticsearch/tasks/configure_index.yml b/deployment/roles/elasticsearch/tasks/configure_index.yml
new file mode 100644
index 0000000..09739be
--- /dev/null
+++ b/deployment/roles/elasticsearch/tasks/configure_index.yml
@@ -0,0 +1,44 @@
+#
+#  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 : Wait for Elasticsearch Host to Start
+  wait_for:
+    host: "{{ groups.search[0] }}"
+    port: "{{ elasticsearch_web_port }}"
+    delay: 10
+    timeout: 300
+
+- name: Wait for Green Index Status
+  uri:
+    url: "http://{{ groups.search[0] }}:{{ elasticsearch_web_port }}/_cat/health"
+    method: GET
+    status_code: 200
+    return_content: yes
+  register: result
+  until: result.content.find("green") != -1
+  retries: 10
+  delay: 60
+  run_once: yes
+
+- name: Add Elasticsearch templates for topologies
+  uri:
+    url: "http://{{ groups.search[0] }}:{{ elasticsearch_web_port }}/_template/template_yaf"
+    method: POST
+    body: "{{ lookup('file','yaf_index.template') }}"
+    status_code: 200
+    body_format: json
+  run_once: yes

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/68aab6e9/deployment/roles/elasticsearch/tasks/main.yml
----------------------------------------------------------------------
diff --git a/deployment/roles/elasticsearch/tasks/main.yml b/deployment/roles/elasticsearch/tasks/main.yml
index fa8d4f3..d9b2f44 100644
--- a/deployment/roles/elasticsearch/tasks/main.yml
+++ b/deployment/roles/elasticsearch/tasks/main.yml
@@ -15,12 +15,6 @@
 #  limitations under the License.
 #
 ---
-- name: Install libselinux-python and java
-  yum: name={{item}}
-  with_items:
-    - libselinux-python
-    - java-1.8.0-openjdk
-
 - name: Add Elasticsearch GPG key.
   rpm_key:
     key: https://packages.elastic.co/GPG-KEY-elasticsearch
@@ -33,7 +27,13 @@
     mode: 0644
 
 - name: Install Elasticsearch.
-  yum: pkg=elasticsearch state=installed
+  yum:
+    name: elasticsearch
+    state: installed
+  register: result
+  until: result.rc == 0
+  retries: 5
+  delay: 10
 
 - name: Create Data Directories
   file:
@@ -59,7 +59,8 @@
     - { regexp: '#discovery\.zen\.ping\.unicast\.hosts',
     line: 'discovery.zen.ping.unicast.hosts: [ {{ es_hosts }} ]'}
     - { regexp: '#path\.data', line: 'path.data: {{     elasticsearch_data_dir }}' }
-  notify: restart elasticsearch
 
 - name: Start Elasticsearch.
   service: name=elasticsearch state=started enabled=yes
+
+- include: configure_index.yml

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/68aab6e9/deployment/roles/epel/tasks/main.yml
----------------------------------------------------------------------
diff --git a/deployment/roles/epel/tasks/main.yml b/deployment/roles/epel/tasks/main.yml
new file mode 100644
index 0000000..db4e70b
--- /dev/null
+++ b/deployment/roles/epel/tasks/main.yml
@@ -0,0 +1,30 @@
+#
+#  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: Get epel-repo rpm
+  get_url:
+    dest: /tmp/epel-release.rpm
+    url: http://dl.fedoraproject.org/pub/epel/6/x86_64/epel-release-6-8.noarch.rpm
+
+- name: Install epel-repo rpm
+  yum:
+    pkg: /tmp/epel-release.rpm
+    state: installed
+  register: result
+  until: result.rc == 0
+  retries: 5
+  delay: 10

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/68aab6e9/deployment/roles/flume/meta/main.yml
----------------------------------------------------------------------
diff --git a/deployment/roles/flume/meta/main.yml b/deployment/roles/flume/meta/main.yml
new file mode 100644
index 0000000..ff35a5a
--- /dev/null
+++ b/deployment/roles/flume/meta/main.yml
@@ -0,0 +1,20 @@
+#
+#  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:
+  - java_jdk
+  - libselinux-python

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/68aab6e9/deployment/roles/flume/tasks/main.yml
----------------------------------------------------------------------
diff --git a/deployment/roles/flume/tasks/main.yml b/deployment/roles/flume/tasks/main.yml
index 7080b55..8576c3c 100644
--- a/deployment/roles/flume/tasks/main.yml
+++ b/deployment/roles/flume/tasks/main.yml
@@ -24,9 +24,12 @@
 - name: Install flume
   yum: name={{item}}
   with_items:
-    - libselinux-python
     - flume
     - flume-agent
+  register: result
+  until: result.rc == 0
+  retries: 5
+  delay: 10
 
 - name: Create flume-env.sh
   shell: cp /etc/flume/conf/flume-env.sh.template /etc/flume/conf/flume-env.sh

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/68aab6e9/deployment/roles/hadoop_setup/meta/main.yml
----------------------------------------------------------------------
diff --git a/deployment/roles/hadoop_setup/meta/main.yml b/deployment/roles/hadoop_setup/meta/main.yml
new file mode 100644
index 0000000..8f0bf2b
--- /dev/null
+++ b/deployment/roles/hadoop_setup/meta/main.yml
@@ -0,0 +1,20 @@
+#
+#  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
+

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/68aab6e9/deployment/roles/httplib2/tasks/main.yml
----------------------------------------------------------------------
diff --git a/deployment/roles/httplib2/tasks/main.yml b/deployment/roles/httplib2/tasks/main.yml
new file mode 100644
index 0000000..5502cf4
--- /dev/null
+++ b/deployment/roles/httplib2/tasks/main.yml
@@ -0,0 +1,20 @@
+#
+#  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: Install python httplib2 dependency
+  pip:
+    name: httplib2
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/68aab6e9/deployment/roles/java_jdk/defaults/main.yml
----------------------------------------------------------------------
diff --git a/deployment/roles/java_jdk/defaults/main.yml b/deployment/roles/java_jdk/defaults/main.yml
new file mode 100644
index 0000000..28f6c71
--- /dev/null
+++ b/deployment/roles/java_jdk/defaults/main.yml
@@ -0,0 +1,18 @@
+#
+#  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.
+#
+---
+java_home: /usr/jdk64/jdk1.8.0_40

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/68aab6e9/deployment/roles/java_jdk/tasks/main.yml
----------------------------------------------------------------------
diff --git a/deployment/roles/java_jdk/tasks/main.yml b/deployment/roles/java_jdk/tasks/main.yml
new file mode 100644
index 0000000..999b9c1
--- /dev/null
+++ b/deployment/roles/java_jdk/tasks/main.yml
@@ -0,0 +1,34 @@
+#
+#  Licensed to the Apache Software Foundation (ASF) under one or more
+#  contributor license agreements.  See the NOTICE file distributed with
+#  this work for additional information regarding copyright ownership.
+#  The ASF licenses this file to You under the Apache License, Version 2.0
+#  (the "License"); you may not use this file except in compliance with
+#  the License.  You may obtain a copy of the License at
+#
+#      http://www.apache.org/licenses/LICENSE-2.0
+#
+#  Unless required by applicable law or agreed to in writing, software
+#  distributed under the License is distributed on an "AS IS" BASIS,
+#  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+#  See the License for the specific language governing permissions and
+#  limitations under the License.
+#
+---
+- name: Check for java at "{{ java_home }}"
+  stat: path="{{ java_home }}"
+  register: jdk_dir
+
+- name: Alternatives link for java
+  alternatives: name={{ item.name }} link={{ item.link }}  path={{ item.path }}
+  with_items:
+    - { name: java, link: /usr/bin/java, path: "{{ java_home }}/bin/java" }
+    - { name: jar, link: /usr/bin/jar, path: "{{ java_home }}/bin/jar" }
+  when: jdk_dir.stat.exists
+
+- name: Install openjdk
+  yum: name={{item}}
+  with_items:
+    - java-1.8.0-openjdk
+    - java-1.8.0-openjdk-devel
+  when: not jdk_dir.stat.exists

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/68aab6e9/deployment/roles/libselinux-python/tasks/main.yml
----------------------------------------------------------------------
diff --git a/deployment/roles/libselinux-python/tasks/main.yml b/deployment/roles/libselinux-python/tasks/main.yml
new file mode 100644
index 0000000..78f5a27
--- /dev/null
+++ b/deployment/roles/libselinux-python/tasks/main.yml
@@ -0,0 +1,25 @@
+#
+#  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: Install libselinux-python
+  yum:
+    name: libselinux-python
+    state: installed
+  register: result
+  until: result.rc == 0
+  retries: 5
+  delay: 10

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/68aab6e9/deployment/roles/metron_common/meta/main.yml
----------------------------------------------------------------------
diff --git a/deployment/roles/metron_common/meta/main.yml b/deployment/roles/metron_common/meta/main.yml
new file mode 100644
index 0000000..4db50aa
--- /dev/null
+++ b/deployment/roles/metron_common/meta/main.yml
@@ -0,0 +1,22 @@
+#
+#  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:
+  - yum-update
+  - epel
+  - ntp
+

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/68aab6e9/deployment/roles/metron_common/tasks/main.yml
----------------------------------------------------------------------
diff --git a/deployment/roles/metron_common/tasks/main.yml b/deployment/roles/metron_common/tasks/main.yml
index 29c0821..64e6ab9 100644
--- a/deployment/roles/metron_common/tasks/main.yml
+++ b/deployment/roles/metron_common/tasks/main.yml
@@ -27,21 +27,9 @@
   fail: msg="Unable to locate staged Metron jars at {{ metron_jar_path }}.  Did you run 'mvn package'?"
   when: metron_jars.stat.exists == True
 
-- name: upgrade all packages
-  yum: name=* state=latest
-
-- name: get epel-repo rpm
-  get_url: dest=/tmp/epel-release.rpm  url=http://dl.fedoraproject.org/pub/epel/6/x86_64/epel-release-6-8.noarch.rpm
-
-- name: install epel-repo rpm
-  yum: pkg=/tmp/epel-release.rpm state=installed
-
-- name: install ntp
-  yum: name=ntp state=present
-
-- name: Ensure ntp is running and enabled
-  service: name=ntpd state=started enabled=yes
-
 - name: Ensure iptables is stopped and is not running at boot time.
   ignore_errors: yes
-  service: name=iptables state=stopped enabled=no
\ No newline at end of file
+  service:
+    name: iptables
+    state: stopped
+    enabled: no

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/68aab6e9/deployment/roles/metron_pcapservice/meta/main.yml
----------------------------------------------------------------------
diff --git a/deployment/roles/metron_pcapservice/meta/main.yml b/deployment/roles/metron_pcapservice/meta/main.yml
new file mode 100644
index 0000000..ddf6aa9
--- /dev/null
+++ b/deployment/roles/metron_pcapservice/meta/main.yml
@@ -0,0 +1,19 @@
+#
+#  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:
+  - java_jdk

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/68aab6e9/deployment/roles/metron_pcapservice/tasks/config-java.yml
----------------------------------------------------------------------
diff --git a/deployment/roles/metron_pcapservice/tasks/config-java.yml b/deployment/roles/metron_pcapservice/tasks/config-java.yml
deleted file mode 100644
index 999b9c1..0000000
--- a/deployment/roles/metron_pcapservice/tasks/config-java.yml
+++ /dev/null
@@ -1,34 +0,0 @@
-#
-#  Licensed to the Apache Software Foundation (ASF) under one or more
-#  contributor license agreements.  See the NOTICE file distributed with
-#  this work for additional information regarding copyright ownership.
-#  The ASF licenses this file to You under the Apache License, Version 2.0
-#  (the "License"); you may not use this file except in compliance with
-#  the License.  You may obtain a copy of the License at
-#
-#      http://www.apache.org/licenses/LICENSE-2.0
-#
-#  Unless required by applicable law or agreed to in writing, software
-#  distributed under the License is distributed on an "AS IS" BASIS,
-#  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-#  See the License for the specific language governing permissions and
-#  limitations under the License.
-#
----
-- name: Check for java at "{{ java_home }}"
-  stat: path="{{ java_home }}"
-  register: jdk_dir
-
-- name: Alternatives link for java
-  alternatives: name={{ item.name }} link={{ item.link }}  path={{ item.path }}
-  with_items:
-    - { name: java, link: /usr/bin/java, path: "{{ java_home }}/bin/java" }
-    - { name: jar, link: /usr/bin/jar, path: "{{ java_home }}/bin/jar" }
-  when: jdk_dir.stat.exists
-
-- name: Install openjdk
-  yum: name={{item}}
-  with_items:
-    - java-1.8.0-openjdk
-    - java-1.8.0-openjdk-devel
-  when: not jdk_dir.stat.exists

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/68aab6e9/deployment/roles/metron_pcapservice/tasks/main.yml
----------------------------------------------------------------------
diff --git a/deployment/roles/metron_pcapservice/tasks/main.yml b/deployment/roles/metron_pcapservice/tasks/main.yml
index 68da876..ee9cac2 100644
--- a/deployment/roles/metron_pcapservice/tasks/main.yml
+++ b/deployment/roles/metron_pcapservice/tasks/main.yml
@@ -15,8 +15,6 @@
 #  limitations under the License.
 #
 ---
-- include: config-java.yml
-
 - name: Check for hbase-site
   stat: path=/etc/hbase/conf/hbase-site.xml
   register: hbase_site

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/68aab6e9/deployment/roles/metron_streaming/meta/main.yml
----------------------------------------------------------------------
diff --git a/deployment/roles/metron_streaming/meta/main.yml b/deployment/roles/metron_streaming/meta/main.yml
new file mode 100644
index 0000000..6820b2c
--- /dev/null
+++ b/deployment/roles/metron_streaming/meta/main.yml
@@ -0,0 +1,20 @@
+#
+#  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/68aab6e9/deployment/roles/metron_streaming/tasks/grok_upload.yml
----------------------------------------------------------------------
diff --git a/deployment/roles/metron_streaming/tasks/grok_upload.yml b/deployment/roles/metron_streaming/tasks/grok_upload.yml
new file mode 100644
index 0000000..1bcf39f
--- /dev/null
+++ b/deployment/roles/metron_streaming/tasks/grok_upload.yml
@@ -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.
+#
+---
+- name: Create HDFS directory for grok patterns
+  command: hdfs dfs -mkdir -p {{ metron_hdfs_output_dir }}/patterns
+  become: yes
+  become_user: hdfs
+
+- name: Assign hfds user as owner of  {{ metron_hdfs_output_dir }}/patterns HDFS directory
+  command: hdfs dfs -chown -R hdfs:hadoop {{ metron_hdfs_output_dir }}/patterns
+  become: yes
+  become_user: hdfs
+
+- name: Assign permissions of HDFS {{ metron_hdfs_output_dir }}/patterns directory
+  command: hdfs dfs -chmod -R 775 {{ metron_hdfs_output_dir }}/patterns
+  become: yes
+  become_user: hdfs
+
+- name: Upload Grok Patterns to hdfs://{{ metron_hdfs_output_dir }}
+  command: hdfs dfs -put {{ metron_directory }}/config/patterns  {{ metron_hdfs_output_dir }}
+  become: yes
+  become_user: hdfs
+

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/68aab6e9/deployment/roles/metron_streaming/tasks/hdfs_filesystem.yml
----------------------------------------------------------------------
diff --git a/deployment/roles/metron_streaming/tasks/hdfs_filesystem.yml b/deployment/roles/metron_streaming/tasks/hdfs_filesystem.yml
new file mode 100644
index 0000000..252e671
--- /dev/null
+++ b/deployment/roles/metron_streaming/tasks/hdfs_filesystem.yml
@@ -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.
+#
+---
+- name: Create root user HDFS directory
+  command: hdfs dfs -mkdir -p /user/root
+  become: yes
+  become_user: hdfs
+
+- name: Assign root as owner of /user/root HDFS directory
+  command: hdfs dfs -chown root:root /user/root
+  become: yes
+  become_user: hdfs
+
+- name: Create Metron HDFS output directory
+  command: hdfs dfs -mkdir -p {{ metron_hdfs_output_dir }}
+  become: yes
+  become_user: hdfs
+
+- name: Assign hdfs as owner of HDFS output directory
+  command: hdfs dfs -chown hdfs:hadoop {{ metron_hdfs_output_dir }}
+  become: yes
+  become_user: hdfs
+
+- name: Assign permissions of HDFS output directory
+  command: hdfs dfs -chmod 775 {{ metron_hdfs_output_dir }}
+  become: yes
+  become_user: hdfs
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/68aab6e9/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 27b087c..c0e0638 100644
--- a/deployment/roles/metron_streaming/tasks/main.yml
+++ b/deployment/roles/metron_streaming/tasks/main.yml
@@ -36,12 +36,6 @@
 - name: Unbundle Metron bundles
   shell: cd {{ metron_directory }} && tar xzvf Metron-Topologies*.tar.gz && tar xzvf Metron-DataLoads*.tar.gz && rm *.tar.gz
 
-- name: Alternatives link for "java"
-  alternatives: name={{ item.name }} link={{ item.link }}  path={{ item.path }}
-  with_items:
-    - { name: java, link: /usr/bin/java, path: "{{ java_home }}/bin/java" }
-    - { name: jar, link: /usr/bin/jar, path: "{{ java_home }}/bin/jar" }
-
 - name: Add hbase-site.xml to topology jar
   shell: cd {{ hbase_config_path }} && jar -uf {{ metron_directory }}/lib/{{ metron_jar_name }} hbase-site.xml
 
@@ -52,17 +46,14 @@
   shell: cd {{ hdfs_config_path }} && jar -uf {{ metron_directory }}/lib/{{ metron_jar_name }} hdfs-site.xml
 
 - name: Get Default mysql passowrd
-  include_vars: "../roles/mysql_server/vars/main.yml"
+  include_vars: "../roles/mysql_server/defaults/main.yml"
   when: mysql_root_password is undefined
 
-- name: Create root user HDFS directory
-  command: su - hdfs -c "hdfs dfs -mkdir -p /user/root && hdfs dfs -chown root:root /user/root"
-
-- name: Create Metron HDFS output directory
-  command: su - hdfs -c "hdfs dfs -mkdir -p {{ metron_hdfs_output_dir }} && hdfs dfs -chown hdfs:hadoop {{ metron_hdfs_output_dir }} && hdfs dfs -chmod 775 {{ metron_hdfs_output_dir }}"
+- include: hdfs_filesystem.yml
+  run_once: true
 
-- name: Upload grok patterns
-  command: su - hdfs -c "hdfs dfs -mkdir -p {{ metron_hdfs_output_dir }}/patterns && hdfs dfs -put {{ metron_directory }}/config/patterns/*  {{ metron_hdfs_output_dir }}/patterns && hdfs dfs -chown -R hdfs:hadoop {{ metron_hdfs_output_dir }}/patterns && hdfs dfs -chmod -R 775 {{ metron_hdfs_output_dir }}/patterns"
+- include: grok_upload.yml
+  run_once: true
 
 - name: Configure Metron topologies
   lineinfile: >
@@ -87,16 +78,10 @@
     - { regexp: "mysql.password=", line: "mysql.password={{ mysql_root_password }}" }
     - { regexp: "index.hdfs.output=", line: "index.hdfs.output={{ metron_hdfs_output_dir }}/enrichment/indexed" }
 
-- name: Add Elasticsearch templates for topologies
-  uri:
-    url: "http://{{ groups.search[0] }}:{{ elasticsearch_web_port }}/_template/template_yaf"
-    method: POST
-    body: "{{ lookup('file','yaf_index.template') }}"
-    status_code: 200
-    body_format: json
 
 - include: source_config.yml
   run_once: true
+
 - include: threat_intel.yml
   run_once: true
   when: threat_intel_bulk_load == True

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/68aab6e9/deployment/roles/metron_ui/tasks/main.yml
----------------------------------------------------------------------
diff --git a/deployment/roles/metron_ui/tasks/main.yml b/deployment/roles/metron_ui/tasks/main.yml
index 861abbd..01ffc17 100644
--- a/deployment/roles/metron_ui/tasks/main.yml
+++ b/deployment/roles/metron_ui/tasks/main.yml
@@ -16,12 +16,19 @@
 #
 ---
 - name: Install Metron UI dependencies
-  yum: pkg={{ item.package }} state=installed
+  yum:
+    pkg: "{{ item.package }}"
+    state: installed
   with_items:
       - { package: 'libpcap-devel'}
       - { package: 'wireshark'}
       - { package: 'nodejs'}
       - { package: 'npm'}
+  register: result
+  until: result.rc == 0
+  retries: 5
+  delay: 10
+
 
 - name: Copy Metron UI source
   copy:

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/68aab6e9/deployment/roles/mysql_server/tasks/main.yml
----------------------------------------------------------------------
diff --git a/deployment/roles/mysql_server/tasks/main.yml b/deployment/roles/mysql_server/tasks/main.yml
index e97b245..a484ed0 100644
--- a/deployment/roles/mysql_server/tasks/main.yml
+++ b/deployment/roles/mysql_server/tasks/main.yml
@@ -30,6 +30,10 @@
   yum:
     pkg: /tmp/{{ mysql_rpm_version }}.rpm
     state: installed
+  register: result
+  until: result.rc == 0
+  retries: 5
+  delay: 10
 
 - name: Install MySQL
   yum:
@@ -38,6 +42,10 @@
   with_items:
     - "mysql-community-server"
     - "MySQL-python"
+  register: result
+  until: result.rc == 0
+  retries: 5
+  delay: 10
 
 - name: Start MySQL
   service:

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/68aab6e9/deployment/roles/ntp/tasks/main.yml
----------------------------------------------------------------------
diff --git a/deployment/roles/ntp/tasks/main.yml b/deployment/roles/ntp/tasks/main.yml
new file mode 100644
index 0000000..7b1b9a8
--- /dev/null
+++ b/deployment/roles/ntp/tasks/main.yml
@@ -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.
+#
+---
+- name: Install ntp
+  yum:
+    name: ntp
+    state: present
+  register: result
+  until: result.rc == 0
+  retries: 5
+  delay: 10
+
+- name: Ensure ntp is running and enabled
+  service:
+    name: ntpd
+    state: started
+    enabled: yes

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/68aab6e9/deployment/roles/pcap_replay/meta/main.yml
----------------------------------------------------------------------
diff --git a/deployment/roles/pcap_replay/meta/main.yml b/deployment/roles/pcap_replay/meta/main.yml
index 2df379c..0c47853 100644
--- a/deployment/roles/pcap_replay/meta/main.yml
+++ b/deployment/roles/pcap_replay/meta/main.yml
@@ -15,20 +15,7 @@
 #  limitations under the License.
 #
 ---
-galaxy_info:
-  author: Nick Allen
-  description:
-  company: Metron
+dependencies:
+  - libselinux-python
+  - build-tools
 
-  license: license (Apache, CC-BY, Nick Allen)
-  min_ansible_version: 1.2
-
-  platforms:
-  - name: CentOS
-    versions:
-      - 6
-
-  categories:
-  - networking
-
-dependencies: []

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/68aab6e9/deployment/roles/pcap_replay/tasks/main.yml
----------------------------------------------------------------------
diff --git a/deployment/roles/pcap_replay/tasks/main.yml b/deployment/roles/pcap_replay/tasks/main.yml
index 449710e..68005b9 100644
--- a/deployment/roles/pcap_replay/tasks/main.yml
+++ b/deployment/roles/pcap_replay/tasks/main.yml
@@ -15,9 +15,6 @@
 #  limitations under the License.
 #
 ---
-- name: Install prerequisites
-  yum: name=libselinux-python
-
 - include: tcpreplay.yml
 
 - include: service.yml

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/68aab6e9/deployment/roles/pcap_replay/tasks/tcpreplay.yml
----------------------------------------------------------------------
diff --git a/deployment/roles/pcap_replay/tasks/tcpreplay.yml b/deployment/roles/pcap_replay/tasks/tcpreplay.yml
index 851a6ae..e24dcf1 100644
--- a/deployment/roles/pcap_replay/tasks/tcpreplay.yml
+++ b/deployment/roles/pcap_replay/tasks/tcpreplay.yml
@@ -15,13 +15,6 @@
 #  limitations under the License.
 #
 ---
-- name: Install prerequisites
-  yum: name={{ item }}
-  with_items:
-    - "@Development tools"
-    - libpcap
-    - libpcap-devel
-
 - name: Download tcpreplay
   get_url:
     url: "https://github.com/appneta/tcpreplay/releases/download/v{{ tcpreplay_version }}/tcpreplay-{{ tcpreplay_version }}.tar.gz"

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/68aab6e9/deployment/roles/pycapa/meta/main.yml
----------------------------------------------------------------------
diff --git a/deployment/roles/pycapa/meta/main.yml b/deployment/roles/pycapa/meta/main.yml
new file mode 100644
index 0000000..85c04df
--- /dev/null
+++ b/deployment/roles/pycapa/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
+  - epel
+  - python-pip

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/68aab6e9/deployment/roles/pycapa/tasks/main.yml
----------------------------------------------------------------------
diff --git a/deployment/roles/pycapa/tasks/main.yml b/deployment/roles/pycapa/tasks/main.yml
index 71e5964..8649f3a 100644
--- a/deployment/roles/pycapa/tasks/main.yml
+++ b/deployment/roles/pycapa/tasks/main.yml
@@ -15,18 +15,17 @@
 #  limitations under the License.
 #
 ---
-- name: Install EPEL repository
-  yum: name=epel-release
-
 - name: Install python and tcpdump
-  yum: name={{item}}
+  yum:
+    name: "{{item}}"
   with_items:
     - python
     - tcpdump
     - git
-
-- name: Install pip
-  easy_install: name=pip state=latest
+  register: result
+  until: result.rc == 0
+  retries: 5
+  delay: 10
 
 - name: Clone pycapa repo
   git: repo={{ pycapa_repo }} dest={{ pycapa_home }}
@@ -43,8 +42,13 @@
 - name: Turn on promiscuous mode for {{ sniff_interface }}
   shell: "ip link set {{ sniff_interface }} promisc on"
 
-- name: Install prerequisites
-  yum: name=daemonize
+- name: Install Daemonize
+  yum:
+    name: daemonize
+  register: result
+  until: result.rc == 0
+  retries: 5
+  delay: 10
 
 - set_fact:
     pycapa: /opt/pycapa/pycapa/pycapa_cli.py

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/68aab6e9/deployment/roles/python-pip/tasks/main.yml
----------------------------------------------------------------------
diff --git a/deployment/roles/python-pip/tasks/main.yml b/deployment/roles/python-pip/tasks/main.yml
new file mode 100644
index 0000000..809aca4
--- /dev/null
+++ b/deployment/roles/python-pip/tasks/main.yml
@@ -0,0 +1,25 @@
+#
+#  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: Install python-pip
+  yum:
+    name: python-pip
+    state: installed
+  register: result
+  until: result.rc == 0
+  retries: 5
+  delay: 10

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/68aab6e9/deployment/roles/snort/meta/main.yml
----------------------------------------------------------------------
diff --git a/deployment/roles/snort/meta/main.yml b/deployment/roles/snort/meta/main.yml
new file mode 100644
index 0000000..df99998
--- /dev/null
+++ b/deployment/roles/snort/meta/main.yml
@@ -0,0 +1,22 @@
+#
+#  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
+  - epel
+  - libselinux-python
+  - build-tools

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/68aab6e9/deployment/roles/snort/tasks/daq.yml
----------------------------------------------------------------------
diff --git a/deployment/roles/snort/tasks/daq.yml b/deployment/roles/snort/tasks/daq.yml
index 8f0dcae..50d2049 100644
--- a/deployment/roles/snort/tasks/daq.yml
+++ b/deployment/roles/snort/tasks/daq.yml
@@ -27,4 +27,10 @@
     creates: /root/rpmbuild/RPMS/x86_64/daq-{{ daq_version }}.x86_64.rpm
 
 - name: Install daq
-  yum: name=/root/rpmbuild/RPMS/x86_64/daq-{{ daq_version }}.x86_64.rpm
+  yum:
+    name: /root/rpmbuild/RPMS/x86_64/daq-{{ daq_version }}.x86_64.rpm
+  register: result
+  until: result.rc == 0
+  retries: 5
+  delay: 10
+

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/68aab6e9/deployment/roles/snort/tasks/main.yml
----------------------------------------------------------------------
diff --git a/deployment/roles/snort/tasks/main.yml b/deployment/roles/snort/tasks/main.yml
index 43b9c16..80755be 100644
--- a/deployment/roles/snort/tasks/main.yml
+++ b/deployment/roles/snort/tasks/main.yml
@@ -15,22 +15,6 @@
 #  limitations under the License.
 #
 ---
-- name: Install EPEL repository
-  yum: name=epel-release
-
-- name: Install snort prerequisites
-  yum: name={{ item }}
-  with_items:
-    - "@Development tools"
-    - libdnet-devel
-    - rpm-build
-    - libpcap
-    - libpcap-devel
-    - pcre
-    - pcre-devel
-    - zlib-devel
-    - libselinux-python
-
 - include: daq.yml
 
 - include: snort.yml

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/68aab6e9/deployment/roles/snort/tasks/snort.yml
----------------------------------------------------------------------
diff --git a/deployment/roles/snort/tasks/snort.yml b/deployment/roles/snort/tasks/snort.yml
index c08bc93..b56ca74 100644
--- a/deployment/roles/snort/tasks/snort.yml
+++ b/deployment/roles/snort/tasks/snort.yml
@@ -15,20 +15,6 @@
 #  limitations under the License.
 #
 ---
-- name: Install development tools
-  yum: name="@Development tools" state=present
-
-- name: Install snort prerequisites
-  yum: name={{ item }}
-  with_items:
-    - libdnet-devel
-    - rpm-build
-    - libpcap
-    - libpcap-devel
-    - pcre
-    - pcre-devel
-    - zlib-devel
-
 - name: Download daq
   get_url:
     url: "https://snort.org/downloads/snort/daq-{{ daq_version }}.src.rpm"
@@ -41,7 +27,12 @@
     creates: /root/rpmbuild/RPMS/x86_64/daq-{{ daq_version }}.x86_64.rpm
 
 - name: Install daq
-  yum: name=/root/rpmbuild/RPMS/x86_64/daq-{{ daq_version }}.x86_64.rpm
+  yum:
+    name: /root/rpmbuild/RPMS/x86_64/daq-{{ daq_version }}.x86_64.rpm
+  register: result
+  until: result.rc == 0
+  retries: 5
+  delay: 10
 
 - name: Download snort
   get_url:
@@ -55,7 +46,12 @@
     creates: /root/rpmbuild/RPMS/x86_64/snort-{{ snort_version }}.x86_64.rpm
 
 - name: Install snort
-  yum: name=/root/rpmbuild/RPMS/x86_64/snort-{{ snort_version }}.x86_64.rpm
+  yum:
+    name: /root/rpmbuild/RPMS/x86_64/snort-{{ snort_version }}.x86_64.rpm
+  register: result
+  until: result.rc == 0
+  retries: 5
+  delay: 10
 
 - name: Download snort community rules
   get_url:

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/68aab6e9/deployment/roles/tap_interface/tasks/main.yml
----------------------------------------------------------------------
diff --git a/deployment/roles/tap_interface/tasks/main.yml b/deployment/roles/tap_interface/tasks/main.yml
index d4590f7..1de3abe 100644
--- a/deployment/roles/tap_interface/tasks/main.yml
+++ b/deployment/roles/tap_interface/tasks/main.yml
@@ -16,9 +16,14 @@
 #
 ---
 - name: Install tunctl
-  yum: name=tunctl
+  yum:
+    name: tunctl
+    state: installed
+  register: result
+  until: result.rc == 0
+  retries: 5
+  delay: 10
 
-#TODO - only run when tap_if does not exist
 - name: Create {{ tap_if }}
   command: tunctl -p
 

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/68aab6e9/deployment/roles/yaf/meta/main.yml
----------------------------------------------------------------------
diff --git a/deployment/roles/yaf/meta/main.yml b/deployment/roles/yaf/meta/main.yml
new file mode 100644
index 0000000..324f2ff
--- /dev/null
+++ b/deployment/roles/yaf/meta/main.yml
@@ -0,0 +1,22 @@
+#
+#  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
+  - build-tools
+  - java_jdk
+  - libselinux-python

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/68aab6e9/deployment/roles/yaf/tasks/dependencies.yml
----------------------------------------------------------------------
diff --git a/deployment/roles/yaf/tasks/dependencies.yml b/deployment/roles/yaf/tasks/dependencies.yml
deleted file mode 100644
index 1a28b2a..0000000
--- a/deployment/roles/yaf/tasks/dependencies.yml
+++ /dev/null
@@ -1,31 +0,0 @@
-#
-#  Licensed to the Apache Software Foundation (ASF) under one or more
-#  contributor license agreements.  See the NOTICE file distributed with
-#  this work for additional information regarding copyright ownership.
-#  The ASF licenses this file to You under the Apache License, Version 2.0
-#  (the "License"); you may not use this file except in compliance with
-#  the License.  You may obtain a copy of the License at
-#
-#      http://www.apache.org/licenses/LICENSE-2.0
-#
-#  Unless required by applicable law or agreed to in writing, software
-#  distributed under the License is distributed on an "AS IS" BASIS,
-#  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-#  See the License for the specific language governing permissions and
-#  limitations under the License.
-#
----
-- name: Install prerequisites
-  yum: name={{ item }}
-  with_items:
-    - "@Development tools"
-    - libselinux-python
-    - libpcap
-    - libpcap-devel
-    - pcre
-    - pcre-devel
-    - glib2-devel
-    - zlib
-    - zlib-devel
-    - java-1.8.0-openjdk
-    - java-1.8.0-openjdk-devel

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/68aab6e9/deployment/roles/yaf/tasks/kafka-client.yml
----------------------------------------------------------------------
diff --git a/deployment/roles/yaf/tasks/kafka-client.yml b/deployment/roles/yaf/tasks/kafka-client.yml
index 729929a..1674225 100644
--- a/deployment/roles/yaf/tasks/kafka-client.yml
+++ b/deployment/roles/yaf/tasks/kafka-client.yml
@@ -22,6 +22,9 @@
       mode: 0644
 
   - name: Install kafka
-    yum: name={{ item }}
-    with_items:
-      - kafka
+    yum:
+      name: kafka
+    register: result
+    until: result.rc == 0
+    retries: 5
+    delay: 10

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/68aab6e9/deployment/roles/yaf/tasks/main.yml
----------------------------------------------------------------------
diff --git a/deployment/roles/yaf/tasks/main.yml b/deployment/roles/yaf/tasks/main.yml
index e661f50..b583393 100644
--- a/deployment/roles/yaf/tasks/main.yml
+++ b/deployment/roles/yaf/tasks/main.yml
@@ -15,7 +15,6 @@
 #  limitations under the License.
 #
 ---
-- include: dependencies.yml
 - include: kafka-client.yml
 - include: fixbuf.yml
 - include: yaf.yml

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/68aab6e9/deployment/roles/yum-update/tasks/main.yml
----------------------------------------------------------------------
diff --git a/deployment/roles/yum-update/tasks/main.yml b/deployment/roles/yum-update/tasks/main.yml
new file mode 100644
index 0000000..4db6297
--- /dev/null
+++ b/deployment/roles/yum-update/tasks/main.yml
@@ -0,0 +1,26 @@
+#
+#  Licensed to the Apache Software Foundation (ASF) under one or more
+#  contributor license agreements.  See the NOTICE file distributed with
+#  this work for additional information regarding copyright ownership.
+#  The ASF licenses this file to You under the Apache License, Version 2.0
+#  (the "License"); you may not use this file except in compliance with
+#  the License.  You may obtain a copy of the License at
+#
+#      http://www.apache.org/licenses/LICENSE-2.0
+#
+#  Unless required by applicable law or agreed to in writing, software
+#  distributed under the License is distributed on an "AS IS" BASIS,
+#  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+#  See the License for the specific language governing permissions and
+#  limitations under the License.
+#
+---
+- name: Yum Update Packages
+  yum:
+    name: "*"
+    state: latest
+    update_cache: yes
+  register: result
+  until: result.rc == 0
+  retries: 5
+  delay: 10


[2/3] incubator-metron git commit: METRON-72 Create unified enrichment topology (merrimanr via jsirota) closes apache/incubator-metron#50

Posted by js...@apache.org.
METRON-72 Create unified enrichment topology (merrimanr via jsirota) closes apache/incubator-metron#50


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

Branch: refs/heads/master
Commit: c737aa9d90ade31514f21312bd58b0da94c6a98e
Parents: d280837
Author: merrimanr <me...@gmail.com>
Authored: Fri Mar 18 13:30:03 2016 -0700
Committer: James Sirota <Ja...@yahoo.com>
Committed: Fri Mar 18 13:30:03 2016 -0700

----------------------------------------------------------------------
 .../inventory/singlenode-vagrant/group_vars/all |   3 +-
 deployment/roles/metron_ui/files/default.json   | 508 -------------------
 deployment/roles/metron_ui/files/opensoc-ui     |   6 -
 deployment/roles/metron_ui/tasks/main.yml       |  53 +-
 deployment/roles/metron_ui/vars/main.yml        |   1 +
 .../java/org/apache/metron/pcap/PcapUtils.java  |  25 +
 .../org/apache/metron/writer/PcapWriter.java    |   3 +-
 .../metron/parsing/parsers/PcapParser.java      |   9 +-
 metron-streaming/Metron-Pcap_Service/pom.xml    |  27 +-
 .../pcapservice/HBaseConfigurationUtil.java     |   2 +-
 .../pcapservice/PcapReceiverImplRestEasy.java   |   2 +-
 .../metron/pcapservice/RestTestingUtil.java     |  74 +++
 .../metron/pcapservice/rest/PcapService.java    |   1 +
 .../resources/hbase-config-default.properties   |   2 +-
 metron-streaming/Metron-Testing/pom.xml         |  10 +
 .../components/KafkaWithZKComponent.java        |   2 +-
 .../util/integration/util/PcapTestUtil.java     |  77 +++
 .../Metron_Configs/etc/env/config.properties    |   6 +-
 .../Metron_Configs/topologies/pcap/local.yaml   | 440 ----------------
 .../Metron_Configs/topologies/pcap/parse.yaml   |  70 ---
 .../Metron_Configs/topologies/pcap/remote.yaml  | 377 +-------------
 .../Metron_Configs/topologies/pcap/test.yaml    |  74 +++
 .../SampleInput/.PCAPExampleOutput.crc          | Bin 0 -> 44 bytes
 .../resources/SampleInput/PCAPExampleOutput     | Bin 4096 -> 4510 bytes
 .../integration/PcapParserIntegrationTest.java  | 218 ++++++++
 metron-ui/.gitignore                            |   3 -
 metron-ui/config.json                           |   6 +
 metron-ui/lib/metron-ui.js                      |   7 +-
 metron-ui/lib/modules/es-proxy.js               |   2 +-
 metron-ui/lib/modules/login.js                  |   2 +-
 metron-ui/lib/modules/pcap.js                   |  12 +-
 .../lib/public/app/dashboards/default.json      | 461 +++++++++--------
 .../lib/public/app/panels/pcap/module.html      | 106 +---
 metron-ui/lib/public/app/panels/pcap/module.js  |   5 +
 metron-ui/lib/views/alerts.jade                 |  36 +-
 metron-ui/lib/views/index.jade                  |  34 +-
 metron-ui/lib/views/login.jade                  |  34 +-
 metron-ui/package.json                          |  72 +--
 38 files changed, 917 insertions(+), 1853 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/c737aa9d/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 2d1157e..06acaa9 100644
--- a/deployment/inventory/singlenode-vagrant/group_vars/all
+++ b/deployment/inventory/singlenode-vagrant/group_vars/all
@@ -65,7 +65,7 @@ ambari_server_mem: 512
 threat_intel_bulk_load: False
 
 #Sensors
-install_pycapa: False
+install_pycapa: True
 install_bro: True
 install_snort: True
 install_yaf: False
@@ -74,6 +74,7 @@ sniff_interface: eth1
 pcap_replay_interface: "{{ sniff_interface }}"
 storm_topologies:
     - "{{ metron_directory }}/config/topologies/bro/remote.yaml"
+    - "{{ metron_directory }}/config/topologies/pcap/remote.yaml"
     - "{{ metron_directory }}/config/topologies/snort/remote.yaml"
     - "{{ metron_directory }}/config/topologies/enrichment/remote.yaml"
 pcapservice_port: 8081

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/c737aa9d/deployment/roles/metron_ui/files/default.json
----------------------------------------------------------------------
diff --git a/deployment/roles/metron_ui/files/default.json b/deployment/roles/metron_ui/files/default.json
deleted file mode 100644
index f4c1d2f..0000000
--- a/deployment/roles/metron_ui/files/default.json
+++ /dev/null
@@ -1,508 +0,0 @@
-{
-  "title": "Metron",
-  "services": {
-    "query": {
-      "list": {
-        "1": {
-          "id": 1,
-          "color": "#6ED0E0",
-          "alias": "Yaf",
-          "pin": true,
-          "type": "lucene",
-          "enable": true,
-          "query": "_type:yaf_doc"
-        },
-        "2": {
-          "id": 2,
-          "color": "#BA43A9",
-          "alias": "All Events",
-          "pin": true,
-          "type": "lucene",
-          "enable": true,
-          "query": "_type:*_doc"
-        },
-        "3": {
-          "id": 3,
-          "color": "#BA43A9",
-          "alias": "All Alerts",
-          "pin": true,
-          "type": "lucene",
-          "enable": true,
-          "query": "_type:*_alert"
-        },
-        "4": {
-          "id": 4,
-          "color": "#1F78C1",
-          "alias": "Bro Events",
-          "pin": true,
-          "type": "lucene",
-          "enable": true,
-          "query": "_type:bro_doc"
-        },
-        "5": {
-          "id": 5,
-          "color": "#EF843C",
-          "alias": "Bro Alerts",
-          "pin": true,
-          "type": "lucene",
-          "enable": true,
-          "query": "_type:bro_alert"
-        },
-        "6": {
-          "id": 6,
-          "color": "#BA43A9",
-          "alias": "Sourcefire Events",
-          "pin": true,
-          "type": "lucene",
-          "enable": true,
-          "query": "_type:sourcefire_doc"
-        },
-        "7": {
-          "id": 7,
-          "color": "#BF1B00",
-          "alias": "Sourcefire Alerts",
-          "pin": true,
-          "type": "lucene",
-          "enable": true,
-          "query": "_type:sourcefire_alert"
-        }
-      },
-      "ids": [
-        1,
-        2,
-        3,
-        4,
-        5,
-        6,
-        7
-      ]
-    },
-    "filter": {
-      "list": {
-        "0": {
-          "type": "time",
-          "field": "timestamp",
-          "from": "now-24h",
-          "to": "now",
-          "mandate": "must",
-          "active": true,
-          "alias": "",
-          "id": 0
-        }
-      },
-      "ids": [
-        0
-      ]
-    }
-  },
-  "rows": [
-    {
-      "title": "Histogram",
-      "height": "150px",
-      "editable": true,
-      "collapse": false,
-      "collapsable": true,
-      "panels": [
-        {
-          "span": 6,
-          "editable": true,
-          "type": "histogram",
-          "loadingEditor": false,
-          "mode": "count",
-          "time_field": "timestamp",
-          "value_field": null,
-          "x-axis": true,
-          "y-axis": true,
-          "scale": 1,
-          "y_format": "short",
-          "grid": {
-            "max": null,
-            "min": 0
-          },
-          "queries": {
-            "mode": "selected",
-            "ids": [
-              4,
-              5
-            ]
-          },
-          "locked": false,
-          "annotate": {
-            "enable": false,
-            "query": "*",
-            "size": 20,
-            "field": "_type",
-            "sort": [
-              "_score",
-              "desc"
-            ]
-          },
-          "auto_int": true,
-          "resolution": 100,
-          "interval": "10m",
-          "intervals": [
-            "auto",
-            "1s",
-            "1m",
-            "5m",
-            "10m",
-            "30m",
-            "1h",
-            "3h",
-            "12h",
-            "1d",
-            "1w",
-            "1y"
-          ],
-          "lines": false,
-          "fill": 0,
-          "linewidth": 3,
-          "points": false,
-          "pointradius": 5,
-          "bars": true,
-          "stack": false,
-          "spyable": true,
-          "zoomlinks": true,
-          "options": true,
-          "legend": true,
-          "show_query": true,
-          "interactive": true,
-          "legend_counts": true,
-          "timezone": "browser",
-          "percentage": false,
-          "zerofill": true,
-          "derivative": false,
-          "tooltip": {
-            "value_type": "individual",
-            "query_as_alias": true
-          },
-          "title": "Bro Data"
-        },
-        {
-          "span": 6,
-          "editable": true,
-          "type": "histogram",
-          "loadingEditor": false,
-          "mode": "count",
-          "time_field": "timestamp",
-          "value_field": null,
-          "x-axis": true,
-          "y-axis": true,
-          "scale": 1,
-          "y_format": "none",
-          "grid": {
-            "max": null,
-            "min": 0
-          },
-          "queries": {
-            "mode": "selected",
-            "ids": [
-              6,
-              7
-            ]
-          },
-          "locked": false,
-          "annotate": {
-            "enable": false,
-            "query": "*",
-            "size": 20,
-            "field": "_type",
-            "sort": [
-              "_score",
-              "desc"
-            ]
-          },
-          "auto_int": true,
-          "resolution": 100,
-          "interval": "10m",
-          "intervals": [
-            "auto",
-            "1s",
-            "1m",
-            "5m",
-            "10m",
-            "30m",
-            "1h",
-            "3h",
-            "12h",
-            "1d",
-            "1w",
-            "1y"
-          ],
-          "lines": false,
-          "fill": 0,
-          "linewidth": 3,
-          "points": false,
-          "pointradius": 5,
-          "bars": true,
-          "stack": true,
-          "spyable": true,
-          "zoomlinks": true,
-          "options": true,
-          "legend": true,
-          "show_query": true,
-          "interactive": true,
-          "legend_counts": true,
-          "timezone": "browser",
-          "percentage": false,
-          "zerofill": true,
-          "derivative": false,
-          "tooltip": {
-            "value_type": "cumulative",
-            "query_as_alias": true
-          },
-          "title": "Sourcefire Data"
-        }
-      ],
-      "notice": false
-    },
-    {
-      "title": "Alerts",
-      "height": "150px",
-      "editable": true,
-      "collapse": false,
-      "collapsable": true,
-      "panels": [
-        {
-          "error": false,
-          "span": 12,
-          "editable": true,
-          "type": "table",
-          "loadingEditor": false,
-          "size": 10,
-          "pages": 100,
-          "offset": 0,
-          "sort": [
-            "timestamp",
-            "desc"
-          ],
-          "overflow": "min-height",
-          "fields": [
-            "_type",
-            "timestamp",
-            "priority",
-            "designated_host",
-            "description"
-          ],
-          "highlight": [],
-          "sortable": true,
-          "header": true,
-          "paging": true,
-          "field_list": false,
-          "all_fields": false,
-          "trimFactor": 400,
-          "localTime": true,
-          "timeField": "timestamp",
-          "spyable": true,
-          "queries": {
-            "mode": "selected",
-            "ids": [
-              3
-            ]
-          },
-          "locked": false,
-          "style": {
-            "font-size": "9pt"
-          },
-          "normTimes": true,
-          "title": "Alerts"
-        }
-      ],
-      "notice": false
-    },
-    {
-      "title": "Events",
-      "height": "150px",
-      "editable": true,
-      "collapse": false,
-      "collapsable": true,
-      "panels": [
-        {
-          "error": false,
-          "span": 12,
-          "editable": true,
-          "type": "table",
-          "loadingEditor": false,
-          "size": 10,
-          "pages": 100,
-          "offset": 0,
-          "sort": [
-            "message.timestamp",
-            "desc"
-          ],
-          "overflow": "min-height",
-          "fields": [
-            "_type",
-            "message.timestamp",
-            "message.original_string"
-          ],
-          "highlight": [],
-          "sortable": true,
-          "header": true,
-          "paging": true,
-          "field_list": false,
-          "all_fields": false,
-          "trimFactor": 400,
-          "localTime": true,
-          "timeField": "message.timestamp",
-          "spyable": true,
-          "queries": {
-            "mode": "selected",
-            "ids": [
-              2
-            ]
-          },
-          "locked": false,
-          "style": {
-            "font-size": "9pt"
-          },
-          "normTimes": true,
-          "title": "Events"
-        }
-      ],
-      "notice": false
-    },
-    {
-      "title": "PCAP Data",
-      "height": "150px",
-      "editable": true,
-      "collapse": false,
-      "collapsable": true,
-      "panels": [
-        {
-          "error": false,
-          "span": 12,
-          "editable": true,
-          "type": "pcap",
-          "loadingEditor": false,
-          "title": "PCAP Data"
-        },
-        {
-          "error": false,
-          "span": 12,
-          "editable": true,
-          "type": "table",
-          "loadingEditor": false,
-          "size": 100,
-          "pages": 5,
-          "offset": 0,
-          "sort": [
-            "_score",
-            "desc"
-          ],
-          "overflow": "min-height",
-          "fields": [],
-          "highlight": [],
-          "sortable": true,
-          "header": true,
-          "paging": true,
-          "field_list": true,
-          "all_fields": false,
-          "trimFactor": 300,
-          "localTime": false,
-          "timeField": "@timestamp",
-          "spyable": true,
-          "queries": {
-            "mode": "selected",
-            "ids": [
-              1
-            ]
-          },
-          "style": {
-            "font-size": "9pt"
-          },
-          "normTimes": true,
-          "title": "Yaf"
-        }
-      ],
-      "notice": false
-    }
-  ],
-  "editable": true,
-  "failover": false,
-  "index": {
-    "interval": "none",
-    "pattern": "[logstash-]YYYY.MM.DD",
-    "default": "*alert*,*_index*",
-    "warm_fields": false
-  },
-  "style": "dark",
-  "realtime": true,
-  "panel_hints": true,
-  "pulldowns": [
-    {
-      "type": "query",
-      "collapse": false,
-      "notice": false,
-      "enable": true,
-      "query": "*",
-      "pinned": true,
-      "history": [
-        "_type:sourcefire_doc",
-        "_type:bro_doc",
-        "_type:*_doc",
-        "_type:sourcefire_alert",
-        "_type:bro_alert",
-        "_type:*_alert",
-        "*"
-      ],
-      "remember": 10
-    },
-    {
-      "type": "filtering",
-      "collapse": true,
-      "notice": true,
-      "enable": true
-    }
-  ],
-  "nav": [
-    {
-      "type": "timepicker",
-      "collapse": false,
-      "notice": false,
-      "enable": true,
-      "status": "Stable",
-      "time_options": [
-        "5m",
-        "15m",
-        "1h",
-        "6h",
-        "12h",
-        "24h",
-        "2d",
-        "7d",
-        "30d"
-      ],
-      "refresh_intervals": [
-        "5s",
-        "10s",
-        "30s",
-        "1m",
-        "5m",
-        "15m",
-        "30m",
-        "1h",
-        "2h",
-        "1d"
-      ],
-      "timefield": "timestamp",
-      "now": true,
-      "filter_id": 0
-    }
-  ],
-  "loader": {
-    "save_gist": false,
-    "save_elasticsearch": true,
-    "save_local": true,
-    "save_default": true,
-    "save_temp": true,
-    "save_temp_ttl_enable": true,
-    "save_temp_ttl": "30d",
-    "load_gist": false,
-    "load_elasticsearch": true,
-    "load_elasticsearch_size": 20,
-    "load_local": false,
-    "hide": false
-  },
-  "refresh": false
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/c737aa9d/deployment/roles/metron_ui/files/opensoc-ui
----------------------------------------------------------------------
diff --git a/deployment/roles/metron_ui/files/opensoc-ui b/deployment/roles/metron_ui/files/opensoc-ui
deleted file mode 100644
index 16e3dbe..0000000
--- a/deployment/roles/metron_ui/files/opensoc-ui
+++ /dev/null
@@ -1,6 +0,0 @@
-{
-  "auth":false,
-  "secrent":"secret",
-  "elasticsearch": { "url": "http://host:port" },
-  "pcap": { "url": "http://host:port/pcap/pcapGetter","mock": false }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/c737aa9d/deployment/roles/metron_ui/tasks/main.yml
----------------------------------------------------------------------
diff --git a/deployment/roles/metron_ui/tasks/main.yml b/deployment/roles/metron_ui/tasks/main.yml
index eb2220d..861abbd 100644
--- a/deployment/roles/metron_ui/tasks/main.yml
+++ b/deployment/roles/metron_ui/tasks/main.yml
@@ -23,46 +23,33 @@
       - { package: 'nodejs'}
       - { package: 'npm'}
 
-- name: Download nvm installer
-  get_url: url=https://raw.githubusercontent.com/creationix/nvm/v0.30.1/install.sh dest=/tmp/nvm-install.sh
-
-- name: Change nvm installer permissions
-  file: path=/tmp/nvm-install.sh mode=0755
-
-- name: Install nvm
-  shell: /tmp/nvm-install.sh creates=~/.nvm/nvm.sh
-
-- name: Install Metron UI
-  shell: source ~/.bashrc && nvm install 0.10.35 && nvm use 0.10.35 && nvm alias default 0.10.35 && npm install -g opensoc-ui creates=/root/.nvm/v0.10.35/lib/node_modules/opensoc-ui/package.json
-
-- name: Add Metron UI config
+- name: Copy Metron UI source
   copy:
-    src: opensoc-ui
-    dest: ~/.opensoc-ui
-    mode: 0644
+    src: "{{ playbook_dir }}/../../metron-ui"
+    dest: "{{ metron_directory }}"
 
-- name: Configure opensoc-ui
-  lineinfile: >
-    dest=~/.opensoc-ui
+- name: Configure Metron UI
+  lineinfile:
+    dest="{{ metron_directory }}/metron-ui/config.json"
     regexp="{{ item.regexp }}"
     line="{{ item.line }}"
     state=present
   with_items:
     - { regexp: '"elasticsearch":', line: '"elasticsearch": { "url": "http://{{ groups.search[0] }}:{{ elasticsearch_web_port }}" },' }
-    - { regexp: '"pcap":', line: '  "pcap": { "url": "http://{{ groups.web[0] }}:{{ pcapservice_port }}/pcap/pcapGetter","mock": false }' }
+    - { regexp: '"pcap":', line: '  "pcap": { "url": "http://{{ groups.web[0] }}:{{ pcapservice_port }}/pcapGetter","mock": false }' }
 
-- name: Fix pcap_all error
-  replace:
-    dest: /root/.nvm/v0.10.35/lib/node_modules/opensoc-ui/lib/static_dist/app/panels/pcap/module.js
-    regexp: "pcap_all"
-    replace: "pcap_*"
-    backup: yes
+- name: Install Node dependencies
+  npm:
+    name: "{{ item }}"
+    path: "{{ metron_ui_directory }}"
+    global: true
+  with_items:
+    - pm2
 
-- name: Configure default dashboard
-  copy:
-    src: default.json
-    dest: /root/.nvm/v0.10.35/lib/node_modules/opensoc-ui/lib/static_dist/app/dashboards/
-    mode: 0644
+- name: Install Metron UI
+  npm:
+    path: "{{ metron_ui_directory }}"
+    production: no
 
-- name: Start Metron UI service
-  shell: source ~/.bashrc && opensoc-ui start
+- name: Start Metron UI
+  shell: "pm2 start /usr/metron/0.1BETA/metron-ui/lib/metron-ui.js --name metron"

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/c737aa9d/deployment/roles/metron_ui/vars/main.yml
----------------------------------------------------------------------
diff --git a/deployment/roles/metron_ui/vars/main.yml b/deployment/roles/metron_ui/vars/main.yml
index ff647ae..ea41dfa 100644
--- a/deployment/roles/metron_ui/vars/main.yml
+++ b/deployment/roles/metron_ui/vars/main.yml
@@ -17,5 +17,6 @@
 ---
 metron_version: 0.1BETA
 metron_directory: /usr/metron/{{ metron_version }}
+metron_ui_directory: "{{ metron_directory }}/metron-ui"
 metron_jar_url: http://192.168.1.174:8080/repository/internal/com/opensoc/OpenSOC-Topologies/{{ metron_version }}/OpenSOC-Topologies-{{ metron_version }}.jar
 metron_source_url: https://github.com/merrimanr/incubator-metron/archive/master.zip

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/c737aa9d/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 baa3bbd..a046801 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
@@ -25,6 +25,7 @@ import org.apache.commons.lang.StringUtils;
 
 import com.google.common.collect.BiMap;
 import com.google.common.collect.HashBiMap;
+import org.json.simple.JSONObject;
 
 /**
  * The Class PcapUtils.
@@ -265,6 +266,30 @@ public class PcapUtils {
     return sb.toString();
   }
 
+  public static String getSessionKey(JSONObject message) {
+    String srcIp = (String) message.get("ip_src_addr");
+    String dstIp = (String) message.get("ip_dst_addr");
+    Long protocol = (Long) message.get("ip_protocol");
+    Long srcPort = (Long) message.get("ip_src_port");
+    Long dstPort = (Long) message.get("ip_dst_port");
+    Long ipId = (Long) message.get("ip_id");
+    String ipIdString = ipId == null ? null : ipId.toString();
+    Long fragmentOffset = (Long) message.get("frag_offset");
+    String fragmentOffsetString = fragmentOffset == null ? null : fragmentOffset.toString();
+    return PcapUtils.getSessionKey(srcIp, dstIp, protocol.toString(), srcPort.toString(), dstPort.toString(), ipIdString, fragmentOffsetString);
+  }
+
+  public static String getPartialSessionKey(String srcIp, String dstIp,
+                                            String protocol, String srcPort, String dstPort) {
+    StringBuffer sb = new StringBuffer(40);
+    sb.append(convertIpv4IpToHex(srcIp)).append(SESSION_KEY_SEPERATOR)
+            .append(convertIpv4IpToHex(dstIp)).append(SESSION_KEY_SEPERATOR)
+            .append(protocol == null ? "0" : protocol)
+            .append(SESSION_KEY_SEPERATOR).append(srcPort == null ? "0" : srcPort)
+            .append(SESSION_KEY_SEPERATOR).append(dstPort == null ? "0" : dstPort);
+    return sb.toString();
+  }
+
   /**
    * Gets the session key.
    * 

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/c737aa9d/metron-streaming/Metron-Common/src/main/java/org/apache/metron/writer/PcapWriter.java
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-Common/src/main/java/org/apache/metron/writer/PcapWriter.java b/metron-streaming/Metron-Common/src/main/java/org/apache/metron/writer/PcapWriter.java
index b5ab587..3320bda 100644
--- a/metron-streaming/Metron-Common/src/main/java/org/apache/metron/writer/PcapWriter.java
+++ b/metron-streaming/Metron-Common/src/main/java/org/apache/metron/writer/PcapWriter.java
@@ -18,6 +18,7 @@
 package org.apache.metron.writer;
 
 import backtype.storm.tuple.Tuple;
+import org.apache.metron.pcap.PcapUtils;
 import org.json.simple.JSONObject;
 
 import java.util.HashMap;
@@ -34,7 +35,7 @@ public class PcapWriter extends HBaseWriter {
 
   @Override
   public byte[] getKey(Tuple tuple, JSONObject message) {
-    String key = (String) message.get("pcap_id");
+    String key = PcapUtils.getSessionKey(message);
     return key.getBytes();
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/c737aa9d/metron-streaming/Metron-MessageParsers/src/main/java/org/apache/metron/parsing/parsers/PcapParser.java
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-MessageParsers/src/main/java/org/apache/metron/parsing/parsers/PcapParser.java b/metron-streaming/Metron-MessageParsers/src/main/java/org/apache/metron/parsing/parsers/PcapParser.java
index c5677f3..bfd943e 100644
--- a/metron-streaming/Metron-MessageParsers/src/main/java/org/apache/metron/parsing/parsers/PcapParser.java
+++ b/metron-streaming/Metron-MessageParsers/src/main/java/org/apache/metron/parsing/parsers/PcapParser.java
@@ -42,6 +42,7 @@ import java.io.File;
 import java.io.IOException;
 import java.io.Serializable;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.List;
 
 public class PcapParser implements MessageParser<JSONObject>, Serializable {
@@ -96,7 +97,13 @@ public class PcapParser implements MessageParser<JSONObject>, Serializable {
 
   @Override
   public boolean validate(JSONObject message) {
-    return true;
+    List<String> requiredFields = Arrays.asList("ip_src_addr",
+            "ip_dst_addr",
+            "ip_protocol",
+            "ip_src_port",
+            "ip_dst_port");
+    return message.keySet().containsAll(requiredFields);
+
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/c737aa9d/metron-streaming/Metron-Pcap_Service/pom.xml
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-Pcap_Service/pom.xml b/metron-streaming/Metron-Pcap_Service/pom.xml
index 441a65d..bb850ae 100644
--- a/metron-streaming/Metron-Pcap_Service/pom.xml
+++ b/metron-streaming/Metron-Pcap_Service/pom.xml
@@ -114,15 +114,24 @@
 			<version>2.3</version>
 		</dependency>
 		<dependency>
+			<groupId>com.google.guava</groupId>
+			<artifactId>guava</artifactId>
+			<version>${global_hbase_guava_version}</version>
+		</dependency>
+		<dependency>
 			<groupId>org.apache.hbase</groupId>
 			<artifactId>hbase-client</artifactId>
 			<version>${global_hbase_version}</version>
-			<scope>provided</scope>
+			<!--scope>provided</scope-->
 			<exclusions>
 				<exclusion>
 					<groupId>org.slf4j</groupId>
 					<artifactId>slf4j-log4j12</artifactId>
 				</exclusion>
+				<!--exclusion>
+					<groupId>com.google.guava</groupId>
+					<artifactId>guava</artifactId>
+				</exclusion-->
 			</exclusions>
 		</dependency>
 		<dependency>
@@ -141,15 +150,25 @@
 			<groupId>org.apache.hadoop</groupId>
 			<artifactId>hadoop-common</artifactId>
 			<version>${global_hadoop_version}</version>
-			<scope>provided</scope>
+			<!--scope>provided</scope-->
 		</dependency>
 		<dependency>
 			<groupId>org.apache.hadoop</groupId>
 			<artifactId>hadoop-hdfs</artifactId>
 			<version>${global_hadoop_version}</version>
-			<scope>provided</scope>
+			<!--scope>provided</scope-->
+		</dependency>
+		<dependency>
+			<groupId>org.apache.hadoop</groupId>
+			<artifactId>hadoop-client</artifactId>
+			<version>${global_hadoop_version}</version>
+			<exclusions>
+				<exclusion>
+					<groupId>org.slf4j</groupId>
+					<artifactId>slf4j-log4j12</artifactId>
+				</exclusion>
+			</exclusions>
 		</dependency>
-
 		<dependency>
 			<groupId>org.springframework.integration</groupId>
 			<artifactId>spring-integration-http</artifactId>

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/c737aa9d/metron-streaming/Metron-Pcap_Service/src/main/java/org/apache/metron/pcapservice/HBaseConfigurationUtil.java
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-Pcap_Service/src/main/java/org/apache/metron/pcapservice/HBaseConfigurationUtil.java b/metron-streaming/Metron-Pcap_Service/src/main/java/org/apache/metron/pcapservice/HBaseConfigurationUtil.java
index f7457fb..75932ab 100644
--- a/metron-streaming/Metron-Pcap_Service/src/main/java/org/apache/metron/pcapservice/HBaseConfigurationUtil.java
+++ b/metron-streaming/Metron-Pcap_Service/src/main/java/org/apache/metron/pcapservice/HBaseConfigurationUtil.java
@@ -80,7 +80,7 @@ public class HBaseConfigurationUtil {
       if (connectionAvailable()) {
         return;
       }
-      clusterConnection = HConnectionManager.createConnection(read());
+      clusterConnection = HConnectionManager.createConnection(HBaseConfiguration.create());
       addShutdownHook();
       System.out.println("Created HConnection and added shutDownHook");
     } catch (IOException e) {

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/c737aa9d/metron-streaming/Metron-Pcap_Service/src/main/java/org/apache/metron/pcapservice/PcapReceiverImplRestEasy.java
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-Pcap_Service/src/main/java/org/apache/metron/pcapservice/PcapReceiverImplRestEasy.java b/metron-streaming/Metron-Pcap_Service/src/main/java/org/apache/metron/pcapservice/PcapReceiverImplRestEasy.java
index ae59643..e747350 100644
--- a/metron-streaming/Metron-Pcap_Service/src/main/java/org/apache/metron/pcapservice/PcapReceiverImplRestEasy.java
+++ b/metron-streaming/Metron-Pcap_Service/src/main/java/org/apache/metron/pcapservice/PcapReceiverImplRestEasy.java
@@ -222,7 +222,7 @@ public class PcapReceiverImplRestEasy {
 	
 	    PcapsResponse response = null;
 	    try {
-	      String sessionKey = PcapUtils.getSessionKey(srcIp, dstIp, protocol,
+	      String sessionKey = PcapUtils.getPartialSessionKey(srcIp, dstIp, protocol,
 	          srcPort, dstPort);
 	      LOGGER.info("sessionKey =" + sessionKey);
 	      IPcapGetter pcapGetter = PcapGetterHBaseImpl.getInstance();

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/c737aa9d/metron-streaming/Metron-Pcap_Service/src/main/java/org/apache/metron/pcapservice/RestTestingUtil.java
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-Pcap_Service/src/main/java/org/apache/metron/pcapservice/RestTestingUtil.java b/metron-streaming/Metron-Pcap_Service/src/main/java/org/apache/metron/pcapservice/RestTestingUtil.java
index aa6fb29..e452b89 100644
--- a/metron-streaming/Metron-Pcap_Service/src/main/java/org/apache/metron/pcapservice/RestTestingUtil.java
+++ b/metron-streaming/Metron-Pcap_Service/src/main/java/org/apache/metron/pcapservice/RestTestingUtil.java
@@ -223,6 +223,80 @@ public class RestTestingUtil {
 
   }
 
+  private static void getPcapsByIdentifiers(String srcIp, String dstIp, String protocol, String srcPort, String dstPort) {
+    System.out
+            .println("**********************getPcapsByKeysRange ******************************************************************************************");
+    // 1.
+    String url = "http://" + hostName
+            + "/pcapGetter/getPcapsByIdentifiers?srcIp={srcIp}"
+            + "&dstIp={dstIp}" + "&protocol={protocol}" + "&srcPort={srcPort}"
+            + "&dstPort={dstPort}";
+    // default values
+    String startTime = "-1";
+    String endTime = "-1";
+    String maxResponseSize = "6";
+    @SuppressWarnings("rawtypes")
+    Map map = new HashMap();
+    map.put("srcIp", srcIp);
+    map.put("dstIp", dstIp);
+    map.put("protocol", protocol);
+    map.put("srcPort", srcPort);
+    map.put("dstPort", dstPort);
+
+    RestTemplate template = new RestTemplate();
+
+    // set headers and entity to send
+    HttpHeaders headers = new HttpHeaders();
+    headers.set("Accept", MediaType.APPLICATION_OCTET_STREAM_VALUE);
+    HttpEntity<Object> requestEntity = new HttpEntity<Object>(headers);
+
+    // 1.
+    ResponseEntity<byte[]> response1 = template.exchange(url, HttpMethod.GET,
+            requestEntity, byte[].class, map);
+    System.out
+            .println("----------------------------------------------------------------------------------------------------");
+    System.out
+            .format(
+                    "getPcapsByIdentifiers : request= <srcIp=%s; dstIp=%s; protocol=%s; srcPort=%s; dstPort=%s> \n response= %s \n",
+                    srcIp, dstIp, protocol, endTime, srcPort, dstPort, response1);
+    System.out
+            .println("----------------------------------------------------------------------------------------------------");
+    System.out.println();
+
+    // 2. with time range
+    startTime = System.getProperty("startTime", "-1");
+    endTime = System.getProperty("endTime", "-1");
+    map.put("startTime", startTime);
+    map.put("endTime", endTime);
+    ResponseEntity<byte[]> response2 = template.exchange(url, HttpMethod.GET,
+            requestEntity, byte[].class, map);
+    System.out
+            .println("----------------------------------------------------------------------------------------------------");
+    System.out
+            .format(
+                    "getPcapsByIdentifiers : request= <srcIp=%s; dstIp=%s; protocol=%s; srcPort=%s; dstPort=%s> \n response= %s \n",
+                    srcIp, dstIp, protocol, endTime, srcPort, dstPort, response2);
+    System.out
+            .println("----------------------------------------------------------------------------------------------------");
+    System.out.println();
+
+    // 3. with maxResponseSize
+    maxResponseSize = System.getProperty("maxResponseSize", "6");
+    map.put("maxResponseSize", maxResponseSize);
+    ResponseEntity<byte[]> response3 = template.exchange(url, HttpMethod.GET,
+            requestEntity, byte[].class, map);
+    System.out
+            .println("----------------------------------------------------------------------------------------------------");
+    System.out
+            .format(
+                    "getPcapsByIdentifiers : request= <srcIp=%s; dstIp=%s; protocol=%s; srcPort=%s; dstPort=%s> \n response= %s \n",
+                    srcIp, dstIp, protocol, endTime, srcPort, dstPort, response3);
+    System.out
+            .println("----------------------------------------------------------------------------------------------------");
+    System.out.println();
+
+  }
+
   /**
    * The main method.
    * 

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/c737aa9d/metron-streaming/Metron-Pcap_Service/src/main/java/org/apache/metron/pcapservice/rest/PcapService.java
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-Pcap_Service/src/main/java/org/apache/metron/pcapservice/rest/PcapService.java b/metron-streaming/Metron-Pcap_Service/src/main/java/org/apache/metron/pcapservice/rest/PcapService.java
index d11113a..d3bbe24 100644
--- a/metron-streaming/Metron-Pcap_Service/src/main/java/org/apache/metron/pcapservice/rest/PcapService.java
+++ b/metron-streaming/Metron-Pcap_Service/src/main/java/org/apache/metron/pcapservice/rest/PcapService.java
@@ -19,6 +19,7 @@ package org.apache.metron.pcapservice.rest;
 
 import java.io.IOException;
 
+import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.eclipse.jetty.server.Server;
 import org.eclipse.jetty.servlet.ServletContextHandler;
 import org.eclipse.jetty.servlet.ServletHolder;

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/c737aa9d/metron-streaming/Metron-Pcap_Service/src/main/resources/hbase-config-default.properties
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-Pcap_Service/src/main/resources/hbase-config-default.properties b/metron-streaming/Metron-Pcap_Service/src/main/resources/hbase-config-default.properties
index 912485c..0f47193 100644
--- a/metron-streaming/Metron-Pcap_Service/src/main/resources/hbase-config-default.properties
+++ b/metron-streaming/Metron-Pcap_Service/src/main/resources/hbase-config-default.properties
@@ -25,7 +25,7 @@ zookeeper.recovery.retry=0
 #hbase table configuration
 hbase.table.name=pcap
 hbase.table.column.family=t
-hbase.table.column.qualifier=pcap
+hbase.table.column.qualifier=value
 hbase.table.column.maxVersions=5
 
 # scan size limit configuration in MB or KB; if the input is negative or greater than max value throw an error.

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/c737aa9d/metron-streaming/Metron-Testing/pom.xml
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-Testing/pom.xml b/metron-streaming/Metron-Testing/pom.xml
index 5f1c946..7498375 100644
--- a/metron-streaming/Metron-Testing/pom.xml
+++ b/metron-streaming/Metron-Testing/pom.xml
@@ -105,6 +105,16 @@
         </exclusion>
       </exclusions>
     </dependency>
+    <dependency>
+      <groupId>org.apache.metron</groupId>
+      <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/c737aa9d/metron-streaming/Metron-Testing/src/main/java/org/apache/metron/integration/util/integration/components/KafkaWithZKComponent.java
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-Testing/src/main/java/org/apache/metron/integration/util/integration/components/KafkaWithZKComponent.java b/metron-streaming/Metron-Testing/src/main/java/org/apache/metron/integration/util/integration/components/KafkaWithZKComponent.java
index 83ecd42..54b4f27 100644
--- a/metron-streaming/Metron-Testing/src/main/java/org/apache/metron/integration/util/integration/components/KafkaWithZKComponent.java
+++ b/metron-streaming/Metron-Testing/src/main/java/org/apache/metron/integration/util/integration/components/KafkaWithZKComponent.java
@@ -218,7 +218,7 @@ public class KafkaWithZKComponent implements InMemoryComponent {
     }
   }
 
-  public void writeMessages(String topic, List<byte[]> messages) {
+  public void writeMessages(String topic, Collection<byte[]> messages) {
     KafkaProducer<String, byte[]> kafkaProducer = createProducer();
     for(byte[] message: messages) {
       kafkaProducer.send(new ProducerRecord<String, byte[]>(topic, message));

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/c737aa9d/metron-streaming/Metron-Testing/src/main/java/org/apache/metron/integration/util/integration/util/PcapTestUtil.java
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-Testing/src/main/java/org/apache/metron/integration/util/integration/util/PcapTestUtil.java b/metron-streaming/Metron-Testing/src/main/java/org/apache/metron/integration/util/integration/util/PcapTestUtil.java
new file mode 100644
index 0000000..52c8288
--- /dev/null
+++ b/metron-streaming/Metron-Testing/src/main/java/org/apache/metron/integration/util/integration/util/PcapTestUtil.java
@@ -0,0 +1,77 @@
+/**
+ * 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.util;
+
+import kafka.api.FetchRequest;
+import kafka.api.FetchRequestBuilder;
+import kafka.javaapi.FetchResponse;
+import kafka.javaapi.consumer.SimpleConsumer;
+import kafka.message.MessageAndOffset;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.hadoop.io.BytesWritable;
+import org.apache.hadoop.io.IntWritable;
+import org.apache.hadoop.io.SequenceFile;
+import org.apache.hadoop.io.SequenceFile.Writer;
+import org.apache.metron.parsing.parsers.PcapParser;
+import org.json.simple.JSONObject;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Iterator;
+import java.util.List;
+
+public class PcapTestUtil {
+
+  public static final String OUTPUT_PATH = "./metron-streaming/Metron-Topologies/src/main/resources/SampleInput/PCAPExampleOutputTest";
+
+  public static void main(String[] args) throws IOException {
+    String topic = "pcap";
+    SimpleConsumer consumer = new SimpleConsumer("node1", 6667, 100000, 64 * 1024, "consumer");
+    FetchRequest req = new FetchRequestBuilder()
+            .clientId("consumer")
+            .addFetch(topic, 0, 0, 100000)
+            .build();
+    FetchResponse fetchResponse = consumer.fetch(req);
+    Iterator<MessageAndOffset> results = fetchResponse.messageSet(topic, 0).iterator();
+    Writer writer = SequenceFile.createWriter(new Configuration(),
+            Writer.file(new Path(OUTPUT_PATH)),
+            Writer.compression(SequenceFile.CompressionType.NONE),
+            Writer.keyClass(IntWritable.class),
+            Writer.valueClass(BytesWritable.class));
+    int index = 0;
+    int size = 20;
+    PcapParser pcapParser = new PcapParser();
+    pcapParser.init();
+    while(results.hasNext()) {
+      if (index == size) break;
+      ByteBuffer payload = results.next().message().payload();
+      byte[] bytes = new byte[payload.limit()];
+      payload.get(bytes);
+      List<JSONObject> parsed = pcapParser.parse(bytes);
+      if (parsed != null && parsed.size() > 0) {
+        JSONObject message = parsed.get(0);
+        if (pcapParser.validate(message)) {
+          writer.append(new IntWritable(index++), new BytesWritable(bytes));
+        }
+      }
+    }
+    writer.close();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/c737aa9d/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 9f66a95..5d36ef1 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
@@ -17,8 +17,8 @@
 
 ##### Kafka #####
 
-kafka.zk=zkpr1:2181,zkpr2:2181,zkpr3:2181
-kafka.broker=kfka1:6667
+kafka.zk=node1:2181
+kafka.broker=node1:6667
 spout.kafka.topic.asa=asa
 spout.kafka.topic.bro=bro
 spout.kafka.topic.fireeye=fireeye
@@ -89,7 +89,7 @@ bolt.hdfs.compression.codec.class=org.apache.hadoop.io.compress.SnappyCodec
 index.hdfs.output=/tmp/metron/enriched
 
 ##### HBase #####
-bolt.hbase.table.name=pcap_test
+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

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/c737aa9d/metron-streaming/Metron-Topologies/src/main/resources/Metron_Configs/topologies/pcap/local.yaml
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-Topologies/src/main/resources/Metron_Configs/topologies/pcap/local.yaml b/metron-streaming/Metron-Topologies/src/main/resources/Metron_Configs/topologies/pcap/local.yaml
deleted file mode 100644
index 3987a18..0000000
--- a/metron-streaming/Metron-Topologies/src/main/resources/Metron_Configs/topologies/pcap/local.yaml
+++ /dev/null
@@ -1,440 +0,0 @@
-# Licensed to the Apache Software Foundation (ASF) under one
-# or more contributor license agreements.  See the NOTICE file
-# distributed with this work for additional information
-# regarding copyright ownership.  The ASF licenses this file
-# to you under the Apache License, Version 2.0 (the
-# "License"); you may not use this file except in compliance
-# with the License.  You may obtain a copy of the License at
-#
-#     http://www.apache.org/licenses/LICENSE-2.0
-#
-# Unless required by applicable law or agreed to in writing, software
-# distributed under the License is distributed on an "AS IS" BASIS,
-# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-# See the License for the specific language governing permissions and
-# limitations under the License.
-
-name: "pcap-local"
-config:
-    topology.workers: 1
-
-components:
-# Parser
-    -   id: "parser"
-        className: "org.apache.metron.parsing.parsers.PcapParser"
-        configMethods:
-            -   name: "withTsPrecision"
-                args: ["MICRO"]
-# Threat Intel
-    -   id: "ipThreatIntelConfig"
-        className: "org.apache.metron.threatintel.ThreatIntelConfig"
-        configMethods:
-            -   name: "withProviderImpl"
-                args:
-                    - "${hbase.provider.impl}"
-            -   name: "withTrackerHBaseTable"
-                args:
-                    - "${threat.intel.tracker.table}"
-            -   name: "withTrackerHBaseCF"
-                args:
-                    - "${threat.intel.tracker.cf}"
-            -   name: "withHBaseTable"
-                args:
-                    - "${threat.intel.ip.table}"
-            -   name: "withHBaseCF"
-                args:
-                    - "${threat.intel.ip.cf}"
-    -   id: "ipThreatIntelAdapter"
-        className: "org.apache.metron.threatintel.ThreatIntelAdapter"
-        configMethods:
-           -    name: "withConfig"
-                args:
-                    - ref: "ipThreatIntelConfig"
-    -   id: "ipThreatIntelEnrichment"
-        className: "org.apache.metron.domain.Enrichment"
-        properties:
-           - name: "type"
-             value: "ip"
-           - name: "fields"
-             value: ["message/ip_src_addr", "message/ip_dst_addr"]
-           - name: "adapter"
-             ref: "ipThreatIntelAdapter"
-    -   id: "threatIntels"
-        className: "java.util.ArrayList"
-        configMethods:
-            -   name: "add"
-                args:
-                    - ref: "ipThreatIntelEnrichment"
-# Enrichment
-    -   id: "geoEnrichmentAdapter"
-        className: "org.apache.metron.integration.util.mock.MockGeoAdapter"
-    -   id: "geoEnrichment"
-        className: "org.apache.metron.domain.Enrichment"
-        properties:
-            -   name: "type"
-                value:  "geo"
-            -   name: "fields"
-                value: ["ip_src_addr", "ip_dst_addr"]
-            -   name: "adapter"
-                ref: "geoEnrichmentAdapter"
-    -   id: "hostEnrichmentAdapter"
-        className: "org.apache.metron.enrichment.adapters.host.HostFromJSONListAdapter"
-        constructorArgs:
-            - '${org.apache.metron.enrichment.host.known_hosts}'
-    -   id: "hostEnrichment"
-        className: "org.apache.metron.domain.Enrichment"
-        properties:
-            -   name: "type"
-                value:  "host"
-            -   name: "fields"
-                value: ["ip_src_addr", "ip_dst_addr"]
-            -   name: "adapter"
-                ref: "hostEnrichmentAdapter"
-    -   id: "enrichments"
-        className: "java.util.ArrayList"
-        configMethods:
-            -   name: "add"
-                args:
-                    - ref: "geoEnrichment"
-            -   name: "add"
-                args:
-                    - ref: "hostEnrichment"
-#indexing
-    -   id: "indexAdapter"
-        className: "org.apache.metron.indexing.adapters.ESTimedRotatingAdapter"
-    -   id: "metricConfig"
-        className: "org.apache.commons.configuration.BaseConfiguration"
-        configMethods:
-            -   name: "setProperty"
-                args:
-                    - "org.apache.metron.metrics.reporter.graphite"
-                    - "${org.apache.metron.metrics.reporter.graphite}"
-            -   name: "setProperty"
-                args:
-                    - "org.apache.metron.metrics.reporter.console"
-                    - "${org.apache.metron.metrics.reporter.console}"
-            -   name: "setProperty"
-                args:
-                    - "org.apache.metron.metrics.reporter.jmx"
-                    - "${org.apache.metron.metrics.reporter.jmx}"
-            -   name: "setProperty"
-                args:
-                    - "org.apache.metron.metrics.graphite.address"
-                    - "${org.apache.metron.metrics.graphite.address}"
-            -   name: "setProperty"
-                args:
-                    - "org.apache.metron.metrics.graphite.port"
-                    - "${org.apache.metron.metrics.graphite.port}"
-            -   name: "setProperty"
-                args:
-                    - "org.apache.metron.metrics.TelemetryParserBolt.acks"
-                    - "${org.apache.metron.metrics.TelemetryParserBolt.acks}"
-            -   name: "setProperty"
-                args:
-                    - "org.apache.metron.metrics.TelemetryParserBolt.emits"
-                    - "${org.apache.metron.metrics.TelemetryParserBolt.emits}"
-            -   name: "setProperty"
-                args:
-                    - "org.apache.metron.metrics.TelemetryParserBolt.fails"
-                    - "${org.apache.metron.metrics.TelemetryParserBolt.fails}"
-            -   name: "setProperty"
-                args:
-                    - "org.apache.metron.metrics.GenericEnrichmentBolt.acks"
-                    - "${org.apache.metron.metrics.GenericEnrichmentBolt.acks}"
-            -   name: "setProperty"
-                args:
-                    - "org.apache.metron.metrics.GenericEnrichmentBolt.emits"
-                    - "${org.apache.metron.metrics.GenericEnrichmentBolt.emits}"
-            -   name: "setProperty"
-                args:
-                    - "org.apache.metron.metrics.GenericEnrichmentBolt.fails"
-                    - "${org.apache.metron.metrics.GenericEnrichmentBolt.fails}"
-            -   name: "setProperty"
-                args:
-                    - "org.apache.metron.metrics.TelemetryIndexingBolt.acks"
-                    - "${org.apache.metron.metrics.TelemetryIndexingBolt.acks}"
-            -   name: "setProperty"
-                args:
-                    - "org.apache.metron.metrics.TelemetryIndexingBolt.emits"
-                    - "${org.apache.metron.metrics.TelemetryIndexingBolt.emits}"
-            -   name: "setProperty"
-                args:
-                    - "org.apache.metron.metrics.TelemetryIndexingBolt.fails"
-                    - "${org.apache.metron.metrics.TelemetryIndexingBolt.fails}"
-#hbase bolt
-    -   id: "hbaseConfig"
-        className: "org.apache.metron.hbase.TupleTableConfig"
-        configMethods:
-            -   name: "withFields"
-                args:
-                    - "${bolt.hbase.table.fields}"
-            -   name: "withTable"
-                args:
-                    - "${bolt.hbase.table.name}"
-            -   name: "withRowKeyField"
-                args:
-                    - "${bolt.hbase.table.key.tuple.field.name}"
-            -   name: "withTimestampField"
-                args:
-                    - "${bolt.hbase.table.timestamp.tuple.field.name}"
-            -   name: "withBatch"
-                args:
-                    - ${bolt.hbase.enable.batching}
-            -   name: "withConnectorImpl"
-                args:
-                    - "${hbase.provider.impl}"
-spouts:
-    -   id: "testingSpout"
-        className: "org.apache.metron.test.spouts.GenericInternalTestSpout"
-        parallelism: 1
-        configMethods:
-            -   name: "withBinaryConverter"
-                args:
-                    - "${pcap.binary.converter}"
-            -   name: "withFilename"
-                args:
-                    - "${input.path}SampleInput/PCAPExampleOutput"
-            -   name: "withRepeating"
-                args:
-                    - ${testing.repeating}
-bolts:
-    -   id: "hbaseBolt"
-        className: "org.apache.metron.hbase.HBaseBolt"
-        constructorArgs:
-            - ref: "hbaseConfig"
-            - "${kafka.zk}"
-    -   id: "parserBolt"
-        className: "org.apache.metron.bolt.PcapParserBolt"
-        configMethods:
-            -   name: "withMessageParser"
-                args:
-                    - ref: "parser"
-            -   name: "withEnrichments"
-                args:
-                    - ref: "enrichments"
-    -   id: "indexingBolt"
-        className: "org.apache.metron.indexing.TelemetryIndexingBolt"
-        configMethods:
-            -   name: "withIndexIP"
-                args:
-                    - "${es.ip}"
-            -   name: "withIndexPort"
-                args:
-                    - ${es.port}
-            -   name: "withClusterName"
-                args:
-                    - "${es.clustername}"
-            -   name: "withIndexName"
-                args:
-                    - "pcap_index"
-            -   name: "withIndexTimestamp"
-                args:
-                    - "yyyy.MM.dd.hh"
-            -   name: "withDocumentName"
-                args:
-                    - "pcap_doc"
-            -   name: "withBulk"
-                args:
-                    - 1
-            -   name: "withIndexAdapter"
-                args:
-                    - ref: "indexAdapter"
-            -   name: "withMetricConfiguration"
-                args:
-                    - ref: "metricConfig"
-    -   id: "errorIndexingBolt"
-        className: "org.apache.metron.indexing.TelemetryIndexingBolt"
-        configMethods:
-            -   name: "withIndexIP"
-                args:
-                    - "${es.ip}"
-            -   name: "withIndexPort"
-                args:
-                    - ${es.port}
-            -   name: "withClusterName"
-                args:
-                    - "${es.clustername}"
-            -   name: "withIndexName"
-                args:
-                    - "error"
-            -   name: "withIndexTimestamp"
-                args:
-                    - "yyyy.MM"
-            -   name: "withDocumentName"
-                args:
-                    - "pcap_error"
-            -   name: "withBulk"
-                args:
-                    - 1
-            -   name: "withIndexAdapter"
-                args:
-                    - ref: "indexAdapter"
-            -   name: "withMetricConfiguration"
-                args:
-                    - ref: "metricConfig"
-# Threat Intel Bolts
-    -   id: "threatIntelSplitBolt"
-        className: "org.apache.metron.enrichment.bolt.EnrichmentSplitterBolt"
-        configMethods:
-            -   name: "withEnrichments"
-                args:
-                    - ref: "threatIntels"
-    -   id: "ipThreatIntelBolt"
-        className: "org.apache.metron.enrichment.bolt.GenericEnrichmentBolt"
-        configMethods:
-            -   name: "withEnrichment"
-                args:
-                    - ref: "ipThreatIntelEnrichment"
-            -   name: "withMaxCacheSize"
-                args: [10000]
-            -   name: "withMaxTimeRetain"
-                args: [10]
-    -   id: "threatIntelJoinBolt"
-        className: "org.apache.metron.enrichment.bolt.EnrichmentJoinBolt"
-        configMethods:
-            -   name: "withEnrichments"
-                args:
-                    - ref: "threatIntels"
-            -   name: "withType"
-                args:
-                    - "alerts"
-            -   name: "withMaxCacheSize"
-                args: [10000]
-            -   name: "withMaxTimeRetain"
-                args: [10]
-# Enrichment Bolts
-    -   id: "geoEnrichmentBolt"
-        className: "org.apache.metron.enrichment.bolt.GenericEnrichmentBolt"
-        configMethods:
-            -   name: "withEnrichment"
-                args:
-                    - ref: "geoEnrichment"
-            -   name: "withMaxCacheSize"
-                args: [10000]
-            -   name: "withMaxTimeRetain"
-                args: [10]
-    -   id: "hostEnrichmentBolt"
-        className: "org.apache.metron.enrichment.bolt.GenericEnrichmentBolt"
-        configMethods:
-            -   name: "withEnrichment"
-                args:
-                    - ref: "hostEnrichment"
-            -   name: "withMaxCacheSize"
-                args: [10000]
-            -   name: "withMaxTimeRetain"
-                args: [10]
-    -   id: "joinBolt"
-        className: "org.apache.metron.enrichment.bolt.EnrichmentJoinBolt"
-        configMethods:
-        -   name: "withEnrichments"
-            args:
-                - ref: "enrichments"
-        -   name: "withMaxCacheSize"
-            args: [10000]
-        -   name: "withMaxTimeRetain"
-            args: [10]
-
-streams:
-#parser
-    -   name: "spout -> parser"
-        from: "testingSpout"
-        to: "parserBolt"
-        grouping:
-            type: SHUFFLE
-#hbase
-#    -   name: "parser -> hbase"
-#        from: "parserBolt"
-#        to: "hbaseBolt"
-#        grouping:
-#            streamId: "raw"
-#            type: FIELDS
-#            args: ["key"]
-#enrichment
-    -   name: "parser -> host"
-        from: "parserBolt"
-        to: "hostEnrichmentBolt"
-        grouping:
-            streamId: "host"
-            type: FIELDS
-            args: ["key"]
-    -   name: "parser -> geo"
-        from: "parserBolt"
-        to: "geoEnrichmentBolt"
-        grouping:
-            streamId: "geo"
-            type: FIELDS
-            args: ["key"]
-    -   name: "parser -> join"
-        from: "parserBolt"
-        to: "joinBolt"
-        grouping:
-            streamId: "message"
-            type: FIELDS
-            args: ["key"]
-    -   name: "geo -> join"
-        from: "geoEnrichmentBolt"
-        to: "joinBolt"
-        grouping:
-            streamId: "geo"
-            type: FIELDS
-            args: ["key"]
-    -   name: "host -> join"
-        from: "hostEnrichmentBolt"
-        to: "joinBolt"
-        grouping:
-            streamId: "host"
-            type: FIELDS
-            args: ["key"]
-
-#threat intel
-    -   name: "enrichmentJoin -> threatSplit"
-        from: "joinBolt"
-        to: "threatIntelSplitBolt"
-        grouping:
-            streamId: "message"
-            type: FIELDS
-            args: ["key"]
-
-    -   name: "threatSplit -> ip"
-        from: "threatIntelSplitBolt"
-        to: "ipThreatIntelBolt"
-        grouping:
-            streamId: "ip"
-            type: FIELDS
-            args: ["key"]
-
-    -   name: "ip -> join"
-        from: "ipThreatIntelBolt"
-        to: "threatIntelJoinBolt"
-        grouping:
-            streamId: "ip"
-            type: FIELDS
-            args: ["key"]
-    -   name: "threatIntelSplit -> threatIntelJoin"
-        from: "threatIntelSplitBolt"
-        to: "threatIntelJoinBolt"
-        grouping:
-            streamId: "message"
-            type: FIELDS
-            args: ["key"]
-#indexing
-    -   name: "threatIntelJoin -> indexing"
-        from: "threatIntelJoinBolt"
-        to: "indexingBolt"
-        grouping:
-            streamId: "message"
-            type: FIELDS
-            args: ["key"]
-#errors
-    -   name: "parser -> errors"
-        from: "parserBolt"
-        to: "errorIndexingBolt"
-        grouping:
-            streamId: "error"
-            type: SHUFFLE
-    -   name: "indexing -> errors"
-        from: "indexingBolt"
-        to: "errorIndexingBolt"
-        grouping:
-            streamId: "error"
-            type: SHUFFLE

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

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/c737aa9d/metron-streaming/Metron-Topologies/src/main/resources/Metron_Configs/topologies/pcap/remote.yaml
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-Topologies/src/main/resources/Metron_Configs/topologies/pcap/remote.yaml b/metron-streaming/Metron-Topologies/src/main/resources/Metron_Configs/topologies/pcap/remote.yaml
index 5bdbc17..bfc8527 100644
--- a/metron-streaming/Metron-Topologies/src/main/resources/Metron_Configs/topologies/pcap/remote.yaml
+++ b/metron-streaming/Metron-Topologies/src/main/resources/Metron_Configs/topologies/pcap/remote.yaml
@@ -14,137 +14,18 @@
 # See the License for the specific language governing permissions and
 # limitations under the License.
 
-name: "pcap"
+name: "pcap-parse"
 config:
     topology.workers: 1
 
 components:
-# Parser
     -   id: "parser"
         className: "org.apache.metron.parsing.parsers.PcapParser"
-        configMethods:
-            -   name: "withTsPrecision"
-                args: ["MICRO"]
-# Threat Intel
-    -   id: "ipThreatIntelConfig"
-        className: "org.apache.metron.threatintel.ThreatIntelConfig"
-        configMethods:
-            -   name: "withTrackerHBaseTable"
-                args:
-                    - "${threat.intel.tracker.table}"
-            -   name: "withTrackerHBaseCF"
-                args:
-                    - "${threat.intel.tracker.cf}"
-            -   name: "withHBaseTable"
-                args:
-                    - "${threat.intel.ip.table}"
-            -   name: "withHBaseCF"
-                args:
-                    - "${threat.intel.ip.cf}"
-    -   id: "ipThreatIntelAdapter"
-        className: "org.apache.metron.threatintel.ThreatIntelAdapter"
-        configMethods:
-           -    name: "withConfig"
-                args:
-                    - ref: "ipThreatIntelConfig"
-    -   id: "ipThreatIntelEnrichment"
-        className: "org.apache.metron.domain.Enrichment"
-        properties:
-           - name: "type"
-             value: "ip"
-           - name: "fields"
-             value: ["message/ip_src_addr", "message/ip_dst_addr"]
-           - name: "adapter"
-             ref: "ipThreatIntelAdapter"
-    -   id: "threatIntels"
-        className: "java.util.ArrayList"
-        configMethods:
-            -   name: "add"
-                args:
-                    - ref: "ipThreatIntelEnrichment"
-# Enrichment
-
-    -   id: "hostEnrichmentAdapter"
-        className: "org.apache.metron.enrichment.adapters.host.HostFromJSONListAdapter"
+    -   id: "writer"
+        className: "org.apache.metron.writer.PcapWriter"
         constructorArgs:
-            - '${org.apache.metron.enrichment.host.known_hosts}'
-    -   id: "hostEnrichment"
-        className: "org.apache.metron.domain.Enrichment"
-        properties:
-            -   name: "type"
-                value:  "host"
-            -   name: "fields"
-                value: ["ip_src_addr", "ip_dst_addr"]
-            -   name: "adapter"
-                ref: "hostEnrichmentAdapter"
-    -   id: "enrichments"
-        className: "java.util.ArrayList"
-        configMethods:
-            -   name: "add"
-                args:
-                    - ref: "hostEnrichment"
-#indexing
-    -   id: "indexAdapter"
-        className: "org.apache.metron.indexing.adapters.ESTimedRotatingAdapter"
-    -   id: "metricConfig"
-        className: "org.apache.commons.configuration.BaseConfiguration"
-        configMethods:
-            -   name: "setProperty"
-                args:
-                    - "org.apache.metron.metrics.reporter.graphite"
-                    - "${org.apache.metron.metrics.reporter.graphite}"
-            -   name: "setProperty"
-                args:
-                    - "org.apache.metron.metrics.reporter.console"
-                    - "${org.apache.metron.metrics.reporter.console}"
-            -   name: "setProperty"
-                args:
-                    - "org.apache.metron.metrics.reporter.jmx"
-                    - "${org.apache.metron.metrics.reporter.jmx}"
-            -   name: "setProperty"
-                args:
-                    - "org.apache.metron.metrics.graphite.address"
-                    - "${org.apache.metron.metrics.graphite.address}"
-            -   name: "setProperty"
-                args:
-                    - "org.apache.metron.metrics.graphite.port"
-                    - "${org.apache.metron.metrics.graphite.port}"
-            -   name: "setProperty"
-                args:
-                    - "org.apache.metron.metrics.TelemetryParserBolt.acks"
-                    - "${org.apache.metron.metrics.TelemetryParserBolt.acks}"
-            -   name: "setProperty"
-                args:
-                    - "org.apache.metron.metrics.TelemetryParserBolt.emits"
-                    - "${org.apache.metron.metrics.TelemetryParserBolt.emits}"
-            -   name: "setProperty"
-                args:
-                    - "org.apache.metron.metrics.TelemetryParserBolt.fails"
-                    - "${org.apache.metron.metrics.TelemetryParserBolt.fails}"
-            -   name: "setProperty"
-                args:
-                    - "org.apache.metron.metrics.GenericEnrichmentBolt.acks"
-                    - "${org.apache.metron.metrics.GenericEnrichmentBolt.acks}"
-            -   name: "setProperty"
-                args:
-                    - "org.apache.metron.metrics.GenericEnrichmentBolt.emits"
-                    - "${org.apache.metron.metrics.GenericEnrichmentBolt.emits}"
-            -   name: "setProperty"
-                args:
-                    - "org.apache.metron.metrics.GenericEnrichmentBolt.fails"
-                    - "${org.apache.metron.metrics.GenericEnrichmentBolt.fails}"
-            -   name: "setProperty"
-                args:
-                    - "org.apache.metron.metrics.TelemetryIndexingBolt.acks"
-                    - "${org.apache.metron.metrics.TelemetryIndexingBolt.acks}"
-            -   name: "setProperty"
-                args:
-                    - "org.apache.metron.metrics.TelemetryIndexingBolt.emits"
-                    - "${org.apache.metron.metrics.TelemetryIndexingBolt.emits}"
-            -   name: "setProperty"
-                args:
-                    - "org.apache.metron.metrics.TelemetryIndexingBolt.fails"
-                    - "${org.apache.metron.metrics.TelemetryIndexingBolt.fails}"
+            - "${bolt.hbase.table.name}"
+            - "${bolt.hbase.table.fields}"
     -   id: "zkHosts"
         className: "storm.kafka.ZkHosts"
         constructorArgs:
@@ -165,259 +46,25 @@ components:
                 value: true
             -   name: "startOffsetTime"
                 value: -1
-            -   name: "socketTimeoutMs"
-                value: 1000000
-#hbase bolt
-    -   id: "hbaseConfig"
-        className: "org.apache.metron.hbase.TupleTableConfig"
-        configMethods:
-            -   name: "withFields"
-                args:
-                    - "${bolt.hbase.table.fields}"
-            -   name: "withTable"
-                args:
-                    - "${bolt.hbase.table.name}"
-            -   name: "withRowKeyField"
-                args:
-                    - "${bolt.hbase.table.key.tuple.field.name}"
-            -   name: "withTimestampField"
-                args:
-                    - "${bolt.hbase.table.timestamp.tuple.field.name}"
-            -   name: "withBatch"
-                args:
-                    - ${bolt.hbase.enable.batching}
+
 spouts:
     -   id: "kafkaSpout"
         className: "storm.kafka.KafkaSpout"
         constructorArgs:
             - ref: "kafkaConfig"
+
 bolts:
-    -   id: "hbaseBolt"
-        className: "org.apache.metron.hbase.HBaseBolt"
-        constructorArgs:
-            - ref: "hbaseConfig"
-            - "${kafka.zk}"
     -   id: "parserBolt"
-        className: "org.apache.metron.bolt.PcapParserBolt"
-        constructorArgs:
-            - "${kafka.zk}"
-        configMethods:
-            -   name: "withMessageParser"
-                args:
-                    - ref: "parser"
-            -   name: "withEnrichments"
-                args:
-                    - ref: "enrichments"
-    -   id: "indexingBolt"
-        className: "org.apache.metron.indexing.TelemetryIndexingBolt"
-        constructorArgs:
-            - "${kafka.zk}"
-        configMethods:
-            -   name: "withIndexIP"
-                args:
-                    - "${es.ip}"
-            -   name: "withIndexPort"
-                args:
-                    - ${es.port}
-            -   name: "withClusterName"
-                args:
-                    - "${es.clustername}"
-            -   name: "withIndexName"
-                args:
-                    - "pcap_index"
-            -   name: "withIndexTimestamp"
-                args:
-                    - "yyyy.MM.dd.hh"
-            -   name: "withDocumentName"
-                args:
-                    - "pcap_doc"
-            -   name: "withBulk"
-                args:
-                    - 1
-            -   name: "withIndexAdapter"
-                args:
-                    - ref: "indexAdapter"
-            -   name: "withMetricConfiguration"
-                args:
-                    - ref: "metricConfig"
-    -   id: "errorIndexingBolt"
-        className: "org.apache.metron.indexing.TelemetryIndexingBolt"
-        constructorArgs:
-            - "${kafka.zk}"
-        configMethods:
-            -   name: "withIndexIP"
-                args:
-                    - "${es.ip}"
-            -   name: "withIndexPort"
-                args:
-                    - ${es.port}
-            -   name: "withClusterName"
-                args:
-                    - "${es.clustername}"
-            -   name: "withIndexName"
-                args:
-                    - "error"
-            -   name: "withIndexTimestamp"
-                args:
-                    - "yyyy.MM"
-            -   name: "withDocumentName"
-                args:
-                    - "pcap_error"
-            -   name: "withBulk"
-                args:
-                    - 1
-            -   name: "withIndexAdapter"
-                args:
-                    - ref: "indexAdapter"
-            -   name: "withMetricConfiguration"
-                args:
-                    - ref: "metricConfig"
-# Threat Intel Bolts
-    -   id: "threatIntelSplitBolt"
-        className: "org.apache.metron.enrichment.bolt.EnrichmentSplitterBolt"
-        constructorArgs:
-            - "${kafka.zk}"
-        configMethods:
-            -   name: "withEnrichments"
-                args:
-                    - ref: "threatIntels"
-    -   id: "ipThreatIntelBolt"
-        className: "org.apache.metron.enrichment.bolt.GenericEnrichmentBolt"
-        constructorArgs:
-            - "${kafka.zk}"
-        configMethods:
-            -   name: "withEnrichment"
-                args:
-                    - ref: "ipThreatIntelEnrichment"
-            -   name: "withMaxCacheSize"
-                args: [10000]
-            -   name: "withMaxTimeRetain"
-                args: [10]
-    -   id: "threatIntelJoinBolt"
-        className: "org.apache.metron.enrichment.bolt.EnrichmentJoinBolt"
-        constructorArgs:
-            - "${kafka.zk}"
-        configMethods:
-            -   name: "withEnrichments"
-                args:
-                    - ref: "threatIntels"
-            -   name: "withMaxCacheSize"
-                args: [10000]
-            -   name: "withMaxTimeRetain"
-                args: [10]
-# Enrichment Bolts
-    -   id: "hostEnrichmentBolt"
-        className: "org.apache.metron.enrichment.bolt.GenericEnrichmentBolt"
+        className: "org.apache.metron.bolt.ParserBolt"
         constructorArgs:
             - "${kafka.zk}"
-        configMethods:
-            -   name: "withEnrichment"
-                args:
-                    - ref: "hostEnrichment"
-            -   name: "withMaxCacheSize"
-                args: [10000]
-            -   name: "withMaxTimeRetain"
-                args: [10]
-    -   id: "joinBolt"
-        className: "org.apache.metron.enrichment.bolt.EnrichmentJoinBolt"
-        constructorArgs:
-            - "${kafka.zk}"
-        configMethods:
-        -   name: "withEnrichments"
-            args:
-                - ref: "enrichments"
-        -   name: "withMaxCacheSize"
-            args: [10000]
-        -   name: "withMaxTimeRetain"
-            args: [10]
+            - "pcap"
+            - ref: "parser"
+            - ref: "writer"
 
 streams:
-#parser
-    -   name: "spout -> parser"
+    -   name: "spout -> bolt"
         from: "kafkaSpout"
         to: "parserBolt"
         grouping:
             type: SHUFFLE
-#hbase
-    -   name: "parser -> hbase"
-        from: "parserBolt"
-        to: "hbaseBolt"
-        grouping:
-            streamId: "raw"
-            type: FIELDS
-            args: ["key"]
-#enrichment
-    -   name: "parser -> host"
-        from: "parserBolt"
-        to: "hostEnrichmentBolt"
-        grouping:
-            streamId: "host"
-            type: FIELDS
-            args: ["key"]
-    -   name: "parser -> join"
-        from: "parserBolt"
-        to: "joinBolt"
-        grouping:
-            streamId: "message"
-            type: FIELDS
-            args: ["key"]
-    -   name: "host -> join"
-        from: "hostEnrichmentBolt"
-        to: "joinBolt"
-        grouping:
-            streamId: "host"
-            type: FIELDS
-            args: ["key"]
-
-#threat intel
-    -   name: "enrichmentJoin -> threatSplit"
-        from: "joinBolt"
-        to: "threatIntelSplitBolt"
-        grouping:
-            streamId: "message"
-            type: FIELDS
-            args: ["key"]
-
-    -   name: "threatSplit -> ip"
-        from: "threatIntelSplitBolt"
-        to: "ipThreatIntelBolt"
-        grouping:
-            streamId: "ip"
-            type: FIELDS
-            args: ["key"]
-
-    -   name: "ip -> join"
-        from: "ipThreatIntelBolt"
-        to: "threatIntelJoinBolt"
-        grouping:
-            streamId: "ip"
-            type: FIELDS
-            args: ["key"]
-    -   name: "threatIntelSplit -> threatIntelJoin"
-        from: "threatIntelSplitBolt"
-        to: "threatIntelJoinBolt"
-        grouping:
-            streamId: "message"
-            type: FIELDS
-            args: ["key"]
-#indexing
-    -   name: "threatIntelJoin -> indexing"
-        from: "threatIntelJoinBolt"
-        to: "indexingBolt"
-        grouping:
-            streamId: "message"
-            type: FIELDS
-            args: ["key"]
-#errors
-    -   name: "parser -> errors"
-        from: "parserBolt"
-        to: "errorIndexingBolt"
-        grouping:
-            streamId: "error"
-            type: SHUFFLE
-    -   name: "indexing -> errors"
-        from: "indexingBolt"
-        to: "errorIndexingBolt"
-        grouping:
-            streamId: "error"
-            type: SHUFFLE

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/c737aa9d/metron-streaming/Metron-Topologies/src/main/resources/Metron_Configs/topologies/pcap/test.yaml
----------------------------------------------------------------------
diff --git a/metron-streaming/Metron-Topologies/src/main/resources/Metron_Configs/topologies/pcap/test.yaml b/metron-streaming/Metron-Topologies/src/main/resources/Metron_Configs/topologies/pcap/test.yaml
new file mode 100644
index 0000000..a3a79fd
--- /dev/null
+++ b/metron-streaming/Metron-Topologies/src/main/resources/Metron_Configs/topologies/pcap/test.yaml
@@ -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: "pcap-test"
+config:
+    topology.workers: 1
+
+components:
+    -   id: "parser"
+        className: "org.apache.metron.parsing.parsers.PcapParser"
+    -   id: "writer"
+        className: "org.apache.metron.writer.PcapWriter"
+        constructorArgs:
+            - "${bolt.hbase.table.name}"
+            - "${bolt.hbase.table.fields}"
+        configMethods:
+            -   name: "withProviderImpl"
+                args:
+                    - "${hbase.provider.impl}"
+    -   id: "zkHosts"
+        className: "storm.kafka.ZkHosts"
+        constructorArgs:
+            - "${kafka.zk}"
+    -   id: "kafkaConfig"
+        className: "storm.kafka.SpoutConfig"
+        constructorArgs:
+            # zookeeper hosts
+            - ref: "zkHosts"
+            # topic name
+            - "${spout.kafka.topic.pcap}"
+            # zk root
+            - ""
+            # id
+            - "${spout.kafka.topic.pcap}"
+        properties:
+            -   name: "ignoreZkOffsets"
+                value: false
+            -   name: "startOffsetTime"
+                value: -2
+
+spouts:
+    -   id: "kafkaSpout"
+        className: "storm.kafka.KafkaSpout"
+        constructorArgs:
+            - ref: "kafkaConfig"
+
+bolts:
+    -   id: "parserBolt"
+        className: "org.apache.metron.bolt.ParserBolt"
+        constructorArgs:
+            - "${kafka.zk}"
+            - "pcap"
+            - ref: "parser"
+            - ref: "writer"
+
+streams:
+    -   name: "spout -> bolt"
+        from: "kafkaSpout"
+        to: "parserBolt"
+        grouping:
+            type: SHUFFLE

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

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/c737aa9d/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
index 2de0edc..e730181 100644
Binary files a/metron-streaming/Metron-Topologies/src/main/resources/SampleInput/PCAPExampleOutput and b/metron-streaming/Metron-Topologies/src/main/resources/SampleInput/PCAPExampleOutput differ