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

[6/6] incubator-metron git commit: METRON-119 - Move PCAP infrastructure from HBase closes apache/incubator-metron#93

METRON-119 - Move PCAP infrastructure from HBase closes apache/incubator-metron#93


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

Branch: refs/heads/master
Commit: 28c250d15b9dbde2551f30362580c67f2e9172a3
Parents: 5f3bee5
Author: cstella <ce...@gmail.com>
Authored: Fri Apr 29 12:28:13 2016 -0400
Committer: cstella <ce...@gmail.com>
Committed: Fri Apr 29 12:28:13 2016 -0400

----------------------------------------------------------------------
 metron-deployment/amazon-ec2/playbook.yml       |   2 +-
 metron-deployment/inventory/dev-vagrant/hosts   |   4 +
 .../inventory/metron_example/hosts              |   3 +
 .../inventory/singlenode-vagrant/group_vars/all |   8 +-
 .../inventory/singlenode-vagrant/hosts          |   4 +
 metron-deployment/playbooks/metron_install.yml  |   8 +-
 .../roles/ambari_config/defaults/main.yml       |   3 +
 .../roles/ambari_config/vars/single_node_vm.yml |   3 +
 .../roles/ambari_config/vars/small_cluster.yml  |   3 +
 .../roles/metron_pcapservice/defaults/main.yml  |   4 +-
 .../roles/metron_pcapservice/tasks/main.yml     |   7 -
 .../metron_pcapservice/tasks/pcapservice.yml    |   5 -
 .../metron_pcapservice/templates/pcapservice    |   4 +-
 .../roles/metron_streaming/defaults/main.yml    |  13 +-
 .../roles/metron_streaming/tasks/main.yml       |  28 +-
 .../metron_streaming/tasks/metron_topology.yml  |  10 +-
 .../metron_streaming/tasks/source_config.yml    |   2 +-
 .../roles/metron_ui/tasks/main.yml              |   2 +-
 metron-platform/metron-api/README.md            |  32 +
 metron-platform/metron-api/README.txt           |  16 -
 metron-platform/metron-api/pom.xml              | 443 +++++-----
 .../OnlyDeleteExpiredFilesCompactionPolicy.java |  54 --
 .../apache/metron/api/ConfigurationManager.java |   7 +-
 .../api/helper/service/PcapServiceCli.java      | 236 +++---
 .../pcapservice/CellTimestampComparator.java    |  40 -
 .../metron/pcapservice/ConfigurationUtil.java   | 246 +-----
 .../pcapservice/HBaseConfigConstants.java       |  57 --
 .../pcapservice/HBaseConfigurationUtil.java     | 179 ----
 .../apache/metron/pcapservice/IPcapGetter.java  | 102 ---
 .../apache/metron/pcapservice/IPcapScanner.java |  66 --
 .../metron/pcapservice/PcapGetterHBaseImpl.java | 826 -------------------
 .../apache/metron/pcapservice/PcapHelper.java   | 222 -----
 .../pcapservice/PcapReceiverImplRestEasy.java   | 381 ++++-----
 .../pcapservice/PcapScannerHBaseImpl.java       | 319 -------
 .../metron/pcapservice/PcapsResponse.java       |  55 +-
 .../pcapservice/rest/JettyServiceRunner.java    |   5 +-
 .../metron/pcapservice/rest/PcapService.java    |  44 +-
 .../main/resources/config-definition-hbase.xml  |  50 --
 .../resources/hbase-config-default.properties   |  57 --
 .../CellTimestampComparatorTest.java            | 109 ---
 .../pcapservice/ConfigurationUtilTest.java      |  44 +-
 .../pcapservice/HBaseConfigurationUtilTest.java |  69 --
 .../pcapservice/HBaseIntegrationTest.java       |  88 --
 .../pcapservice/PcapGetterHBaseImplTest.java    | 553 -------------
 .../metron/pcapservice/PcapHelperTest.java      | 335 --------
 .../PcapReceiverImplRestEasyTest.java           | 197 +++++
 .../pcapservice/PcapScannerHBaseImplTest.java   | 249 ------
 .../src/test/resources/hbase-config.properties  |  57 --
 metron-platform/metron-common/pom.xml           |   9 +-
 .../src/main/assembly/assembly.xml              |  11 +-
 .../org/apache/metron/common/Constants.java     |  19 +
 .../metron/common/spout/kafka/SpoutConfig.java  |  57 ++
 .../utils/timestamp/TimestampConverter.java     |  25 +
 .../utils/timestamp/TimestampConverters.java    |  39 +
 .../src/main/scripts/zk_load_configs.sh         |   8 +-
 metron-platform/metron-data-management/pom.xml  |  13 +-
 .../src/main/bash/flatfile_loader.sh            |   5 +-
 .../main/bash/prune_elasticsearch_indices.sh    |   2 +-
 .../src/main/bash/prune_hdfs_files.sh           |   2 +-
 .../src/main/bash/threatintel_bulk_load.sh      |   5 +-
 .../src/main/bash/threatintel_bulk_prune.sh     |   5 +-
 .../src/main/bash/threatintel_taxii_load.sh     |   5 +-
 .../dataloads/bulk/HDFSDataPrunerTest.java      |  31 +-
 metron-platform/metron-elasticsearch/pom.xml    |   4 +-
 .../src/main/assembly/assembly.xml              |   5 +-
 .../scripts/start_elasticsearch_topology.sh     |   6 +-
 metron-platform/metron-enrichment/pom.xml       |   8 +-
 .../src/main/assembly/assembly.xml              |   3 +-
 .../src/main/scripts/latency_summarizer.sh      |   2 +-
 metron-platform/metron-integration-test/pom.xml |  45 +-
 .../metron/integration/ComponentRunner.java     |   2 +-
 .../components/FluxTopologyComponent.java       |   3 +
 .../components/KafkaWithZKComponent.java        |   9 +-
 .../integration/components/MRComponent.java     |  68 ++
 .../metron/integration/utils/KafkaUtil.java     |  28 +-
 .../src/main/assembly/assembly.xml              |   6 +-
 .../src/main/flux/pcap/remote.yaml              |  70 --
 .../metron-parsers/src/main/flux/pcap/test.yaml |  74 --
 .../org/apache/metron/parsers/GrokParser.java   |   5 +-
 .../metron/parsers/bro/BasicBroParser.java      |  27 +-
 .../apache/metron/parsers/pcap/PcapParser.java  | 229 -----
 .../metron/parsers/snort/BasicSnortParser.java  | 249 +++---
 .../src/main/scripts/start_parser_topology.sh   |   4 +-
 .../integration/PcapParserIntegrationTest.java  | 214 -----
 metron-platform/metron-pcap-backend/README.md   |  75 ++
 metron-platform/metron-pcap-backend/pom.xml     | 234 ++++++
 .../src/main/assembly/assembly.xml              |  64 ++
 .../src/main/config/pcap.properties             |  24 +
 .../src/main/flux/pcap/remote.yaml              |  66 ++
 .../metron/spout/pcap/HDFSWriterCallback.java   | 181 ++++
 .../metron/spout/pcap/HDFSWriterConfig.java     |  98 +++
 .../metron/spout/pcap/KafkaToHDFSSpout.java     |  38 +
 .../metron/spout/pcap/PartitionHDFSWriter.java  | 190 +++++
 .../apache/metron/spout/pcap/SpoutConfig.java   |  35 +
 .../metron/spout/pcap/scheme/FromKeyScheme.java |  71 ++
 .../spout/pcap/scheme/FromPacketScheme.java     |  58 ++
 .../spout/pcap/scheme/KeyConvertible.java       |  25 +
 .../spout/pcap/scheme/TimestampScheme.java      |  46 ++
 .../pcap/scheme/TimestampSchemeCreator.java     |  26 +
 .../org/apache/metron/utils/PcapInspector.java  | 159 ++++
 .../src/main/scripts/pcap_inspector.sh          |  34 +
 .../src/main/scripts/start_pcap_topology.sh     |  23 +
 .../org/apache/metron/pcap/PcapFilterTest.java  | 269 ++++++
 .../org/apache/metron/pcap/PcapJobTest.java     | 121 +++
 .../PcapTopologyIntegrationTest.java            | 398 +++++++++
 metron-platform/metron-pcap/pom.xml             |  20 +
 .../java/org/apache/metron/pcap/Endianness.java |  33 +
 .../java/org/apache/metron/pcap/PcapHelper.java | 299 +++++++
 .../org/apache/metron/pcap/mr/PcapFilter.java   | 121 +++
 .../java/org/apache/metron/pcap/mr/PcapJob.java | 230 ++++++
 .../metron/pcap/spout/HDFSWriterCallback.java   | 168 ----
 .../metron/pcap/spout/HDFSWriterConfig.java     |  97 ---
 .../src/main/java/storm/kafka/Callback.java     |   4 +-
 .../java/storm/kafka/CallbackCollector.java     | 295 +++----
 .../java/storm/kafka/CallbackKafkaSpout.java    | 107 +--
 .../src/main/java/storm/kafka/EmitContext.java  | 221 ++---
 .../org/apache/metron/pcap/PcapHelperTest.java  |  78 ++
 metron-platform/metron-solr/pom.xml             |   4 +-
 .../metron-solr/src/main/assembly/assembly.xml  |   6 +-
 .../src/main/scripts/start_solr_topology.sh     |   6 +-
 metron-platform/pom.xml                         |  12 +-
 121 files changed, 4730 insertions(+), 5754 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/28c250d1/metron-deployment/amazon-ec2/playbook.yml
----------------------------------------------------------------------
diff --git a/metron-deployment/amazon-ec2/playbook.yml b/metron-deployment/amazon-ec2/playbook.yml
index 16d281d..cef6a2c 100644
--- a/metron-deployment/amazon-ec2/playbook.yml
+++ b/metron-deployment/amazon-ec2/playbook.yml
@@ -28,7 +28,7 @@
     - 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=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,hadoop_client,metron,ec2,pcap_server
     - include: tasks/create-hosts.yml host_count=1 host_type=ambari_slave,enrichment,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

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/28c250d1/metron-deployment/inventory/dev-vagrant/hosts
----------------------------------------------------------------------
diff --git a/metron-deployment/inventory/dev-vagrant/hosts b/metron-deployment/inventory/dev-vagrant/hosts
index 6fd8b18..528094b 100644
--- a/metron-deployment/inventory/dev-vagrant/hosts
+++ b/metron-deployment/inventory/dev-vagrant/hosts
@@ -39,6 +39,9 @@ node1
 [mysql]
 node1
 
+[pcap_server]
+node1
+
 [metron:children]
 enrichment
 search
@@ -46,3 +49,4 @@ web
 sensors
 mysql
 hadoop_client
+pcap_server

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/28c250d1/metron-deployment/inventory/metron_example/hosts
----------------------------------------------------------------------
diff --git a/metron-deployment/inventory/metron_example/hosts b/metron-deployment/inventory/metron_example/hosts
index 0d01327..d190434 100644
--- a/metron-deployment/inventory/metron_example/hosts
+++ b/metron-deployment/inventory/metron_example/hosts
@@ -32,6 +32,9 @@ node8
 [hadoop_client]
 node9
 
+[pcap_server]
+node9
+
 #3rd ambari_slave
 [enrichment]
 node1

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/28c250d1/metron-deployment/inventory/singlenode-vagrant/group_vars/all
----------------------------------------------------------------------
diff --git a/metron-deployment/inventory/singlenode-vagrant/group_vars/all b/metron-deployment/inventory/singlenode-vagrant/group_vars/all
index 9b85aeb..decd7f3 100644
--- a/metron-deployment/inventory/singlenode-vagrant/group_vars/all
+++ b/metron-deployment/inventory/singlenode-vagrant/group_vars/all
@@ -70,9 +70,9 @@ pcap_replay: True
 sniff_interface: eth1
 pcap_replay_interface: "{{ sniff_interface }}"
 storm_parser_topologies:
-    - "{{ metron_directory }}/flux/bro/remote.yaml"
-    - "{{ metron_directory }}/flux/snort/remote.yaml"
-    - "{{ metron_directory }}/flux/yaf/remote.yaml"
+    - "bro"
+    - "snort"
+    - "yaf"
 pcapservice_port: 8081
 
 #Search
@@ -83,4 +83,4 @@ solr_number_shards: 1
 solr_replication_factor: 1
 elasticsearch_transport_port: 9300
 elasticsearch_network_interface: eth1
-elasticsearch_web_port: 9200
\ No newline at end of file
+elasticsearch_web_port: 9200

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/28c250d1/metron-deployment/inventory/singlenode-vagrant/hosts
----------------------------------------------------------------------
diff --git a/metron-deployment/inventory/singlenode-vagrant/hosts b/metron-deployment/inventory/singlenode-vagrant/hosts
index 6fd8b18..93c53c2 100644
--- a/metron-deployment/inventory/singlenode-vagrant/hosts
+++ b/metron-deployment/inventory/singlenode-vagrant/hosts
@@ -36,6 +36,9 @@ node1
 [sensors]
 node1
 
+[pcap_server]
+node1
+
 [mysql]
 node1
 
@@ -46,3 +49,4 @@ web
 sensors
 mysql
 hadoop_client
+pcap_server

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/28c250d1/metron-deployment/playbooks/metron_install.yml
----------------------------------------------------------------------
diff --git a/metron-deployment/playbooks/metron_install.yml b/metron-deployment/playbooks/metron_install.yml
index f6bc492..4654cfc 100644
--- a/metron-deployment/playbooks/metron_install.yml
+++ b/metron-deployment/playbooks/metron_install.yml
@@ -87,10 +87,16 @@
   tags:
     - enrichment
 
+- hosts: pcap_server
+  become: true
+  roles:
+    - { role: metron_pcapservice, when: install_elasticsearch | default(True) == True }
+  tags:
+    - pcap_service
+
 - hosts: web
   become: true
   roles:
     - { role: metron_ui, when: install_elasticsearch | default(True) == True }
-    - { role: metron_pcapservice, when: install_elasticsearch | default(True) == True }
   tags:
     - web

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/28c250d1/metron-deployment/roles/ambari_config/defaults/main.yml
----------------------------------------------------------------------
diff --git a/metron-deployment/roles/ambari_config/defaults/main.yml b/metron-deployment/roles/ambari_config/defaults/main.yml
index 507b6e3..88ea47a 100644
--- a/metron-deployment/roles/ambari_config/defaults/main.yml
+++ b/metron-deployment/roles/ambari_config/defaults/main.yml
@@ -28,3 +28,6 @@ nodemanager_log_dirs: /hadoop/yarn/log
 storm_local_dir: /hadoop/storm
 kafka_log_dirs: /kafka-log
 cluster_type: small_cluster
+nodemanager_mem_mb : 4096
+mapred_map_mem_mb : 1024
+mapred_reduce_mem_mb : 1024

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/28c250d1/metron-deployment/roles/ambari_config/vars/single_node_vm.yml
----------------------------------------------------------------------
diff --git a/metron-deployment/roles/ambari_config/vars/single_node_vm.yml b/metron-deployment/roles/ambari_config/vars/single_node_vm.yml
index 6b18825..cc0ec28 100644
--- a/metron-deployment/roles/ambari_config/vars/single_node_vm.yml
+++ b/metron-deployment/roles/ambari_config/vars/single_node_vm.yml
@@ -60,11 +60,14 @@ configurations:
       jobhistory_heapsize: 256
   - mapred-site:
       mapreduce.jobhistory.recovery.store.leveldb.path : '{{ jhs_recovery_store_ldb_path }}'
+      mapreduce.map.memory.mb : '{{ mapred_map_mem_mb }}'
+      mapreduce.reduce.memory.mb : '{{ mapred_reduce_mem_mb }}'
   - yarn-site:
       yarn.nodemanager.local-dirs : '{{ nodemanager_local_dirs }}'
       yarn.timeline-service.leveldb-timeline-store.path: '{{ timeline_ldb_store_path }}'
       yarn.timeline-service.leveldb-state-store.path: '{{ timeline_ldb_state_path }}'
       yarn.nodemanager.log-dirs: '{{ nodemanager_log_dirs }}'
+      yarn.nodemanager.resource.memory-mb : '{{ nodemanager_mem_mb }}'
   - storm-site:
       supervisor.slots.ports: "[6700, 6701, 6702, 6703]"
       storm.local.dir: '{{ storm_local_dir }}'

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/28c250d1/metron-deployment/roles/ambari_config/vars/small_cluster.yml
----------------------------------------------------------------------
diff --git a/metron-deployment/roles/ambari_config/vars/small_cluster.yml b/metron-deployment/roles/ambari_config/vars/small_cluster.yml
index ff985dc..78e76ee 100644
--- a/metron-deployment/roles/ambari_config/vars/small_cluster.yml
+++ b/metron-deployment/roles/ambari_config/vars/small_cluster.yml
@@ -54,11 +54,14 @@ configurations:
       dfs.journalnode.edits.dir: '{{ journalnode_edits_dir | default("/hadoop/hdfs/journalnode") }}'
   - mapred-site:
       mapreduce.jobhistory.recovery.store.leveldb.path : '{{ jhs_recovery_store_ldb_path | default("/hadoop/mapreduce/jhs") }}'
+      mapreduce.map.memory.mb : '{{ mapred_map_mem_mb }}'
+      mapreduce.reduce.memory.mb : '{{ mapred_reduce_mem_mb }}'
   - yarn-site:
       yarn.nodemanager.local-dirs : '{{ nodemanager_local_dirs| default("/hadoop/yarn/local") }}'
       yarn.timeline-service.leveldb-timeline-store.path: '{{ timeline_ldb_store_path | default("/hadoop/yarn/timeline") }}'
       yarn.timeline-service.leveldb-state-store.path: '{{ timeline_ldb_state_path| default("/hadoop/yarn/timeline") }}'
       yarn.nodemanager.log-dirs: '{{ nodemanager_log_dirs| default("/hadoop/yarn/log") }}'
+      yarn.nodemanager.resource.memory-mb : '{{ nodemanager_mem_mb }}'
   - storm-site:
       supervisor.slots.ports: "[6700, 6701, 6702, 6703]"
       storm.local.dir: '{{ storm_local_dir | default("/hadoop/storm") }}'

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/28c250d1/metron-deployment/roles/metron_pcapservice/defaults/main.yml
----------------------------------------------------------------------
diff --git a/metron-deployment/roles/metron_pcapservice/defaults/main.yml b/metron-deployment/roles/metron_pcapservice/defaults/main.yml
index 2ff6c6b..c19470a 100644
--- a/metron-deployment/roles/metron_pcapservice/defaults/main.yml
+++ b/metron-deployment/roles/metron_pcapservice/defaults/main.yml
@@ -17,8 +17,10 @@
 ---
 metron_version: 0.1BETA
 metron_directory: /usr/metron/{{ metron_version }}
-pcapservice_jar_name: metron-api-{{ metron_version }}-jar-with-dependencies.jar
+pcapservice_jar_name: metron-api-{{ metron_version }}.jar
 pcapservice_jar_src: "{{ playbook_dir }}/../../metron-platform/metron-api/target/{{ pcapservice_jar_name }}"
 pcapservice_jar_dst: "{{ metron_directory }}/lib/{{ pcapservice_jar_name }}"
 pcapservice_port: 8081
 hbase_config_path: "/etc/hbase/conf"
+query_hdfs_path: "/tmp"
+pcap_hdfs_path: "/apps/metron/pcap"

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/28c250d1/metron-deployment/roles/metron_pcapservice/tasks/main.yml
----------------------------------------------------------------------
diff --git a/metron-deployment/roles/metron_pcapservice/tasks/main.yml b/metron-deployment/roles/metron_pcapservice/tasks/main.yml
index ee9cac2..9afdc3b 100644
--- a/metron-deployment/roles/metron_pcapservice/tasks/main.yml
+++ b/metron-deployment/roles/metron_pcapservice/tasks/main.yml
@@ -15,11 +15,4 @@
 #  limitations under the License.
 #
 ---
-- name: Check for hbase-site
-  stat: path=/etc/hbase/conf/hbase-site.xml
-  register: hbase_site
-
-- include: config-hbase.yml
-  when: not hbase_site.stat.exists
-
 - include: pcapservice.yml

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/28c250d1/metron-deployment/roles/metron_pcapservice/tasks/pcapservice.yml
----------------------------------------------------------------------
diff --git a/metron-deployment/roles/metron_pcapservice/tasks/pcapservice.yml b/metron-deployment/roles/metron_pcapservice/tasks/pcapservice.yml
index 651f7fb..50a9e51 100644
--- a/metron-deployment/roles/metron_pcapservice/tasks/pcapservice.yml
+++ b/metron-deployment/roles/metron_pcapservice/tasks/pcapservice.yml
@@ -26,11 +26,6 @@
     src: "{{ pcapservice_jar_src }}"
     dest: "{{ pcapservice_jar_dst }}"
 
-- name: Add hbase-site.xml to pcapservice jar
-  shell: "jar -uf {{ pcapservice_jar_dst }} hbase-site.xml"
-  args:
-    chdir: "{{ hbase_config_path }}"
-
 - name: Install service script
   template: src=pcapservice dest=/etc/init.d/pcapservice mode=0755
 

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/28c250d1/metron-deployment/roles/metron_pcapservice/templates/pcapservice
----------------------------------------------------------------------
diff --git a/metron-deployment/roles/metron_pcapservice/templates/pcapservice b/metron-deployment/roles/metron_pcapservice/templates/pcapservice
index a3ad92b..054133d 100644
--- a/metron-deployment/roles/metron_pcapservice/templates/pcapservice
+++ b/metron-deployment/roles/metron_pcapservice/templates/pcapservice
@@ -28,8 +28,8 @@ SCRIPTNAME=/etc/init.d/$NAME
 LOGFILE="/var/log/metron_pcapservice.log"
 EXTRA_ARGS="${@:2}"
 DAEMON_PATH="/"
-DAEMON="/usr/bin/java"
-DAEMONOPTS="-cp {{ pcapservice_jar_dst }} org.apache.metron.pcapservice.rest.PcapService -port {{ pcapservice_port }}"
+DAEMON="/usr/bin/yarn jar"
+DAEMONOPTS="{{ pcapservice_jar_dst }} org.apache.metron.pcapservice.rest.PcapService -port {{ pcapservice_port }} -query_hdfs_path {{ query_hdfs_path }} -pcap_hdfs_path {{ pcap_hdfs_path }}"
 
 case "$1" in
   start)

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/28c250d1/metron-deployment/roles/metron_streaming/defaults/main.yml
----------------------------------------------------------------------
diff --git a/metron-deployment/roles/metron_streaming/defaults/main.yml b/metron-deployment/roles/metron_streaming/defaults/main.yml
index 568e998..b82c3a6 100644
--- a/metron-deployment/roles/metron_streaming/defaults/main.yml
+++ b/metron-deployment/roles/metron_streaming/defaults/main.yml
@@ -18,14 +18,17 @@
 metron_directory: /usr/metron/{{ metron_version }}
 metron_solr_jar_name: metron-solr-{{ metron_version }}.jar
 metron_elasticsearch_jar_name: metron-elasticsearch-{{ metron_version }}.jar
+metron_pcap_jar_name: metron-pcap-backend-{{ metron_version }}.jar
 metron_parsers_jar_name: metron-parsers-{{ metron_version }}.jar
 
 metron_common_bundle_name: metron-common-{{ metron_version }}-archive.tar.gz
+metron_pcap_bundle_name: metron-pcap-backend-{{ metron_version }}-archive.tar.gz
 metron_data_management_bundle_name: metron-data-management-{{ metron_version }}-archive.tar.gz
 metron_enrichment_bundle_name: metron-enrichment-{{ metron_version }}-archive.tar.gz
 metron_solr_bundle_name: metron-solr-{{ metron_version }}-archive.tar.gz
 metron_elasticsearch_bundle_name: metron-elasticsearch-{{ metron_version }}-archive.tar.gz
 metron_parsers_bundle_name: metron-parsers-{{ metron_version }}-archive.tar.gz
+metron_pcap_bundle_path: "{{ playbook_dir }}/../../metron-platform/metron-pcap-backend/target/{{ metron_pcap_bundle_name }}"
 metron_common_bundle_path: "{{ playbook_dir }}/../../metron-platform/metron-common/target/{{ metron_common_bundle_name }}"
 metron_data_management_bundle_path: "{{ playbook_dir }}/../../metron-platform/metron-data-management/target/{{ metron_data_management_bundle_name }}"
 metron_enrichment_bundle_path: "{{ playbook_dir }}/../../metron-platform/metron-enrichment/target/{{ metron_enrichment_bundle_name }}"
@@ -37,11 +40,13 @@ metron_parsers_bundle_path: "{{ playbook_dir }}/../../metron-platform/metron-par
 config_path: "{{ metron_directory }}/config"
 zookeeper_config_path: "{{ config_path }}/zookeeper"
 zookeeper_global_config_path: "{{ zookeeper_config_path }}/global.json"
+metron_pcap_properties_config_path: "{{ metron_directory }}/config/pcap.properties"
 metron_solr_properties_config_path: "{{ metron_directory }}/config/solr.properties"
 metron_elasticsearch_properties_config_path: "{{ metron_directory }}/config/elasticsearch.properties"
 metron_parsers_properties_config_path: "{{ metron_directory }}/config/parsers.properties"
 hbase_config_path: "/etc/hbase/conf"
 hdfs_config_path: "/etc/hadoop/conf"
+pcap_hdfs_path: "/apps/metron/pcap"
 
 threat_intel_bulk_load: True
 threat_intel_bin: "{{ metron_directory }}/bin/threatintel_bulk_load.sh"
@@ -54,12 +59,10 @@ bro_topic: bro
 yaf_topic: yaf
 snort_topic: snort
 enrichments_topic: enrichments
-storm_enrichment_topology:
-    - "{{ metron_directory }}/flux/enrichment/remote.yaml"
 storm_parser_topologies:
-    - "{{ metron_directory }}/flux/bro/remote.yaml"
-    - "{{ metron_directory }}/flux/snort/remote.yaml"
-    - "{{ metron_directory }}/flux/yaf/remote.yaml"
+    - "bro"
+    - "snort"
+    - "yaf"
 
 hdfs_retention_days: 30
 hdfs_bro_purge_cronjob: "{{ metron_directory }}/bin/prune_hdfs_files.sh -f {{ hdfs_url }} -g '/apps/metron/enrichment/indexed/bro_doc/*enrichment-*' -s $(date -d '{{ hdfs_retention_days }} days ago' +%m/%d/%Y) -n 1 >> /var/log/bro-purge/cron-hdfs-bro-purge.log 2>&1"

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/28c250d1/metron-deployment/roles/metron_streaming/tasks/main.yml
----------------------------------------------------------------------
diff --git a/metron-deployment/roles/metron_streaming/tasks/main.yml b/metron-deployment/roles/metron_streaming/tasks/main.yml
index 2f22dba..6d5a9fb 100644
--- a/metron-deployment/roles/metron_streaming/tasks/main.yml
+++ b/metron-deployment/roles/metron_streaming/tasks/main.yml
@@ -19,7 +19,7 @@
   file: path="{{ metron_directory }}/{{ item.name }}"  state=directory mode=0755
   with_items:
       - { name: 'lib'}
-      - { name: 'scripts'}
+      - { name: 'bin'}
       - { name: 'config'}
 
 
@@ -53,8 +53,13 @@
     src: "{{ metron_common_bundle_path }}"
     dest: "{{ metron_directory }}"
 
+- name: Copy Metron Pcap bundle
+  copy:
+    src: "{{ metron_pcap_bundle_path }}"
+    dest: "{{ metron_directory }}"
+
 - name: Unbundle Metron bundles
-  shell: cd {{ metron_directory }} && tar xzvf metron-solr*.tar.gz && tar xzvf metron-elasticsearch*.tar.gz && tar xzvf metron-enrichment*.tar.gz && tar xzvf metron-parsers*.tar.gz && tar xzvf metron-data-management*.tar.gz && tar xzvf metron-common*.tar.gz && rm *.tar.gz
+  shell: cd {{ metron_directory }} && tar xzvf metron-solr*.tar.gz && tar xzvf metron-elasticsearch*.tar.gz && tar xzvf metron-enrichment*.tar.gz && tar xzvf metron-parsers*.tar.gz && tar xzvf metron-data-management*.tar.gz && tar xzvf metron-common*.tar.gz && tar xzvf metron-pcap-backend*.tar.gz && rm *.tar.gz
 
 - name: Add *-site.xml files to topology jars
   shell: cd {{ item.config_path }} && jar -uf {{ metron_directory }}/lib/{{ item.jar_name }} {{ item.file_name }}
@@ -65,6 +70,11 @@
       - { config_path: "{{ hbase_config_path }}", jar_name: "{{ metron_elasticsearch_jar_name }}", file_name: "hbase-site.xml" }
       - { config_path: "{{ hdfs_config_path }}", jar_name: "{{ metron_elasticsearch_jar_name }}", file_name: "core-site.xml" }
       - { config_path: "{{ hdfs_config_path }}", jar_name: "{{ metron_elasticsearch_jar_name }}", file_name: "hdfs-site.xml" }
+      - { config_path: "{{ hdfs_config_path }}", jar_name: "{{ metron_pcap_jar_name }}", file_name: "core-site.xml" }
+      - { config_path: "{{ hdfs_config_path }}", jar_name: "{{ metron_pcap_jar_name }}", file_name: "hdfs-site.xml" }
+      - { config_path: "{{ hbase_config_path }}", jar_name: "{{ metron_parsers_jar_name }}", file_name: "hbase-site.xml" }
+      - { config_path: "{{ hdfs_config_path }}", jar_name: "{{ metron_parsers_jar_name }}", file_name: "core-site.xml" }
+      - { config_path: "{{ hdfs_config_path }}", jar_name: "{{ metron_parsers_jar_name }}", file_name: "hdfs-site.xml" }
 
 - name: Get Default mysql passowrd
   include_vars: "../roles/mysql_server/defaults/main.yml"
@@ -76,6 +86,16 @@
 - include: grok_upload.yml
   run_once: true
 
+- name: Configure Metron Pcap Topology
+  lineinfile: >
+    dest={{ metron_pcap_properties_config_path }}
+    regexp="{{ item.regexp }}"
+    line="{{ item.line }}"
+  with_items:
+    - { regexp: "kafka.zk=", line: "kafka.zk={{ zookeeper_url }}" }
+    - { regexp: "kafka.pcap.out=", line: "kafka.pcap.out={{ pcap_hdfs_path }}" }
+    - { regexp: "spout.kafka.topic.pcap=", line: "spout.kafka.topic.pcap={{ pycapa_topic }}" }
+
 - name: Configure Metron Parser Topologies
   lineinfile:
     dest: "{{ metron_parsers_properties_config_path }}"
@@ -97,9 +117,7 @@
     - { regexp: "es.port=", line: "es.port={{ elasticsearch_transport_port }}" }
     - { regexp: "es.clustername=", line: "es.clustername={{ elasticsearch_cluster_name }}" }
     - { regexp: "bolt.hdfs.file.system.url=", line: "bolt.hdfs.file.system.url={{ hdfs_url }}" }
-    - { regexp: "spout.kafka.topic.pcap=", line: "spout.kafka.topic.pcap={{ pycapa_topic }}" }
     - { regexp: "spout.kafka.topic.bro=", line: "spout.kafka.topic.bro={{ bro_topic }}" }
-    - { regexp: "bolt.hbase.table.name=", line: "bolt.hbase.table.name={{ pcap_hbase_table }}" }
     - { regexp: "threat.intel.tracker.table=", line: "threat.intel.tracker.table={{ tracker_hbase_table }}" }
     - { regexp: "threat.intel.tracker.cf=", line: "threat.intel.tracker.cf=t" }
     - { regexp: "threat.intel.simple.hbase.table=", line: "threat.intel.simple.hbase.table={{ threatintel_hbase_table }}" }
@@ -125,9 +143,7 @@
     - { regexp: "es.port=", line: "es.port={{ elasticsearch_transport_port }}" }
     - { regexp: "es.clustername=", line: "es.clustername={{ elasticsearch_cluster_name }}" }
     - { regexp: "bolt.hdfs.file.system.url=", line: "bolt.hdfs.file.system.url={{ hdfs_url }}" }
-    - { regexp: "spout.kafka.topic.pcap=", line: "spout.kafka.topic.pcap={{ pycapa_topic }}" }
     - { regexp: "spout.kafka.topic.bro=", line: "spout.kafka.topic.bro={{ bro_topic }}" }
-    - { regexp: "bolt.hbase.table.name=", line: "bolt.hbase.table.name={{ pcap_hbase_table }}" }
     - { regexp: "threat.intel.tracker.table=", line: "threat.intel.tracker.table={{ tracker_hbase_table }}" }
     - { regexp: "threat.intel.tracker.cf=", line: "threat.intel.tracker.cf=t" }
     - { regexp: "threat.intel.simple.hbase.table=", line: "threat.intel.simple.hbase.table={{ threatintel_hbase_table }}" }

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/28c250d1/metron-deployment/roles/metron_streaming/tasks/metron_topology.yml
----------------------------------------------------------------------
diff --git a/metron-deployment/roles/metron_streaming/tasks/metron_topology.yml b/metron-deployment/roles/metron_streaming/tasks/metron_topology.yml
index 3d64f2b..a4354d8 100644
--- a/metron-deployment/roles/metron_streaming/tasks/metron_topology.yml
+++ b/metron-deployment/roles/metron_streaming/tasks/metron_topology.yml
@@ -16,18 +16,14 @@
 #
 ---
 - name: Submit Metron Parser topologies
-  command: storm jar {{ metron_directory }}/lib/{{ metron_parsers_jar_name }} org.apache.storm.flux.Flux  --filter {{ metron_parsers_properties_config_path }} --remote {{ item }}
+  command: "{{ metron_directory }}/bin/start_parser_topology.sh {{ item }}"
   with_items:
       - "{{ storm_parser_topologies }}"
 
 - name: Submit Solr Metron Enrichment topology
-  command: storm jar {{ metron_directory }}/lib/{{ metron_solr_jar_name }} org.apache.storm.flux.Flux  --filter {{ metron_solr_properties_config_path }} --remote {{ item }}
-  with_items:
-      - "{{ storm_enrichment_topology }}"
+  command: "{{ metron_directory }}/bin/start_solr_topology.sh"
   when: install_solr | default(False) == True
 
 - name: Submit Elasticsearch Metron Enrichment topology
-  command: storm jar {{ metron_directory }}/lib/{{ metron_elasticsearch_jar_name }} org.apache.storm.flux.Flux  --filter {{ metron_elasticsearch_properties_config_path }} --remote {{ item }}
-  with_items:
-      - "{{ storm_enrichment_topology }}"
+  command: "{{ metron_directory }}/bin/start_elasticsearch_topology.sh"
   when: install_elasticsearch | default(False) == True

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/28c250d1/metron-deployment/roles/metron_streaming/tasks/source_config.yml
----------------------------------------------------------------------
diff --git a/metron-deployment/roles/metron_streaming/tasks/source_config.yml b/metron-deployment/roles/metron_streaming/tasks/source_config.yml
index 897d0f1..f3039af 100644
--- a/metron-deployment/roles/metron_streaming/tasks/source_config.yml
+++ b/metron-deployment/roles/metron_streaming/tasks/source_config.yml
@@ -43,7 +43,7 @@
     - ../roles/metron_streaming/files/config/
 
 - name: Load Config
-  shell: "{{ metron_directory }}/scripts/zk_load_configs.sh -p {{ zookeeper_config_path }} -z {{ zookeeper_url }} && touch {{ zookeeper_config_path }}/configured"
+  shell: "{{ metron_directory }}/bin/zk_load_configs.sh -p {{ zookeeper_config_path }} -z {{ zookeeper_url }} && touch {{ zookeeper_config_path }}/configured"
 
 
 

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/28c250d1/metron-deployment/roles/metron_ui/tasks/main.yml
----------------------------------------------------------------------
diff --git a/metron-deployment/roles/metron_ui/tasks/main.yml b/metron-deployment/roles/metron_ui/tasks/main.yml
index fd3422b..30f12a4 100644
--- a/metron-deployment/roles/metron_ui/tasks/main.yml
+++ b/metron-deployment/roles/metron_ui/tasks/main.yml
@@ -39,7 +39,7 @@
     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 }}/pcapGetter","mock": false }' }
+    - { regexp: '"pcap":', line: '  "pcap": { "url": "http://{{ groups.pcap_server[0] }}:{{ pcapservice_port }}/pcapGetter","mock": false }' }
 
 - name: Install Node dependencies
   npm:

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/28c250d1/metron-platform/metron-api/README.md
----------------------------------------------------------------------
diff --git a/metron-platform/metron-api/README.md b/metron-platform/metron-api/README.md
new file mode 100644
index 0000000..8214672
--- /dev/null
+++ b/metron-platform/metron-api/README.md
@@ -0,0 +1,32 @@
+# Metron PCAP Service
+
+The purpose of the Metron PCAP service is to provide a middle tier to
+negotiate retrieving packet capture data which flows into Metron.  This
+packet data is of a form which `libpcap` based tools can read.
+
+## Starting the Service
+
+You can start the service either via the init.d script installed,
+`/etc/init.d/pcapservice` or directly via the `yarn jar` command:
+`yarn jar $METRON_HOME/lib/metron-api-$METRON_VERSION.jar org.apache.metron.pcapservice.rest.PcapService -port $SERVICE_PORT -query_hdfs_path $QUERY_PATH -pcap_hdfs_path $PCAP_PATH`
+
+where
+* `METRON_HOME` is the location of the metron installation
+* `METRON_VERSION` is the version of the metron installation
+* `SERVICE_PORT` is the port to bind the REST service to.
+* `QUERY_PATH` is the temporary location to store query results.  They are deleted after the service reads them.
+* `PCAP_PATH` is the path to the packet data on HDFS
+
+## The `/pcapGetter/getPcapsByIdentifiers` Endpoint
+
+This endpoint takes the following query parameters and returns the subset of
+packets matching this query:
+* `srcIp` : The source IP to match on
+* `srcPort` : The source port to match on
+* `dstIp` : The destination IP to match on
+* `dstPort` : The destination port to match on
+* `startTime` : The start time in milliseconds
+* `endTime` : The end time in milliseconds
+
+All of these parameters are optional.  In the case of a missing
+parameter, it is treated as a wildcard.

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/28c250d1/metron-platform/metron-api/README.txt
----------------------------------------------------------------------
diff --git a/metron-platform/metron-api/README.txt b/metron-platform/metron-api/README.txt
deleted file mode 100644
index 9d545ab..0000000
--- a/metron-platform/metron-api/README.txt
+++ /dev/null
@@ -1,16 +0,0 @@
-'hbase' module of 'metron' project contains the code to communicate with HBase. This module has several APIs ( refer IPcapGetter.java, IPcapScanner.java files ) 
-to fetch pcaps from HBase. Following APIs have been created under this module implementation.
-
-APIs ( in IPcapGetter.java) to get pcaps using keys :
- 1. public PcapsResponse getPcaps(List<String> keys, String lastRowKey, long startTime, long endTime, boolean includeReverseTraffic, boolean includeDuplicateLastRow, long maxResultSize) throws IOException;
- 2. public PcapsResponse getPcaps(String key, long startTime, long endTime, boolean includeReverseTraffic) throws IOException;
- 3. public PcapsResponse getPcaps(List<String> keys) throws IOException;
- 4. public PcapsResponse getPcaps(String key) throws IOException;
-
-APIs ( in IPcapScanner.java) to get pcaps using key range :
- 1. public byte[] getPcaps(String startKey, String endKey, long maxResponseSize, long startTime, long endTime) throws IOException;
- 2. public byte[] getPcaps(String startKey, String endKey) throws IOException;
- 
- 
-Refer the wiki documentation for further details : https://hwcsco.atlassian.net/wiki/pages/viewpage.action?pageId=5242892
- 	

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/28c250d1/metron-platform/metron-api/pom.xml
----------------------------------------------------------------------
diff --git a/metron-platform/metron-api/pom.xml b/metron-platform/metron-api/pom.xml
index 0decfa2..90cca37 100644
--- a/metron-platform/metron-api/pom.xml
+++ b/metron-platform/metron-api/pom.xml
@@ -11,6 +11,7 @@
   the specific language governing permissions and limitations under the License..
   -->
 <project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+
 	xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
 	<modelVersion>4.0.0</modelVersion>
 	<parent>
@@ -24,27 +25,23 @@
 		<project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
 		<flume.version>${global_flume_version}</flume.version>
 		<hadoop.version>${global_hadoop_version}</hadoop.version>
-		<maven.compiler.target>1.7</maven.compiler.target>
-		<maven.compiler.source>1.7</maven.compiler.source>
 		<slf4j.version>${global_slf4j_version}</slf4j.version>
 		<zookeeper.version>3.4.5.2.0.6.0-76</zookeeper.version>
 		<logger.version>1.2.15</logger.version>
 
-		<storm-kafka.version>0.9.2-incubating</storm-kafka.version>
-		<storm-hdfs.version>0.0.7-SNAPSHOT</storm-hdfs.version>
-		<storm-hbase.version>0.0.5-SNAPSHOT</storm-hbase.version>
-
-		<spring.integration.version>3.0.0.RELEASE</spring.integration.version>
-		<spring.version>3.2.6.RELEASE</spring.version>
-		<commons-fileupload.version>1.2.2</commons-fileupload.version>
-		<commons-io.version>2.4</commons-io.version>
-		<commons-configuration.version>1.10</commons-configuration.version>
-		<commons-lang.version>2.6</commons-lang.version>
-		<commons-collections.version>3.2.1</commons-collections.version>
-		<commons-beanutils.version>1.8.3</commons-beanutils.version>
-		<commons-jexl.version>2.1.1</commons-jexl.version>
-
+        <storm-kafka.version>0.9.2-incubating</storm-kafka.version>
+        <storm-hdfs.version>0.0.7-SNAPSHOT</storm-hdfs.version>
+        <storm-hbase.version>0.0.5-SNAPSHOT</storm-hbase.version>
 
+        <spring.integration.version>3.0.0.RELEASE</spring.integration.version>
+        <spring.version>3.2.6.RELEASE</spring.version>
+        <commons-fileupload.version>1.2.2</commons-fileupload.version>
+        <commons-io.version>2.4</commons-io.version>
+        <commons-configuration.version>1.10</commons-configuration.version>
+        <commons-lang.version>2.6</commons-lang.version>
+        <commons-collections.version>3.2.1</commons-collections.version>
+        <commons-beanutils.version>1.8.3</commons-beanutils.version>
+        <commons-jexl.version>2.1.1</commons-jexl.version>
 		<junit.version>${global_junit_version}</junit.version>
 		<hamcrest.version>1.3</hamcrest.version>
 		<mockito.version>1.9.5</mockito.version>
@@ -67,216 +64,210 @@
 			<version>${project.parent.version}</version>
 		</dependency>
 
-		<dependency>
-			<groupId>commons-beanutils</groupId>
-			<artifactId>commons-beanutils</artifactId>
-			<version>${commons-beanutils.version}</version>
-		</dependency>
-		<dependency>
-			<groupId>org.apache.commons</groupId>
-			<artifactId>commons-jexl</artifactId>
-			<version>${commons-jexl.version}</version>
-		</dependency>
+        <dependency>
+            <groupId>commons-beanutils</groupId>
+            <artifactId>commons-beanutils</artifactId>
+            <version>${commons-beanutils.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.commons</groupId>
+            <artifactId>commons-jexl</artifactId>
+            <version>${commons-jexl.version}</version>
+        </dependency>
 
-		<dependency>
-			<artifactId>commons-configuration</artifactId>
-			<groupId>commons-configuration</groupId>
-			<version>${commons-configuration.version}</version>
-			<exclusions>
-				<exclusion>
-					<groupId>org.slf4j</groupId>
-					<artifactId>slf4j-api</artifactId>
-				</exclusion>
-			</exclusions>
-		</dependency>
-		<dependency>
-			<groupId>junit</groupId>
-			<artifactId>junit</artifactId>
-			<version>${junit.version}</version>
-			<scope>test</scope>
-		</dependency>
-		<dependency>
-			<groupId>org.powermock</groupId>
-			<artifactId>powermock-api-mockito</artifactId>
-			<version>1.5</version>
-			<scope>test</scope>
-		</dependency>
-		<dependency>
-			<groupId>org.powermock</groupId>
-			<artifactId>powermock-core</artifactId>
-			<version>1.5</version>
-			<scope>test</scope>
-		</dependency>
-		<dependency>
-			<groupId>org.powermock</groupId>
-			<artifactId>powermock-module-junit4</artifactId>
-			<version>1.5</version>
-			<scope>test</scope>
-		</dependency>
-		<dependency>
-			<groupId>joda-time</groupId>
-			<artifactId>joda-time</artifactId>
-			<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-->
-			<exclusions>
-				<exclusion>
-					<groupId>org.slf4j</groupId>
-					<artifactId>slf4j-log4j12</artifactId>
-				</exclusion>
-				<!--exclusion>
-					<groupId>com.google.guava</groupId>
-					<artifactId>guava</artifactId>
-				</exclusion-->
-			</exclusions>
-		</dependency>
-		<dependency>
-			<groupId>org.apache.hbase</groupId>
-			<artifactId>hbase-testing-util</artifactId>
-			<version>${global_hbase_version}</version>
-			<scope>provided</scope>
-			<exclusions>
-				<exclusion>
-					<groupId>org.slf4j</groupId>
-					<artifactId>slf4j-log4j12</artifactId>
-				</exclusion>
-			</exclusions>
-		</dependency>
-		<dependency>
-			<groupId>org.apache.hadoop</groupId>
-			<artifactId>hadoop-common</artifactId>
-			<version>${global_hadoop_version}</version>
-			<!--scope>provided</scope-->
-		</dependency>
-		<dependency>
-			<groupId>org.apache.hadoop</groupId>
-			<artifactId>hadoop-hdfs</artifactId>
-			<version>${global_hadoop_version}</version>
-			<!--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>
-			<version>${spring.integration.version}</version>
-		</dependency>
-		<dependency>
-			<groupId>org.springframework</groupId>
-			<artifactId>spring-webmvc</artifactId>
-			<version>${spring.version}</version>
-		</dependency>
-		<dependency>
-			<groupId>log4j</groupId>
-			<artifactId>log4j</artifactId>
-			<version>${logger.version}</version>
-			<exclusions>
-				<exclusion>
-					<groupId>com.sun.jmx</groupId>
-					<artifactId>jmxri</artifactId>
-				</exclusion>
-				<exclusion>
-					<groupId>com.sun.jdmk</groupId>
-					<artifactId>jmxtools</artifactId>
-				</exclusion>
-				<exclusion>
-					<groupId>javax.jms</groupId>
-					<artifactId>jms</artifactId>
-				</exclusion>
-			</exclusions>
-		</dependency>
-		<dependency>
-			<groupId>org.jboss.resteasy</groupId>
-			<artifactId>resteasy-jaxrs</artifactId>
-			<version>3.0.1.Final</version>
-			<exclusions>
-				<exclusion>
-					<groupId>org.slf4j</groupId>
-					<artifactId>slf4j-simple</artifactId>
-				</exclusion>
-			</exclusions>
-		</dependency>
-		<dependency>
-			<groupId>org.jboss.resteasy</groupId>
-			<artifactId>resteasy-jaxb-provider</artifactId>
-			<version>3.0.1.Final</version>
-			<scope>compile</scope>
-		</dependency>
-		<dependency>
-			<groupId>org.jboss.resteasy</groupId>
-			<artifactId>async-http-servlet-3.0</artifactId>
-			<version>3.0.1.Final</version>
-			<scope>compile</scope>
-		</dependency>
-		<dependency>
-			<groupId>org.eclipse.jetty</groupId>
-			<artifactId>jetty-server</artifactId>
-			<version>9.3.0.M0</version>
-		</dependency>
-		<dependency>
-			<groupId>org.eclipse.jetty</groupId>
-			<artifactId>jetty-servlet</artifactId>
-			<version>9.3.0.M0</version>
-		</dependency>
-		<dependency>
-			<groupId>org.slf4j</groupId>
-			<artifactId>slf4j-simple</artifactId>
-			<version>${global_slf4j_version}</version>
-		</dependency>
-		<dependency>
-			<groupId>org.slf4j</groupId>
-			<artifactId>slf4j-api</artifactId>
-			<version>${global_slf4j_version}</version>
-		</dependency>
-		<dependency>
-			<groupId>org.slf4j</groupId>
-			<artifactId>slf4j-log4j12</artifactId>
-			<version>${global_slf4j_version}</version>
-		</dependency>
-	</dependencies>
+        <dependency>
+            <artifactId>commons-configuration</artifactId>
+            <groupId>commons-configuration</groupId>
+            <version>${commons-configuration.version}</version>
+            <exclusions>
+                <exclusion>
+                    <groupId>org.slf4j</groupId>
+                    <artifactId>slf4j-api</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+        <dependency>
+            <groupId>junit</groupId>
+            <artifactId>junit</artifactId>
+            <version>${junit.version}</version>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.powermock</groupId>
+            <artifactId>powermock-api-mockito</artifactId>
+            <version>1.5</version>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.powermock</groupId>
+            <artifactId>powermock-core</artifactId>
+            <version>1.5</version>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.powermock</groupId>
+            <artifactId>powermock-module-junit4</artifactId>
+            <version>1.5</version>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>joda-time</groupId>
+            <artifactId>joda-time</artifactId>
+            <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.hadoop</groupId>
+            <artifactId>hadoop-common</artifactId>
+            <version>${global_hadoop_version}</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.hadoop</groupId>
+            <artifactId>hadoop-hdfs</artifactId>
+            <version>${global_hadoop_version}</version>
+        </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>
+            <version>${spring.integration.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>org.springframework</groupId>
+            <artifactId>spring-webmvc</artifactId>
+            <version>${spring.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>log4j</groupId>
+            <artifactId>log4j</artifactId>
+            <version>${logger.version}</version>
+            <exclusions>
+                <exclusion>
+                    <groupId>com.sun.jmx</groupId>
+                    <artifactId>jmxri</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>com.sun.jdmk</groupId>
+                    <artifactId>jmxtools</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>javax.jms</groupId>
+                    <artifactId>jms</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+        <dependency>
+            <groupId>org.jboss.resteasy</groupId>
+            <artifactId>resteasy-jaxrs</artifactId>
+            <version>3.0.1.Final</version>
+            <exclusions>
+                <exclusion>
+                    <groupId>org.slf4j</groupId>
+                    <artifactId>slf4j-simple</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+        <dependency>
+            <groupId>org.jboss.resteasy</groupId>
+            <artifactId>resteasy-jaxb-provider</artifactId>
+            <version>3.0.1.Final</version>
+            <scope>compile</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.jboss.resteasy</groupId>
+            <artifactId>async-http-servlet-3.0</artifactId>
+            <version>3.0.1.Final</version>
+            <scope>compile</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.eclipse.jetty</groupId>
+            <artifactId>jetty-server</artifactId>
+            <version>9.3.0.M0</version>
+        </dependency>
+        <dependency>
+            <groupId>org.eclipse.jetty</groupId>
+            <artifactId>jetty-servlet</artifactId>
+            <version>9.3.0.M0</version>
+        </dependency>
+        <dependency>
+            <groupId>org.slf4j</groupId>
+            <artifactId>slf4j-simple</artifactId>
+            <version>${global_slf4j_version}</version>
+        </dependency>
+        <dependency>
+            <groupId>org.slf4j</groupId>
+            <artifactId>slf4j-api</artifactId>
+            <version>${global_slf4j_version}</version>
+        </dependency>
+        <dependency>
+            <groupId>org.slf4j</groupId>
+            <artifactId>slf4j-log4j12</artifactId>
+            <version>${global_slf4j_version}</version>
+        </dependency>
+    </dependencies>
+
+    <build>
+        <plugins>
+            <plugin>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-shade-plugin</artifactId>
+                <version>${global_shade_version}</version>
+                <configuration>
+                    <createDependencyReducedPom>true</createDependencyReducedPom>
+                    <artifactSet>
+                        <excludes>
+                            <exclude>*slf4j*</exclude>
+                        </excludes>
+                    </artifactSet>
+                </configuration>
+                <executions>
+                    <execution>
+                        <phase>package</phase>
+                        <goals>
+                            <goal>shade</goal>
+                        </goals>
+                        <configuration>
+                            <relocations>
+                                <relocation>
+                                    <pattern>com.google.common</pattern>
+                                    <shadedPattern>org.apache.metron.guava</shadedPattern>
+                                </relocation>
+                                <relocation>
+                                    <pattern>org.apache.commons.configuration</pattern>
+                                    <shadedPattern>org.apache.metron.commons.configuration</shadedPattern>
+                                </relocation>
+                            </relocations>
+                            <transformers>
+                                <transformer
+                                        implementation="org.apache.maven.plugins.shade.resource.DontIncludeResourceTransformer">
+                                    <resource>.yaml</resource>
+                                </transformer>
+                                <transformer
+                                        implementation="org.apache.maven.plugins.shade.resource.ServicesResourceTransformer" />
+                                <transformer
+                                        implementation="org.apache.maven.plugins.shade.resource.ManifestResourceTransformer">
+                                    <mainClass></mainClass>
+                                </transformer>
+                            </transformers>
+                        </configuration>
+                    </execution>
+                </executions>
+            </plugin>
+        </plugins>
+    </build>
 
-	<build>
-		<plugins>
-			<plugin>
-				<artifactId>maven-assembly-plugin</artifactId>
-				<configuration>
-					<archive>
-						<manifest>
-							<mainClass>org.apache.metron.pcapservice.rest.PcapService</mainClass>
-						</manifest>
-					</archive>
-					<descriptorRefs>
-						<descriptorRef>jar-with-dependencies</descriptorRef>
-					</descriptorRefs>
-				</configuration>
-				<executions>
-					<execution>
-						<id>make-assembly</id> <!-- this is used for inheritance merges -->
-						<phase>package</phase> <!-- bind to the packaging phase -->
-						<goals>
-							<goal>single</goal>
-						</goals>
-					</execution>
-				</executions>
-			</plugin>
-		</plugins>
-	</build>
 </project>

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/28c250d1/metron-platform/metron-api/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/OnlyDeleteExpiredFilesCompactionPolicy.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-api/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/OnlyDeleteExpiredFilesCompactionPolicy.java b/metron-platform/metron-api/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/OnlyDeleteExpiredFilesCompactionPolicy.java
deleted file mode 100644
index f0d9f5e..0000000
--- a/metron-platform/metron-api/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/OnlyDeleteExpiredFilesCompactionPolicy.java
+++ /dev/null
@@ -1,54 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.regionserver.compactions;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hbase.regionserver.compactions.RatioBasedCompactionPolicy;
-
-import java.io.IOException;
-import java.util.ArrayList;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.regionserver.StoreConfigInformation;
-import org.apache.hadoop.hbase.regionserver.StoreFile;
-
-public class OnlyDeleteExpiredFilesCompactionPolicy extends RatioBasedCompactionPolicy {
-  private static final Log LOG = LogFactory.getLog(OnlyDeleteExpiredFilesCompactionPolicy.class);
-
-  /**
-   * Constructor.
-   * 
-   * @param conf
-   *          The Conf.
-   * @param storeConfigInfo
-   *          Info about the store.
-   */
-  public OnlyDeleteExpiredFilesCompactionPolicy(final Configuration conf, final StoreConfigInformation storeConfigInfo) {
-    super(conf, storeConfigInfo);
-  }
-
-  @Override
-  final ArrayList<StoreFile> applyCompactionPolicy(final ArrayList<StoreFile> candidates, final boolean mayUseOffPeak,
-      final boolean mayBeStuck) throws IOException {
-    LOG.info("Sending empty list for compaction to avoid compaction and do only deletes of files older than TTL");
-
-    return new ArrayList<StoreFile>();
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/28c250d1/metron-platform/metron-api/src/main/java/org/apache/metron/api/ConfigurationManager.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-api/src/main/java/org/apache/metron/api/ConfigurationManager.java b/metron-platform/metron-api/src/main/java/org/apache/metron/api/ConfigurationManager.java
index 5b3e4ae..f9dbfc2 100644
--- a/metron-platform/metron-api/src/main/java/org/apache/metron/api/ConfigurationManager.java
+++ b/metron-platform/metron-api/src/main/java/org/apache/metron/api/ConfigurationManager.java
@@ -23,10 +23,7 @@ import java.io.File;
 import java.util.HashMap;
 import java.util.Map;
 
-import org.apache.commons.configuration.CombinedConfiguration;
-import org.apache.commons.configuration.Configuration;
-import org.apache.commons.configuration.ConfigurationException;
-import org.apache.commons.configuration.DefaultConfigurationBuilder;
+import org.apache.commons.configuration.*;
 import org.apache.commons.lang.StringUtils;
 import org.apache.log4j.Logger;
 
@@ -71,7 +68,7 @@ public class ConfigurationManager {
       try {
         configuration = builder.getConfiguration(true);
         configurationsCache.put(fielPath, configuration);
-      } catch (ConfigurationException e) {
+      } catch (ConfigurationException|ConfigurationRuntimeException e) {
         LOGGER.info("Exception in loading property files.", e);
       }
     }

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/28c250d1/metron-platform/metron-api/src/main/java/org/apache/metron/api/helper/service/PcapServiceCli.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-api/src/main/java/org/apache/metron/api/helper/service/PcapServiceCli.java b/metron-platform/metron-api/src/main/java/org/apache/metron/api/helper/service/PcapServiceCli.java
index 9aa4714..09b8c7a 100644
--- a/metron-platform/metron-api/src/main/java/org/apache/metron/api/helper/service/PcapServiceCli.java
+++ b/metron-platform/metron-api/src/main/java/org/apache/metron/api/helper/service/PcapServiceCli.java
@@ -24,104 +24,146 @@ import org.apache.commons.cli.HelpFormatter;
 import org.apache.commons.cli.Option;
 import org.apache.commons.cli.Options;
 import org.apache.commons.cli.ParseException;
+import org.apache.commons.configuration.ConfigurationUtils;
+import org.apache.log4j.PropertyConfigurator;
 
 public class PcapServiceCli {
 
-	private String[] args = null;
-	private Options options = new Options();
-
-	int port = 8081;
-	String uri = "/pcapGetter";
-
-	public int getPort() {
-		return port;
-	}
-
-	public void setPort(int port) {
-		this.port = port;
-	}
-
-	public String getUri() {
-		return uri;
-	}
-
-	public void setUri(String uri) {
-		this.uri = uri;
-	}
-
-	public PcapServiceCli(String[] args) {
-
-		this.args = args;
-
-		Option help = new Option("h", "Display help menue");
-		options.addOption(help);
-		options.addOption(
-				"port",
-				true,
-				"OPTIONAL ARGUMENT [portnumber] If this argument sets the port for starting the service.  If this argument is not set the port will start on defaut port 8081");
-		options.addOption(
-				"endpoint_uri",
-				true,
-				"OPTIONAL ARGUMENT [/uri/to/service] This sets the URI for the service to be hosted.  The default URI is /pcapGetter");
-	}
-
-	public void parse() {
-		CommandLineParser parser = new BasicParser();
-
-		CommandLine cmd = null;
-
-		try {
-			cmd = parser.parse(options, args);
-		} catch (ParseException e1) {
-
-			e1.printStackTrace();
-		}
-
-		if (cmd.hasOption("h"))
-			help();
-
-		if (cmd.hasOption("port")) {
-
-			try {
-				port = Integer.parseInt(cmd.getOptionValue("port").trim());
-			} catch (Exception e) {
-
-				System.out.println("[Metron] Invalid value for port entered");
-				help();
-			}
-		}
-		if (cmd.hasOption("endpoint_uri")) {
-
-			try {
-
-				if (uri == null || uri.equals(""))
-					throw new Exception("invalid uri");
-
-				uri = cmd.getOptionValue("uri").trim();
-
-				if (uri.charAt(0) != '/')
-					uri = "/" + uri;
-
-				if (uri.charAt(uri.length()) == '/')
-					uri = uri.substring(0, uri.length() - 1);
-
-			} catch (Exception e) {
-				System.out.println("[Metron] Invalid URI entered");
-				help();
-			}
-		}
-
-	}
-
-	private void help() {
-		// This prints out some help
-		HelpFormatter formater = new HelpFormatter();
-
-		formater.printHelp("Topology Options:", options);
-
-		// System.out
-		// .println("[Metron] Example usage: \n storm jar Metron-Topologies-0.3BETA-SNAPSHOT.jar org.apache.metron.topology.Bro -local_mode true -config_path Metron_Configs/ -generator_spout true");
-
-		System.exit(0);
-	}
+  private String[] args = null;
+  private Options options = new Options();
+
+  int port = 8081;
+  String uri = "/pcapGetter";
+  String pcapHdfsPath= "/apps/metron/pcap";
+  String queryHdfsPath = "/apps/metron/pcap_query";
+  public int getPort() {
+    return port;
+  }
+
+  public void setPort(int port) {
+    this.port = port;
+  }
+
+  public String getUri() {
+    return uri;
+  }
+
+  public void setUri(String uri) {
+    this.uri = uri;
+  }
+
+  public String getPcapHdfsPath() {
+    return pcapHdfsPath;
+  }
+
+  public String getQueryHdfsPath() {
+    return queryHdfsPath;
+  }
+  public PcapServiceCli(String[] args) {
+
+    this.args = args;
+
+    Option help = new Option("h", "Display help menu");
+    options.addOption(help);
+    options.addOption(
+            "port",
+            true,
+            "OPTIONAL ARGUMENT [portnumber] If this argument sets the port for starting the service.  If this argument is not set the port will start on defaut port 8081");
+    options.addOption(
+            "endpoint_uri",
+            true,
+            "OPTIONAL ARGUMENT [/uri/to/service] This sets the URI for the service to be hosted.  The default URI is /pcapGetter");
+    options.addOption(
+            "query_hdfs_path",
+            true,
+            "[query_hdfs_loc] The location in HDFS to temporarily store query results.  They will be cleaned up after the query is returned."
+    );
+    options.addOption(
+            "pcap_hdfs_path",
+            true,
+            "[pcap_hdfs_path] The location in HDFS where PCAP raw data is stored in sequence files."
+    );
+    options.addOption(
+            "log4j",
+            true,
+            "OPTIONAL ARGUMENT [log4j] The log4j properties."
+    );
+  }
+
+  public void parse() {
+    CommandLineParser parser = new BasicParser();
+
+    CommandLine cmd = null;
+
+    try {
+      cmd = parser.parse(options, args);
+    } catch (ParseException e1) {
+
+      e1.printStackTrace();
+    }
+
+    if (cmd.hasOption("h")) {
+      help();
+    }
+
+    if(cmd.hasOption("log4j")) {
+      PropertyConfigurator.configure(cmd.getOptionValue("log4j"));
+    }
+
+    if (cmd.hasOption("port")) {
+
+      try {
+        port = Integer.parseInt(cmd.getOptionValue("port").trim());
+      } catch (Exception e) {
+
+        System.out.println("[Metron] Invalid value for port entered");
+        help();
+      }
+    }
+    if(cmd.hasOption("pcap_hdfs_path")) {
+      pcapHdfsPath = cmd.getOptionValue("pcap_hdfs_path");
+    }
+    else {
+      throw new IllegalStateException("You must specify the pcap hdfs path");
+    }
+    if(cmd.hasOption("query_hdfs_path")) {
+      queryHdfsPath = cmd.getOptionValue("query_hdfs_path");
+    }
+    else {
+      throw new IllegalStateException("You must specify the query temp hdfs path");
+    }
+    if (cmd.hasOption("endpoint_uri")) {
+
+      try {
+
+        if (uri == null || uri.equals(""))
+          throw new Exception("invalid uri");
+
+        uri = cmd.getOptionValue("uri").trim();
+
+        if (uri.charAt(0) != '/')
+          uri = "/" + uri;
+
+        if (uri.charAt(uri.length()) == '/')
+          uri = uri.substring(0, uri.length() - 1);
+
+      } catch (Exception e) {
+        System.out.println("[Metron] Invalid URI entered");
+        help();
+      }
+    }
+
+  }
+
+  private void help() {
+    // This prints out some help
+    HelpFormatter formater = new HelpFormatter();
+
+    formater.printHelp("Topology Options:", options);
+
+    // System.out
+    // .println("[Metron] Example usage: \n storm jar Metron-Topologies-0.3BETA-SNAPSHOT.jar org.apache.metron.topology.Bro -local_mode true -config_path Metron_Configs/ -generator_spout true");
+
+    System.exit(0);
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/28c250d1/metron-platform/metron-api/src/main/java/org/apache/metron/pcapservice/CellTimestampComparator.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-api/src/main/java/org/apache/metron/pcapservice/CellTimestampComparator.java b/metron-platform/metron-api/src/main/java/org/apache/metron/pcapservice/CellTimestampComparator.java
deleted file mode 100644
index abea5fa..0000000
--- a/metron-platform/metron-api/src/main/java/org/apache/metron/pcapservice/CellTimestampComparator.java
+++ /dev/null
@@ -1,40 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.metron.pcapservice;
-
-import java.util.Comparator;
-
-import org.apache.hadoop.hbase.Cell;
-
-/**
- * Comparator created for sorting pcaps cells based on the timestamp (asc).
- * 
- * @author Sayi
- */
-public class CellTimestampComparator implements Comparator<Cell> {
-
-  /*
-   * (non-Javadoc)
-   * 
-   * @see java.util.Comparator#compare(java.lang.Object, java.lang.Object)
-   */
-  
-  public int compare(Cell o1, Cell o2) {
-    return Long.valueOf(o1.getTimestamp()).compareTo(o2.getTimestamp());
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/28c250d1/metron-platform/metron-api/src/main/java/org/apache/metron/pcapservice/ConfigurationUtil.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-api/src/main/java/org/apache/metron/pcapservice/ConfigurationUtil.java b/metron-platform/metron-api/src/main/java/org/apache/metron/pcapservice/ConfigurationUtil.java
index e5c464f..0ad0a43 100644
--- a/metron-platform/metron-api/src/main/java/org/apache/metron/pcapservice/ConfigurationUtil.java
+++ b/metron-platform/metron-api/src/main/java/org/apache/metron/pcapservice/ConfigurationUtil.java
@@ -17,270 +17,48 @@
  */
 package org.apache.metron.pcapservice;
 
+import org.apache.commons.configuration.BaseConfiguration;
 import org.apache.commons.configuration.Configuration;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.springframework.util.Assert;
 
-import org.apache.metron.api.ConfigurationManager;
 
 
 
 /**
  * utility class for this module which loads commons configuration to fetch
- * properties from underlying resources to communicate with hbase.
+ * properties from underlying resources to communicate with HDFS.
  * 
- * @author Sayi
  */
 public class ConfigurationUtil {
 
-	/** Configuration definition file name for fetching pcaps from hbase */
-	private static final String configDefFileName = "config-definition-hbase.xml";
-	
-	/** property configuration. */
 	private static Configuration propConfiguration = null;
 
 
 	/**
-	 * The Enum SizeUnit.
-	 */
-	public enum SizeUnit {
-
-		/** The kb. */
-		KB,
-		/** The mb. */
-		MB
-	};
-
-	/** The Constant DEFAULT_HCONNECTION_RETRY_LIMIT. */
-	private static final int DEFAULT_HCONNECTION_RETRY_LIMIT = 0;
-
-	/**
 	 * Loads configuration resources 
 	 * @return Configuration
 	 */
-	public static Configuration getConfiguration() {
+	public synchronized static Configuration getConfiguration() {
 		if(propConfiguration == null){
-			propConfiguration =  ConfigurationManager.getConfiguration(configDefFileName);
+			propConfiguration = new BaseConfiguration();
 		}
 		return propConfiguration;
 	}
 
-	/**
-	 * Returns the configured default result size in bytes, if the user input is
-	 * null; otherwise, returns the user input after validating with the
-	 * configured max value. Throws IllegalArgumentException if : 1. input is
-	 * less than or equals to 0 OR 2. input is greater than configured
-	 * {hbase.scan.max.result.size} value
-	 * 
-	 * @param input
-	 *            the input
-	 * @return long
-	 */
-	public static long validateMaxResultSize(String input) {
-		if (input == null) {
-			return getDefaultResultSize();
-		}
-		// validate the user input
-		long value = convertToBytes(Long.parseLong(input), getResultSizeUnit());
-		Assert.isTrue(
-				isAllowableResultSize(value),
-				"'maxResponseSize' param value must be positive and less than {hbase.scan.max.result.size} value");
-		return convertToBytes(value, getResultSizeUnit());
+	public static String getPcapOutputPath() {
+		return getConfiguration().getString("pcap.output.path");
 	}
 
-	/**
-	 * Checks if is allowable result size.
-	 * 
-	 * @param input
-	 *            the input
-	 * @return true, if is allowable result size
-	 */
-	public static boolean isAllowableResultSize(long input) {
-		if (input <= 0 || input > getMaxResultSize()) {
-			return false;
-		}
-		return true;
+	public static void setPcapOutputPath(String path) {
+		getConfiguration().setProperty("pcap.output.path", path);
 	}
 
-	/**
-	 * Returns the configured default result size in bytes.
-	 * 
-	 * @return long
-	 */
-	public static long getDefaultResultSize() {
-		float value = ConfigurationUtil.getConfiguration().getFloat(
-				"hbase.scan.default.result.size");
-		return convertToBytes(value, getResultSizeUnit());
+	public static String getTempQueryOutputPath() {
+		return getConfiguration().getString("temp.query.output.path");
 	}
-
-	/**
-	 * Returns the configured max result size in bytes.
-	 * 
-	 * @return long
-	 */
-	public static long getMaxResultSize() {
-		float value = ConfigurationUtil.getConfiguration().getFloat(
-				"hbase.scan.max.result.size");
-		return convertToBytes(value, getResultSizeUnit());
+	public static void setTempQueryOutputPath(String path) {
+		getConfiguration().setProperty("temp.query.output.path", path);
 	}
 
-	/**
-	 * Returns the configured max row size in bytes.
-	 * 
-	 * @return long
-	 */
-	public static long getMaxRowSize() {
-		float maxRowSize = ConfigurationUtil.getConfiguration().getFloat(
-				"hbase.table.max.row.size");
-		return convertToBytes(maxRowSize, getRowSizeUnit());
-	}
 
-	/**
-	 * Gets the result size unit.
-	 * 
-	 * @return the result size unit
-	 */
-	public static SizeUnit getResultSizeUnit() {
-		return SizeUnit.valueOf(ConfigurationUtil.getConfiguration()
-				.getString("hbase.scan.result.size.unit"));
-	}
-
-	/**
-	 * Gets the row size unit.
-	 * 
-	 * @return the row size unit
-	 */
-	public static SizeUnit getRowSizeUnit() {
-		return SizeUnit.valueOf(ConfigurationUtil.getConfiguration()
-				.getString("hbase.table.row.size.unit"));
-	}
-
-	/**
-	 * Gets the connection retry limit.
-	 * 
-	 * @return the connection retry limit
-	 */
-	public static int getConnectionRetryLimit() {
-		return ConfigurationUtil.getConfiguration().getInt(
-				"hbase.hconnection.retries.number",
-				DEFAULT_HCONNECTION_RETRY_LIMIT);
-	}
-
-	/**
-	 * Checks if is default include reverse traffic.
-	 * 
-	 * @return true, if is default include reverse traffic
-	 */
-	public static boolean isDefaultIncludeReverseTraffic() {
-		return ConfigurationUtil.getConfiguration().getBoolean(
-				"pcaps.include.reverse.traffic");
-	}
-
-	/**
-	 * Gets the table name.
-	 * 
-	 * @return the table name
-	 */
-	public static byte[] getTableName() {
-		return Bytes.toBytes(ConfigurationUtil.getConfiguration().getString(
-				"hbase.table.name"));
-	}
-
-	/**
-	 * Gets the column family.
-	 * 
-	 * @return the column family
-	 */
-	public static byte[] getColumnFamily() {
-		return Bytes.toBytes(ConfigurationUtil.getConfiguration().getString(
-				"hbase.table.column.family"));
-	}
-
-	/**
-	 * Gets the column qualifier.
-	 * 
-	 * @return the column qualifier
-	 */
-	public static byte[] getColumnQualifier() {
-		return Bytes.toBytes(ConfigurationUtil.getConfiguration().getString(
-				"hbase.table.column.qualifier"));
-	}
-
-	/**
-	 * Gets the max versions.
-	 * 
-	 * @return the max versions
-	 */
-	public static int getMaxVersions() {
-		return ConfigurationUtil.getConfiguration().getInt(
-				"hbase.table.column.maxVersions");
-	}
-
-	/**
-	 * Gets the configured tokens in rowkey.
-	 * 
-	 * @return the configured tokens in rowkey
-	 */
-	public static int getConfiguredTokensInRowkey() {
-		return ConfigurationUtil.getConfiguration().getInt(
-				"hbase.table.row.key.tokens");
-	}
-
-	/**
-	 * Gets the minimum tokens in inputkey.
-	 * 
-	 * @return the minimum tokens in inputkey
-	 */
-	public static int getMinimumTokensInInputkey() {
-		return ConfigurationUtil.getConfiguration().getInt(
-				"rest.api.input.key.min.tokens");
-	}
-
-	/**
-	 * Gets the appending token digits.
-	 * 
-	 * @return the appending token digits
-	 */
-	public static int getAppendingTokenDigits() {
-		return ConfigurationUtil.getConfiguration().getInt(
-				"hbase.table.row.key.token.appending.digits");
-	}
-
-	/**
-	 * Convert to bytes.
-	 * 
-	 * @param value
-	 *            the value
-	 * @param unit
-	 *            the unit
-	 * @return the long
-	 */
-	public static long convertToBytes(float value, SizeUnit unit) {
-		if (SizeUnit.KB == unit) {
-			return (long) (value * 1024);
-		}
-		if (SizeUnit.MB == unit) {
-			return (long) (value * 1024 * 1024);
-		}
-		return (long) value;
-	}
-
-	/**
-	 * The main method.
-	 * 
-	 * @param args
-	 *            the arguments
-	 */
-	public static void main(String[] args) {
-		long r1 = getMaxRowSize();
-		System.out.println("getMaxRowSizeInBytes = " + r1);
-		long r2 = getMaxResultSize();
-		System.out.println("getMaxAllowableResultSizeInBytes = " + r2);
-
-		SizeUnit u1 = getRowSizeUnit();
-		System.out.println("getMaxRowSizeUnit = " + u1.toString());
-		SizeUnit u2 = getResultSizeUnit();
-		System.out.println("getMaxAllowableResultsSizeUnit = " + u2.toString());
-	}
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/28c250d1/metron-platform/metron-api/src/main/java/org/apache/metron/pcapservice/HBaseConfigConstants.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-api/src/main/java/org/apache/metron/pcapservice/HBaseConfigConstants.java b/metron-platform/metron-api/src/main/java/org/apache/metron/pcapservice/HBaseConfigConstants.java
deleted file mode 100644
index ffd81ff..0000000
--- a/metron-platform/metron-api/src/main/java/org/apache/metron/pcapservice/HBaseConfigConstants.java
+++ /dev/null
@@ -1,57 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.metron.pcapservice;
-
-/**
- * HBase configuration properties.
- * 
- * @author Sayi
- */
-public class HBaseConfigConstants {
-
-  /** The Constant HBASE_ZOOKEEPER_QUORUM. */
-  public static final String HBASE_ZOOKEEPER_QUORUM = "hbase.zookeeper.quorum";
-
-  /** The Constant HBASE_ZOOKEEPER_CLIENT_PORT. */
-  public static final String HBASE_ZOOKEEPER_CLIENT_PORT = "hbase.zookeeper.clientPort";
-
-  /** The Constant HBASE_ZOOKEEPER_SESSION_TIMEOUT. */
-  public static final String HBASE_ZOOKEEPER_SESSION_TIMEOUT = "zookeeper.session.timeout";
-
-  /** The Constant HBASE_ZOOKEEPER_RECOVERY_RETRY. */
-  public static final String HBASE_ZOOKEEPER_RECOVERY_RETRY = "zookeeper.recovery.retry";
-
-  /** The Constant HBASE_CLIENT_RETRIES_NUMBER. */
-  public static final String HBASE_CLIENT_RETRIES_NUMBER = "hbase.client.retries.number";
-
-  /** The delimeter. */
-  String delimeter = "-";
-
-  /** The regex. */
-  String regex = "\\-";
-
-  /** The Constant PCAP_KEY_DELIMETER. */
-  public static final String PCAP_KEY_DELIMETER = "-";
-
-  /** The Constant START_KEY. */
-  public static final String START_KEY = "startKey";
-
-  /** The Constant END_KEY. */
-  public static final String END_KEY = "endKey";
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/28c250d1/metron-platform/metron-api/src/main/java/org/apache/metron/pcapservice/HBaseConfigurationUtil.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-api/src/main/java/org/apache/metron/pcapservice/HBaseConfigurationUtil.java b/metron-platform/metron-api/src/main/java/org/apache/metron/pcapservice/HBaseConfigurationUtil.java
deleted file mode 100644
index 75932ab..0000000
--- a/metron-platform/metron-api/src/main/java/org/apache/metron/pcapservice/HBaseConfigurationUtil.java
+++ /dev/null
@@ -1,179 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.metron.pcapservice;
-
-import java.io.IOException;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.HBaseConfiguration;
-import org.apache.hadoop.hbase.client.HConnection;
-import org.apache.hadoop.hbase.client.HConnectionManager;
-import org.apache.log4j.Logger;
-import org.mortbay.log.Log;
-
-/**
- * Utility class which creates HConnection instance when the first request is
- * received and registers a shut down hook which closes the connection when the
- * JVM exits. Creates new connection to the cluster only if the existing
- * connection is closed for unknown reasons. Also creates Configuration with
- * HBase resources using configuration properties.
- * 
- * @author Sayi
- * 
- */
-public class HBaseConfigurationUtil {
-
-  /** The Constant LOGGER. */
-  private static final Logger LOGGER = Logger
-      .getLogger(HBaseConfigurationUtil.class);
-
-  /** Configuration which holds all HBase properties. */
-  private static Configuration config;
-
-  /**
-   * A cluster connection which knows how to find master node and locate regions
-   * on the cluster.
-   */
-  private static HConnection clusterConnection = null;
-
-  /**
-   * Creates HConnection instance when the first request is received and returns
-   * the same instance for all subsequent requests if the connection is still
-   * open.
-   * 
-   * @return HConnection instance
-   * @throws IOException
-   *           Signals that an I/O exception has occurred.
-   */
-  public static HConnection getConnection() throws IOException {
-    if (!connectionAvailable()) {
-      synchronized (HBaseConfigurationUtil.class) {
-        createClusterConncetion();
-      }
-    }
-    return clusterConnection;
-  }
-
-  /**
-   * Creates the cluster conncetion.
-   * 
-   * @throws IOException
-   *           Signals that an I/O exception has occurred.
-   */
-  private static void createClusterConncetion() throws IOException {
-    try {
-      if (connectionAvailable()) {
-        return;
-      }
-      clusterConnection = HConnectionManager.createConnection(HBaseConfiguration.create());
-      addShutdownHook();
-      System.out.println("Created HConnection and added shutDownHook");
-    } catch (IOException e) {
-      LOGGER
-          .error(
-              "Exception occurred while creating HConnection using HConnectionManager",
-              e);
-      throw e;
-    }
-  }
-
-  /**
-   * Connection available.
-   * 
-   * @return true, if successful
-   */
-  private static boolean connectionAvailable() {
-    if (clusterConnection == null) {
-      System.out.println("clusterConnection=" + clusterConnection);
-      return false;
-    }
-    System.out.println("clusterConnection.isClosed()="
-        + clusterConnection.isClosed());
-    return clusterConnection != null && !clusterConnection.isClosed();
-  }
-
-  /**
-   * Adds the shutdown hook.
-   */
-  private static void addShutdownHook() {
-    Runtime.getRuntime().addShutdownHook(new Thread(new Runnable() {
-      public void run() {
-        System.out
-            .println("Executing ShutdownHook HBaseConfigurationUtil : Closing HConnection");
-        try {
-          clusterConnection.close();
-        } catch (IOException e) {
-          Log.debug("Caught ignorable exception ", e);
-        }
-      }
-    }, "HBaseConfigurationUtilShutDown"));
-  }
-
-  /**
-   * Closes the underlying connection to cluster; ignores if any exception is
-   * thrown.
-   */
-  public static void closeConnection() {
-    if (clusterConnection != null) {
-      try {
-        clusterConnection.close();
-      } catch (IOException e) {
-        Log.debug("Caught ignorable exception ", e);
-      }
-    }
-  }
-
-  /**
-   * This method creates Configuration with HBase resources using configuration
-   * properties. The same Configuration object will be used to communicate with
-   * all HBase tables;
-   * 
-   * @return Configuration object
-   */
-  public static Configuration read() {
-    if (config == null) {
-      synchronized (HBaseConfigurationUtil.class) {
-        if (config == null) {
-          config = HBaseConfiguration.create();
-
-          config.set(
-              HBaseConfigConstants.HBASE_ZOOKEEPER_QUORUM,
-              ConfigurationUtil.getConfiguration().getString(
-                  "hbase.zookeeper.quorum"));
-          config.set(
-              HBaseConfigConstants.HBASE_ZOOKEEPER_CLIENT_PORT,
-              ConfigurationUtil.getConfiguration().getString(
-                  "hbase.zookeeper.clientPort"));
-          config.set(
-              HBaseConfigConstants.HBASE_CLIENT_RETRIES_NUMBER,
-              ConfigurationUtil.getConfiguration().getString(
-                  "hbase.client.retries.number"));
-          config.set(
-              HBaseConfigConstants.HBASE_ZOOKEEPER_SESSION_TIMEOUT,
-              ConfigurationUtil.getConfiguration().getString(
-                  "zookeeper.session.timeout"));
-          config.set(
-              HBaseConfigConstants.HBASE_ZOOKEEPER_RECOVERY_RETRY,
-              ConfigurationUtil.getConfiguration().getString(
-                  "zookeeper.recovery.retry"));
-        }
-      }
-    }
-    return config;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/28c250d1/metron-platform/metron-api/src/main/java/org/apache/metron/pcapservice/IPcapGetter.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-api/src/main/java/org/apache/metron/pcapservice/IPcapGetter.java b/metron-platform/metron-api/src/main/java/org/apache/metron/pcapservice/IPcapGetter.java
deleted file mode 100644
index 6176707..0000000
--- a/metron-platform/metron-api/src/main/java/org/apache/metron/pcapservice/IPcapGetter.java
+++ /dev/null
@@ -1,102 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.metron.pcapservice;
-
-import java.io.IOException;
-import java.util.List;
-
-/**
- * interface to all 'keys' based pcaps fetching methods.
- * 
- * @author Sayi
- */
-public interface IPcapGetter {
-
-  /**
-   * Gets the pcaps for the input list of keys and lastRowKey.
-   * 
-   * @param keys
-   *          the list of keys for which pcaps are to be retrieved
-   * @param lastRowKey
-   *          last row key from the previous partial response
-   * @param startTime
-   *          the start time in system milliseconds to be used to filter the
-   *          pcaps. The value is set to '0' if the caller sends negative value
-   * @param endTime
-   *          the end time in system milliseconds to be used to filter the
-   *          pcaps. The value is set to Long.MAX_VALUE if the caller sends
-   *          negative value. 'endTime' must be greater than the 'startTime'.
-   * @param includeReverseTraffic
-   *          indicates whether or not to include pcaps from the reverse traffic
-   * @param includeDuplicateLastRow
-   *          indicates whether or not to include the last row from the previous
-   *          partial response
-   * @param maxResultSize
-   *          the max result size
-   * @return PcapsResponse with all matching pcaps merged together
-   * @throws IOException
-   *           Signals that an I/O exception has occurred.
-   */
-  public PcapsResponse getPcaps(List<String> keys, String lastRowKey,
-      long startTime, long endTime, boolean includeReverseTraffic,
-      boolean includeDuplicateLastRow, long maxResultSize) throws IOException;
-
-  /**
-   * Gets the pcaps for the input key.
-   * 
-   * @param key
-   *          the key for which pcaps is to be retrieved.
-   * @param startTime
-   *          the start time in system milliseconds to be used to filter the
-   *          pcaps. The value is set to '0' if the caller sends negative value
-   * @param endTime
-   *          the end time in system milliseconds to be used to filter the
-   *          pcaps.The value is set to Long.MAX_VALUE if the caller sends
-   *          negative value. 'endTime' must be greater than the 'startTime'.
-   * @param includeReverseTraffic
-   *          indicates whether or not to include pcaps from the reverse traffic
-   * @return PcapsResponse with all matching pcaps merged together
-   * @throws IOException
-   *           Signals that an I/O exception has occurred.
-   */
-  public PcapsResponse getPcaps(String key, long startTime, long endTime,
-      boolean includeReverseTraffic) throws IOException;
-
-  /**
-   * Gets the pcaps for the input list of keys.
-   * 
-   * @param keys
-   *          the list of keys for which pcaps are to be retrieved.
-   * @return PcapsResponse with all matching pcaps merged together
-   * @throws IOException
-   *           Signals that an I/O exception has occurred.
-   */
-  public PcapsResponse getPcaps(List<String> keys) throws IOException;
-
-  /**
-   * Gets the pcaps for the input key.
-   * 
-   * @param key
-   *          the key for which pcaps is to be retrieved.
-   * @return PcapsResponse with all matching pcaps merged together
-   * @throws IOException
-   *           Signals that an I/O exception has occurred.
-   */
-  public PcapsResponse getPcaps(String key) throws IOException;
-
-}