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

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

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/Metron_0.1BETA
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