You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@eagle.apache.org by yo...@apache.org on 2016/09/07 17:41:57 UTC

[01/52] [abbrv] incubator-eagle git commit: [EAGLE-461] Convert MR history app with new app framework [Forced Update!]

Repository: incubator-eagle
Updated Branches:
  refs/heads/master 0474d5916 -> 21187b55c (forced update)


http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/0bde482b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/storm/JobHistorySpout.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/storm/JobHistorySpout.java b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/storm/JobHistorySpout.java
index 3daae37..ca4a94f 100644
--- a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/storm/JobHistorySpout.java
+++ b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/storm/JobHistorySpout.java
@@ -18,19 +18,20 @@
 
 package org.apache.eagle.jpm.mr.history.storm;
 
-import backtype.storm.spout.SpoutOutputCollector;
-import backtype.storm.task.TopologyContext;
-import backtype.storm.topology.OutputFieldsDeclarer;
-import backtype.storm.topology.base.BaseRichSpout;
-import org.apache.eagle.jpm.mr.history.common.JHFConfigManager;
+import org.apache.eagle.jpm.mr.history.MRHistoryJobConfig;
 import org.apache.eagle.jpm.mr.history.crawler.*;
-import org.apache.eagle.jpm.mr.historyentity.JobProcessTimeStampEntity;
 import org.apache.eagle.jpm.mr.history.zkres.JobHistoryZKStateManager;
+import org.apache.eagle.jpm.mr.historyentity.JobProcessTimeStampEntity;
 import org.apache.eagle.jpm.util.JobIdFilter;
 import org.apache.eagle.jpm.util.JobIdFilterByPartition;
 import org.apache.eagle.jpm.util.JobIdPartitioner;
 import org.apache.eagle.service.client.IEagleServiceClient;
 import org.apache.eagle.service.client.impl.EagleServiceClientImpl;
+
+import backtype.storm.spout.SpoutOutputCollector;
+import backtype.storm.task.TopologyContext;
+import backtype.storm.topology.OutputFieldsDeclarer;
+import backtype.storm.topology.base.BaseRichSpout;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -43,28 +44,30 @@ import java.util.Map;
 /**
  * Zookeeper znode structure
  * -zkRoot
- *   - partitions
- *      - 0 (20150101)
- *      - 1 (20150101)
- *      - 2 (20150101)
- *      - ... ...
- *      - N-1 (20150102)
- *   - jobs
- *      - 20150101
- *        - job1
- *        - job2
- *        - job3
- *      - 20150102
- *        - job1
- *        - job2
- *        - job3
- *
+ * - partitions
+ * - 0 (20150101)
+ * - 1 (20150101)
+ * - 2 (20150101)
+ * - ... ...
+ * - N-1 (20150102)
+ * - jobs
+ * - 20150101
+ * - job1
+ * - job2
+ * - job3
+ * - 20150102
+ * - job1
+ * - job2
+ * - job3
+ * <p>
  * Spout can have multiple instances, which is supported by storm parallelism primitive.
- *
+ * </p>
+ * <p>
  * Under znode partitions, N child znodes (name is 0 based integer) would be created with each znode mapped to one spout instance. All jobs will be partitioned into N
  * partitions by applying JobPartitioner class to each job Id. The value of each partition znode is the date when the last job in this partition
  * is successfully processed.
- *
+ * </p>
+ * <p>
  * processing steps
  * 1) In constructor,
  * 2) In open(), calculate jobPartitionId for current spout (which should be exactly same to spout taskId within TopologyContext)
@@ -74,10 +77,9 @@ import java.util.Map;
  * 7) process job files (job history file and job configuration xml file)
  * 8) add job Id to current date slot say for example 20150102 after this job is successfully processed
  * 9) clean up all slots with date less than currentProcessDate - 2 days. (2 days should be configurable)
- *
+ * </p>
  * Note:
  * if one spout instance crashes and is brought up again, open() method would be invoked again, we need think of this scenario.
- *
  */
 
 public class JobHistorySpout extends BaseRichSpout {
@@ -90,20 +92,18 @@ public class JobHistorySpout extends BaseRichSpout {
     private JobHistoryContentFilter contentFilter;
     private JobHistorySpoutCollectorInterceptor interceptor;
     private JHFInputStreamCallback callback;
-    private JHFConfigManager configManager;
-    private JobHistoryLCM m_jhfLCM;
-    private final static int MAX_RETRY_TIMES = 3;
+    private MRHistoryJobConfig configManager;
+    private JobHistoryLCM jhfLCM;
+    private static final int MAX_RETRY_TIMES = 3;
 
-    public JobHistorySpout(JobHistoryContentFilter filter, JHFConfigManager configManager) {
+    public JobHistorySpout(JobHistoryContentFilter filter, MRHistoryJobConfig configManager) {
         this(filter, configManager, new JobHistorySpoutCollectorInterceptor());
     }
 
     /**
-     * mostly this constructor signature is for unit test purpose as you can put customized interceptor here
-     * @param filter
-     * @param adaptor
+     * mostly this constructor signature is for unit test purpose as you can put customized interceptor here.
      */
-    public JobHistorySpout(JobHistoryContentFilter filter, JHFConfigManager configManager, JobHistorySpoutCollectorInterceptor adaptor) {
+    public JobHistorySpout(JobHistoryContentFilter filter, MRHistoryJobConfig configManager, JobHistorySpoutCollectorInterceptor adaptor) {
         this.contentFilter = filter;
         this.configManager = configManager;
         this.interceptor = adaptor;
@@ -131,15 +131,15 @@ public class JobHistorySpout extends BaseRichSpout {
         partitionId = calculatePartitionId(context);
         // sanity verify 0<=partitionId<=numTotalPartitions-1
         if (partitionId < 0 || partitionId > numTotalPartitions) {
-            throw new IllegalStateException("partitionId should be less than numTotalPartitions with partitionId " +
-                    partitionId + " and numTotalPartitions " + numTotalPartitions);
+            throw new IllegalStateException("partitionId should be less than numTotalPartitions with partitionId "
+                + partitionId + " and numTotalPartitions " + numTotalPartitions);
         }
         Class<? extends JobIdPartitioner> partitionerCls = configManager.getControlConfig().partitionerCls;
         JobIdPartitioner partitioner;
         try {
             partitioner = partitionerCls.newInstance();
         } catch (Exception e) {
-            LOG.error("failing instantiating job partitioner class " + partitionerCls,e);
+            LOG.error("failing instantiating job partitioner class " + partitionerCls, e);
             throw new IllegalStateException(e);
         }
         JobIdFilter jobIdFilter = new JobIdFilterByPartition(partitioner, numTotalPartitions, partitionId);
@@ -148,14 +148,14 @@ public class JobHistorySpout extends BaseRichSpout {
         interceptor.setSpoutOutputCollector(collector);
 
         try {
-            m_jhfLCM = new JobHistoryDAOImpl(configManager.getJobHistoryEndpointConfig());
+            jhfLCM = new JobHistoryDAOImpl(configManager.getJobHistoryEndpointConfig());
             driver = new JHFCrawlerDriverImpl(configManager.getJobHistoryEndpointConfig(),
-                    configManager.getControlConfig(),
-                    callback,
-                    zkState,
-                    m_jhfLCM,
-                    jobIdFilter,
-                    partitionId);
+                configManager.getControlConfig(),
+                callback,
+                zkState,
+                jhfLCM,
+                jobIdFilter,
+                partitionId);
         } catch (Exception e) {
             LOG.error("failing creating crawler driver");
             throw new IllegalStateException(e);
@@ -171,7 +171,7 @@ public class JobHistorySpout extends BaseRichSpout {
         } catch (Exception ex) {
             LOG.error("fail crawling job history file and continue ...", ex);
             try {
-                m_jhfLCM.freshFileSystem();
+                jhfLCM.freshFileSystem();
             } catch (Exception e) {
                 LOG.error("failed to fresh file system ", e);
             }
@@ -179,27 +179,27 @@ public class JobHistorySpout extends BaseRichSpout {
             try {
                 Thread.sleep(1000);
             } catch (Exception e) {
-
+                // ignored
             }
         }
     }
 
     /**
-     * empty because framework will take care of output fields declaration
+     * empty because framework will take care of output fields declaration.
      */
     @Override
     public void declareOutputFields(OutputFieldsDeclarer declarer) {
     }
 
     /**
-     * add to processedJob
+     * add to processedJob.
      */
     @Override
     public void ack(Object jobId) {
     }
 
     /**
-     * job is not fully processed
+     * job is not fully processed.
      */
     @Override
     public void fail(Object jobId) {
@@ -227,26 +227,28 @@ public class JobHistorySpout extends BaseRichSpout {
             }
         }
 
-        if (minTimeStamp == 0l) {
+        if (minTimeStamp == 0L) {
             return;
         }
 
         LOG.info("update process time stamp {}", minTimeStamp);
-        final JHFConfigManager.EagleServiceConfig eagleServiceConfig = configManager.getEagleServiceConfig();
-        final JHFConfigManager.JobExtractorConfig jobExtractorConfig = configManager.getJobExtractorConfig();
-        Map<String, String> baseTags = new HashMap<String, String>() { {
-            put("site", jobExtractorConfig.site);
-        } };
+        final MRHistoryJobConfig.EagleServiceConfig eagleServiceConfig = configManager.getEagleServiceConfig();
+        final MRHistoryJobConfig.JobExtractorConfig jobExtractorConfig = configManager.getJobExtractorConfig();
+        Map<String, String> baseTags = new HashMap<String, String>() {
+            {
+                put("site", jobExtractorConfig.site);
+            }
+        };
         JobProcessTimeStampEntity entity = new JobProcessTimeStampEntity();
         entity.setCurrentTimeStamp(minTimeStamp);
         entity.setTimestamp(minTimeStamp);
         entity.setTags(baseTags);
 
         IEagleServiceClient client = new EagleServiceClientImpl(
-                eagleServiceConfig.eagleServiceHost,
-                eagleServiceConfig.eagleServicePort,
-                eagleServiceConfig.username,
-                eagleServiceConfig.password);
+            eagleServiceConfig.eagleServiceHost,
+            eagleServiceConfig.eagleServicePort,
+            eagleServiceConfig.username,
+            eagleServiceConfig.password);
 
         client.getJerseyClient().setReadTimeout(jobExtractorConfig.readTimeoutSeconds * 1000);
 
@@ -267,7 +269,7 @@ public class JobHistorySpout extends BaseRichSpout {
                     LOG.error("Got exception to flush, reach max retry times " + MAX_RETRY_TIMES, ex);
                 }
             }
-            tried ++;
+            tried++;
         }
 
         client.getJerseyClient().destroy();

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/0bde482b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/zkres/JobHistoryZKStateLCM.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/zkres/JobHistoryZKStateLCM.java b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/zkres/JobHistoryZKStateLCM.java
index 933b347..cbde88c 100644
--- a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/zkres/JobHistoryZKStateLCM.java
+++ b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/zkres/JobHistoryZKStateLCM.java
@@ -22,12 +22,20 @@ import java.util.List;
 
 public interface JobHistoryZKStateLCM {
     void ensureJobPartitions(int numTotalPartitions);
+
     String readProcessedDate(int partitionId);
+
     List<String> readProcessedJobs(String date);
+
     void updateProcessedDate(int partitionId, String date);
+
     void addProcessedJob(String date, String jobId);
+
     void truncateProcessedJob(String date);
+
     void truncateEverything();
+
     long readProcessedTimeStamp(int partitionId);
+
     void updateProcessedTimeStamp(int partitionId, long timeStamp);
 }

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/0bde482b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/zkres/JobHistoryZKStateManager.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/zkres/JobHistoryZKStateManager.java b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/zkres/JobHistoryZKStateManager.java
index 33d3cb2..feb896e 100644
--- a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/zkres/JobHistoryZKStateManager.java
+++ b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/zkres/JobHistoryZKStateManager.java
@@ -18,11 +18,12 @@
 
 package org.apache.eagle.jpm.mr.history.zkres;
 
+import org.apache.eagle.jpm.mr.history.MRHistoryJobConfig.ZKStateConfig;
+
 import org.apache.curator.framework.CuratorFramework;
 import org.apache.curator.framework.CuratorFrameworkFactory;
 import org.apache.curator.framework.recipes.locks.InterProcessMutex;
 import org.apache.curator.retry.RetryNTimes;
-import org.apache.eagle.jpm.mr.history.common.JHFConfigManager.ZKStateConfig;
 import org.apache.zookeeper.CreateMode;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -45,10 +46,10 @@ public class JobHistoryZKStateManager implements JobHistoryZKStateLCM {
 
     private CuratorFramework newCurator(ZKStateConfig config) throws Exception {
         return CuratorFrameworkFactory.newClient(
-                config.zkQuorum,
-                config.zkSessionTimeoutMs,
-                15000,
-                new RetryNTimes(config.zkRetryTimes, config.zkRetryInterval)
+            config.zkQuorum,
+            config.zkSessionTimeoutMs,
+            15000,
+            new RetryNTimes(config.zkRetryTimes, config.zkRetryInterval)
         );
     }
 
@@ -86,7 +87,7 @@ public class JobHistoryZKStateManager implements JobHistoryZKStateLCM {
             if (_curator.checkExists().forPath(path) != null) {
                 _curator.delete().forPath(path);
             }
-        } catch(Exception ex) {
+        } catch (Exception ex) {
             LOG.error("fail reading forceStartFrom znode", ex);
         }
     }
@@ -102,27 +103,28 @@ public class JobHistoryZKStateManager implements JobHistoryZKStateLCM {
     /**
      * under zkRoot, znode forceStartFrom is used to force job is crawled from that date
      * IF
-     *    forceStartFrom znode is provided, and its value is valid date with format "YYYYMMDD",
+     * forceStartFrom znode is provided, and its value is valid date with format "YYYYMMDD",
      * THEN
-     *    rebuild all partitions with the forceStartFrom
+     * rebuild all partitions with the forceStartFrom
      * ELSE
-     *    IF
-     *       partition structure is changed
-     *    THEN
-     *       IF
-     *          there is valid mindate for existing partitions
-     *       THEN
-     *          rebuild job partitions from that valid mindate
-     *       ELSE
-     *          rebuild job partitions from (today - BACKOFF_DAYS)
-     *       END
-     *    ELSE
-     *      do nothing
-     *    END
+     * IF
+     * partition structure is changed
+     * THEN
+     * IF
+     * there is valid mindate for existing partitions
+     * THEN
+     * rebuild job partitions from that valid mindate
+     * ELSE
+     * rebuild job partitions from (today - BACKOFF_DAYS)
+     * END
+     * ELSE
+     * do nothing
      * END
-     *
-     *
+     * END
+     * <p>
      * forceStartFrom is deleted once its value is used, so next time when topology is restarted, program can run from where topology is stopped last time
+     * </p>
+     * .
      */
     @Override
     public void ensureJobPartitions(int numTotalPartitions) {
@@ -137,7 +139,7 @@ public class JobHistoryZKStateManager implements JobHistoryZKStateLCM {
             if (forceStartFrom != null) {
                 try {
                     minDate = Integer.valueOf(forceStartFrom);
-                } catch(Exception ex) {
+                } catch (Exception ex) {
                     LOG.error("failing converting forceStartFrom znode value to integer with value " + forceStartFrom);
                     throw new IllegalStateException();
                 }
@@ -153,16 +155,18 @@ public class JobHistoryZKStateManager implements JobHistoryZKStateLCM {
                         LOG.info("znode partitions structure is changed, current partition count " + currentCount + ", future count " + numTotalPartitions);
                     }
                 }
-                if (!structureChanged)
+                if (!structureChanged) {
                     return; // do nothing
+                }
 
                 if (pathExists) {
                     List<String> partitions = _curator.getChildren().forPath(path);
                     for (String partition : partitions) {
                         String date = new String(_curator.getData().forPath(path + "/" + partition), "UTF-8");
                         int tmp = Integer.valueOf(date);
-                        if(tmp < minDate)
+                        if (tmp < minDate) {
                             minDate = tmp;
+                        }
                     }
                 }
 
@@ -178,7 +182,7 @@ public class JobHistoryZKStateManager implements JobHistoryZKStateLCM {
         } finally {
             try {
                 lock.release();
-            } catch(Exception e) {
+            } catch (Exception e) {
                 LOG.error("fail releasing lock", e);
                 throw new RuntimeException(e);
             }
@@ -195,9 +199,9 @@ public class JobHistoryZKStateManager implements JobHistoryZKStateLCM {
 
         for (int i = 0; i < numTotalPartitions; i++) {
             _curator.create()
-                    .creatingParentsIfNeeded()
-                    .withMode(CreateMode.PERSISTENT)
-                    .forPath(path + "/" + i, startingDate.getBytes("UTF-8"));
+                .creatingParentsIfNeeded()
+                .withMode(CreateMode.PERSISTENT)
+                .forPath(path + "/" + i, startingDate.getBytes("UTF-8"));
         }
     }
 
@@ -222,9 +226,9 @@ public class JobHistoryZKStateManager implements JobHistoryZKStateLCM {
         try {
             if (_curator.checkExists().forPath(path) == null) {
                 _curator.create()
-                        .creatingParentsIfNeeded()
-                        .withMode(CreateMode.PERSISTENT)
-                        .forPath(path, date.getBytes("UTF-8"));
+                    .creatingParentsIfNeeded()
+                    .withMode(CreateMode.PERSISTENT)
+                    .forPath(path, date.getBytes("UTF-8"));
             } else {
                 _curator.setData().forPath(path, date.getBytes("UTF-8"));
             }
@@ -240,9 +244,9 @@ public class JobHistoryZKStateManager implements JobHistoryZKStateLCM {
         try {
             if (_curator.checkExists().forPath(path) == null) {
                 _curator.create()
-                        .creatingParentsIfNeeded()
-                        .withMode(CreateMode.PERSISTENT)
-                        .forPath(path);
+                    .creatingParentsIfNeeded()
+                    .withMode(CreateMode.PERSISTENT)
+                    .forPath(path);
             } else {
                 _curator.setData().forPath(path);
             }
@@ -311,10 +315,10 @@ public class JobHistoryZKStateManager implements JobHistoryZKStateLCM {
         try {
             if (_curator.checkExists().forPath(path) == null) {
                 _curator.create()
-                        .creatingParentsIfNeeded()
-                        .withMode(CreateMode.PERSISTENT)
-                        .forPath(path);
-                return 0l;
+                    .creatingParentsIfNeeded()
+                    .withMode(CreateMode.PERSISTENT)
+                    .forPath(path);
+                return 0L;
             } else {
                 return Long.parseLong(new String(_curator.getData().forPath(path), "UTF-8"));
             }
@@ -330,9 +334,9 @@ public class JobHistoryZKStateManager implements JobHistoryZKStateLCM {
         try {
             if (_curator.checkExists().forPath(path) == null) {
                 _curator.create()
-                        .creatingParentsIfNeeded()
-                        .withMode(CreateMode.PERSISTENT)
-                        .forPath(path);
+                    .creatingParentsIfNeeded()
+                    .withMode(CreateMode.PERSISTENT)
+                    .forPath(path);
             }
 
             _curator.setData().forPath(path, (timeStamp + "").getBytes("UTF-8"));

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/0bde482b/eagle-jpm/eagle-jpm-mr-history/src/main/resources/META-INF/providers/org.apache.eagle.jpm.mr.history.MRHistoryJobApplicationProvider.xml
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-mr-history/src/main/resources/META-INF/providers/org.apache.eagle.jpm.mr.history.MRHistoryJobApplicationProvider.xml b/eagle-jpm/eagle-jpm-mr-history/src/main/resources/META-INF/providers/org.apache.eagle.jpm.mr.history.MRHistoryJobApplicationProvider.xml
new file mode 100644
index 0000000..5e69a16
--- /dev/null
+++ b/eagle-jpm/eagle-jpm-mr-history/src/main/resources/META-INF/providers/org.apache.eagle.jpm.mr.history.MRHistoryJobApplicationProvider.xml
@@ -0,0 +1,154 @@
+<?xml version="1.0" encoding="UTF-8" ?>
+<!--
+  ~ Licensed to the Apache Software Foundation (ASF) under one or more
+  ~ contributor license agreements.  See the NOTICE file distributed with
+  ~ this work for additional information regarding copyright ownership.
+  ~ The ASF licenses this file to You under the Apache License, Version 2.0
+  ~ (the "License"); you may not use this file except in compliance with
+  ~ the License.  You may obtain a copy of the License at
+  ~
+  ~    http://www.apache.org/licenses/LICENSE-2.0
+  ~
+  ~ Unless required by applicable law or agreed to in writing, software
+  ~ distributed under the License is distributed on an "AS IS" BASIS,
+  ~ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  ~ See the License for the specific language governing permissions and
+  ~ limitations under the License.
+  -->
+
+<application>
+    <type>SPARK_HISTORY_JOB_APP</type>
+    <name>Spark History Job Monitoring</name>
+    <version>0.5.0-incubating</version>
+    <appClass>org.apache.eagle.jpm.mr.history.MRHistoryJobApplication</appClass>
+    <viewPath>/apps/jpm</viewPath>
+    <configuration>
+        <!-- org.apache.eagle.jpm.mr.history.MRHistoryJobConfig -->
+        <property>
+            <name>jobExtractorConfig.site</name>
+            <displayName>Site ID</displayName>
+            <value>sandbox</value>
+        </property>
+        <property>
+            <name>jobExtractorConfig.mrVersion</name>
+            <value>MRVer2</value>
+        </property>
+        <property>
+            <name>jobExtractorConfig.readTimeOutSeconds</name>
+            <displayName>zkPort</displayName>
+            <value>10</value>
+        </property>
+        <property>
+            <name>dataSourceConfig.zkQuorum</name>
+            <value>sandbox.hortonworks.com:2181</value>
+        </property>
+        <property>
+            <name>dataSourceConfig.zkPort</name>
+            <value>2181</value>
+        </property>
+        <property>
+            <name>dataSourceConfig.zkSessionTimeoutMs</name>
+            <value>15000</value>
+        </property>
+        <property>
+            <name>dataSourceConfig.zkRetryTimes</name>
+            <value>3</value>
+        </property>
+        <property>
+            <name>dataSourceConfig.zkRetryInterval</name>
+            <value>20000</value>
+        </property>
+        <property>
+            <name>dataSourceConfig.zkRoot</name>
+            <value>/test_mrjobhistory</value>
+        </property>
+        <property>
+            <name>dataSourceConfig.basePath</name>
+            <value>/mr-history/done</value>
+        </property>
+        <property>
+            <name>dataSourceConfig.jobTrackerName</name>
+            <value></value>
+        </property>
+        <property>
+            <name>dataSourceConfig.nnEndpoint</name>
+            <value>hdfs://sandbox.hortonworks.com:8020</value>
+        </property>
+        <property>
+            <name>dataSourceConfig.pathContainsJobTrackerName</name>
+            <value>false</value>
+        </property>
+        <property>
+            <name>dataSourceConfig.principal</name>
+            <value></value>
+        </property>
+        <property>
+            <name>dataSourceConfig.keytab</name>
+            <value></value>
+        </property>
+        <property>
+            <name>dataSourceConfig.dryRun</name>
+            <value>false</value>
+        </property>
+        <property>
+            <name>dataSourceConfig.partitionerCls</name>
+            <value>org.apache.eagle.jpm.util.DefaultJobIdPartitioner</value>
+        </property>
+        <property>
+            <name>dataSourceConfig.zeroBasedMonth</name>
+            <value>false</value>
+        </property>
+        <property>
+            <name>MRConfigureKeys.jobConfigKey</name>
+            <value>mapreduce.map.output.compress,
+                mapreduce.map.output.compress.codec,
+                mapreduce.output.fileoutputformat.compress,
+                mapreduce.output.fileoutputformat.compress.type,
+                mapreduce.output.fileoutputformat.compress.codec,
+                mapred.output.format.class,
+                dataplatform.etl.info,
+                mapreduce.map.memory.mb,
+                mapreduce.reduce.memory.mb,
+                mapreduce.map.java.opts,
+                mapreduce.reduce.java.opts</value>
+        </property>
+        <property>
+            <name>MRConfigureKeys.jobNameKey</name>
+            <value>eagle.job.name</value>
+        </property>
+        <property>
+            <name>envContextConfig.parallelismConfig.mrHistoryJobExecutor</name>
+            <value>6</value>
+        </property>
+        <property>
+            <name>envContextConfig.tasks.mrHistoryJobExecutor</name>
+            <value>6</value>
+        </property>
+        <property>
+            <name>eagleProps.eagleService.host</name>
+            <description>eagleProps.eagleService.host</description>
+            <value>sandbox.hortonworks.com</value>
+        </property>
+        <property>
+            <name>eagleProps.eagleService.port</name>
+            <description>eagleProps.eagleService.port</description>
+            <value>9099</value>
+        </property>
+        <property>
+            <name>eagleProps.eagleService.username</name>
+            <description>eagleProps.eagleService.username</description>
+            <value>admin</value>
+        </property>
+        <property>
+            <name>eagleProps.eagleService.password</name>
+            <description>eagleProps.eagleService.password</description>
+            <value>secret</value>
+        </property>
+    </configuration>
+    <docs>
+        <install>
+        </install>
+        <uninstall>
+        </uninstall>
+    </docs>
+</application>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/0bde482b/eagle-jpm/eagle-jpm-mr-history/src/main/resources/META-INF/services/org.apache.eagle.app.spi.ApplicationProvider
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-mr-history/src/main/resources/META-INF/services/org.apache.eagle.app.spi.ApplicationProvider b/eagle-jpm/eagle-jpm-mr-history/src/main/resources/META-INF/services/org.apache.eagle.app.spi.ApplicationProvider
new file mode 100644
index 0000000..56a30bd
--- /dev/null
+++ b/eagle-jpm/eagle-jpm-mr-history/src/main/resources/META-INF/services/org.apache.eagle.app.spi.ApplicationProvider
@@ -0,0 +1,16 @@
+# 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.
+
+org.apache.eagle.jpm.mr.history.MRHistoryJobApplicationProvider
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/0bde482b/eagle-jpm/eagle-jpm-mr-history/src/main/resources/application.conf
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-mr-history/src/main/resources/application.conf b/eagle-jpm/eagle-jpm-mr-history/src/main/resources/application.conf
index 23a51fc..13e411f 100644
--- a/eagle-jpm/eagle-jpm-mr-history/src/main/resources/application.conf
+++ b/eagle-jpm/eagle-jpm-mr-history/src/main/resources/application.conf
@@ -15,9 +15,6 @@
 
 {
   "envContextConfig" : {
-    "env" : "local",
-    "topologyName" : "mr_history",
-    "stormConfigFile" : "storm.yaml",
     "parallelismConfig" : {
       "mrHistoryJobExecutor" : 6
     },
@@ -62,21 +59,10 @@
       "password": "secret"
     }
   },
-
+  "appId":"mr_history",
+  "mode":"LOCAL",
   "MRConfigureKeys" : {
     "jobNameKey" : "eagle.job.name",
-    "jobConfigKey" : [
-        "mapreduce.map.output.compress",
-        "mapreduce.map.output.compress.codec",
-        "mapreduce.output.fileoutputformat.compress",
-        "mapreduce.output.fileoutputformat.compress.type",
-        "mapreduce.output.fileoutputformat.compress.codec",
-        "mapred.output.format.class",
-        "dataplatform.etl.info",
-        "mapreduce.map.memory.mb",
-        "mapreduce.reduce.memory.mb",
-        "mapreduce.map.java.opts",
-        "mapreduce.reduce.java.opts"
-    ]
+    "jobConfigKey" : "mapreduce.map.output.compress,mapreduce.map.output.compress.codec,mapreduce.output.fileoutputformat.compress,mapreduce.output.fileoutputformat.compress.type,mapreduce.output.fileoutputformat.compress.codec,mapred.output.format.class, dataplatform.etl.info,mapreduce.map.memory.mb,mapreduce.reduce.memory.mb,mapreduce.map.java.opts,mapreduce.reduce.java.opts"
   }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/0bde482b/eagle-jpm/eagle-jpm-mr-history/src/test/java/org/apache/eagle/jpm/mr/history/MRHistoryJobApplicationProviderTest.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-mr-history/src/test/java/org/apache/eagle/jpm/mr/history/MRHistoryJobApplicationProviderTest.java b/eagle-jpm/eagle-jpm-mr-history/src/test/java/org/apache/eagle/jpm/mr/history/MRHistoryJobApplicationProviderTest.java
new file mode 100644
index 0000000..0a3a3a1
--- /dev/null
+++ b/eagle-jpm/eagle-jpm-mr-history/src/test/java/org/apache/eagle/jpm/mr/history/MRHistoryJobApplicationProviderTest.java
@@ -0,0 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.eagle.jpm.mr.history;
+
+import com.google.inject.Inject;
+import org.apache.eagle.app.test.AppJUnitRunner;
+import org.apache.eagle.app.test.ApplicationSimulator;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+
+@RunWith(AppJUnitRunner.class)
+public class MRHistoryJobApplicationProviderTest {
+    @Inject private ApplicationSimulator simulator;
+
+    @Test
+    public void testRunAsManagedApplicationWithSimulator(){
+        simulator.start(MRHistoryJobApplicationProvider.class);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/0bde482b/eagle-jpm/eagle-jpm-mr-history/src/test/java/org/apache/eagle/jpm/mr/history/MRHistoryJobApplicationTest.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-mr-history/src/test/java/org/apache/eagle/jpm/mr/history/MRHistoryJobApplicationTest.java b/eagle-jpm/eagle-jpm-mr-history/src/test/java/org/apache/eagle/jpm/mr/history/MRHistoryJobApplicationTest.java
new file mode 100644
index 0000000..318a641
--- /dev/null
+++ b/eagle-jpm/eagle-jpm-mr-history/src/test/java/org/apache/eagle/jpm/mr/history/MRHistoryJobApplicationTest.java
@@ -0,0 +1,27 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.eagle.jpm.mr.history;
+
+import com.typesafe.config.ConfigFactory;
+import org.junit.Test;
+
+public class MRHistoryJobApplicationTest {
+    @Test
+    public void testRun(){
+        new MRHistoryJobApplication().run(ConfigFactory.load());
+    }
+}


[28/52] [abbrv] incubator-eagle git commit: Remove useless kafka adminUtil dependency; Add ignoreUnkonwProperties for models to help compatibility

Posted by yo...@apache.org.
Remove useless kafka adminUtil dependency;
Add ignoreUnkonwProperties for models to help compatibility

Author:ralphsu


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

Branch: refs/heads/master
Commit: 8de69a88df0df12363007a47a801e71c29269953
Parents: a66f64c
Author: Ralph, Su <su...@gmail.com>
Authored: Wed Aug 31 18:12:56 2016 +0800
Committer: Ralph, Su <su...@gmail.com>
Committed: Wed Aug 31 18:12:56 2016 +0800

----------------------------------------------------------------------
 .../engine/coordinator/PolicyDefinition.java    |  3 +
 .../alert/engine/coordinator/Publishment.java   |  2 +
 .../engine/coordinator/PublishmentType.java     |  2 +
 .../engine/coordinator/StreamPartition.java     |  2 +
 .../engine/coordinator/StreamSortSpec.java      |  6 +-
 .../eagle/correlation/meta/LocalKafkaTest.java  | 63 --------------------
 .../alert/engine/spout/CreateTopicUtils.java    | 23 +++----
 .../eagle/alert/engine/e2e/Integration1.java    | 21 -------
 8 files changed, 21 insertions(+), 101 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/8de69a88/eagle-core/eagle-alert-parent/eagle-alert/alert-common/src/main/java/org/apache/eagle/alert/engine/coordinator/PolicyDefinition.java
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-alert-parent/eagle-alert/alert-common/src/main/java/org/apache/eagle/alert/engine/coordinator/PolicyDefinition.java b/eagle-core/eagle-alert-parent/eagle-alert/alert-common/src/main/java/org/apache/eagle/alert/engine/coordinator/PolicyDefinition.java
index 827172f..e2dfb6f 100644
--- a/eagle-core/eagle-alert-parent/eagle-alert/alert-common/src/main/java/org/apache/eagle/alert/engine/coordinator/PolicyDefinition.java
+++ b/eagle-core/eagle-alert-parent/eagle-alert/alert-common/src/main/java/org/apache/eagle/alert/engine/coordinator/PolicyDefinition.java
@@ -19,6 +19,7 @@ package org.apache.eagle.alert.engine.coordinator;
 import org.apache.commons.collections.CollectionUtils;
 import org.apache.commons.collections.ListUtils;
 import org.apache.commons.lang3.builder.HashCodeBuilder;
+import org.codehaus.jackson.annotate.JsonIgnoreProperties;
 
 import java.io.Serializable;
 import java.util.ArrayList;
@@ -29,6 +30,7 @@ import java.util.Objects;
  * @since Apr 5, 2016
  *
  */
+@JsonIgnoreProperties(ignoreUnknown = true)
 public class PolicyDefinition implements Serializable{
     private static final long serialVersionUID = 377581499339572414L;
     // unique identifier
@@ -157,6 +159,7 @@ public class PolicyDefinition implements Serializable{
         return false;
     }
 
+    @JsonIgnoreProperties(ignoreUnknown = true)
     public static class Definition implements Serializable{
         private static final long serialVersionUID = -622366527887848346L;
 

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/8de69a88/eagle-core/eagle-alert-parent/eagle-alert/alert-common/src/main/java/org/apache/eagle/alert/engine/coordinator/Publishment.java
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-alert-parent/eagle-alert/alert-common/src/main/java/org/apache/eagle/alert/engine/coordinator/Publishment.java b/eagle-core/eagle-alert-parent/eagle-alert/alert-common/src/main/java/org/apache/eagle/alert/engine/coordinator/Publishment.java
index b16714f..3c1da58 100644
--- a/eagle-core/eagle-alert-parent/eagle-alert/alert-common/src/main/java/org/apache/eagle/alert/engine/coordinator/Publishment.java
+++ b/eagle-core/eagle-alert-parent/eagle-alert/alert-common/src/main/java/org/apache/eagle/alert/engine/coordinator/Publishment.java
@@ -17,6 +17,7 @@
 package org.apache.eagle.alert.engine.coordinator;
 
 import org.apache.commons.lang3.builder.HashCodeBuilder;
+import org.codehaus.jackson.annotate.JsonIgnoreProperties;
 
 import java.util.List;
 import java.util.Map;
@@ -26,6 +27,7 @@ import java.util.Objects;
  * @since Apr 11, 2016
  *
  */
+@JsonIgnoreProperties(ignoreUnknown = true)
 public class Publishment {
 
     private String name;

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/8de69a88/eagle-core/eagle-alert-parent/eagle-alert/alert-common/src/main/java/org/apache/eagle/alert/engine/coordinator/PublishmentType.java
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-alert-parent/eagle-alert/alert-common/src/main/java/org/apache/eagle/alert/engine/coordinator/PublishmentType.java b/eagle-core/eagle-alert-parent/eagle-alert/alert-common/src/main/java/org/apache/eagle/alert/engine/coordinator/PublishmentType.java
index f34b971..a3f2356 100644
--- a/eagle-core/eagle-alert-parent/eagle-alert/alert-common/src/main/java/org/apache/eagle/alert/engine/coordinator/PublishmentType.java
+++ b/eagle-core/eagle-alert-parent/eagle-alert/alert-common/src/main/java/org/apache/eagle/alert/engine/coordinator/PublishmentType.java
@@ -21,7 +21,9 @@ package org.apache.eagle.alert.engine.coordinator;
 import java.util.Objects;
 
 import org.apache.commons.lang3.builder.HashCodeBuilder;
+import org.codehaus.jackson.annotate.JsonIgnoreProperties;
 
+@JsonIgnoreProperties(ignoreUnknown = true)
 public class PublishmentType {
 
     private String type;

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/8de69a88/eagle-core/eagle-alert-parent/eagle-alert/alert-common/src/main/java/org/apache/eagle/alert/engine/coordinator/StreamPartition.java
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-alert-parent/eagle-alert/alert-common/src/main/java/org/apache/eagle/alert/engine/coordinator/StreamPartition.java b/eagle-core/eagle-alert-parent/eagle-alert/alert-common/src/main/java/org/apache/eagle/alert/engine/coordinator/StreamPartition.java
index 7b96024..cdf0dbf 100644
--- a/eagle-core/eagle-alert-parent/eagle-alert/alert-common/src/main/java/org/apache/eagle/alert/engine/coordinator/StreamPartition.java
+++ b/eagle-core/eagle-alert-parent/eagle-alert/alert-common/src/main/java/org/apache/eagle/alert/engine/coordinator/StreamPartition.java
@@ -22,6 +22,7 @@ import java.util.*;
 import org.apache.commons.collections.CollectionUtils;
 import org.apache.commons.lang3.StringUtils;
 import org.apache.commons.lang3.builder.HashCodeBuilder;
+import org.codehaus.jackson.annotate.JsonIgnoreProperties;
 
 /**
  * StreamPartition defines how a data stream is partitioned and sorted
@@ -30,6 +31,7 @@ import org.apache.commons.lang3.builder.HashCodeBuilder;
  * columns are fields based on which stream is grouped
  * sortSpec defines how data is sorted
  */
+@JsonIgnoreProperties(ignoreUnknown = true)
 public class StreamPartition implements Serializable {
     private static final long serialVersionUID = -3361648309136926040L;
 

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/8de69a88/eagle-core/eagle-alert-parent/eagle-alert/alert-common/src/main/java/org/apache/eagle/alert/engine/coordinator/StreamSortSpec.java
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-alert-parent/eagle-alert/alert-common/src/main/java/org/apache/eagle/alert/engine/coordinator/StreamSortSpec.java b/eagle-core/eagle-alert-parent/eagle-alert/alert-common/src/main/java/org/apache/eagle/alert/engine/coordinator/StreamSortSpec.java
index ee20f81..a8b57f2 100644
--- a/eagle-core/eagle-alert-parent/eagle-alert/alert-common/src/main/java/org/apache/eagle/alert/engine/coordinator/StreamSortSpec.java
+++ b/eagle-core/eagle-alert-parent/eagle-alert/alert-common/src/main/java/org/apache/eagle/alert/engine/coordinator/StreamSortSpec.java
@@ -17,15 +17,17 @@
 package org.apache.eagle.alert.engine.coordinator;
 
 
-import java.io.Serializable;
-
 import org.apache.commons.lang.builder.HashCodeBuilder;
 import org.apache.eagle.alert.utils.TimePeriodUtils;
+import org.codehaus.jackson.annotate.JsonIgnoreProperties;
 import org.joda.time.Period;
 
+import java.io.Serializable;
+
 /**
  * streamId is the key
  */
+@JsonIgnoreProperties(ignoreUnknown = true)
 public class StreamSortSpec implements Serializable{
     private static final long serialVersionUID = 3626506441441584937L;
     private String windowPeriod="";

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/8de69a88/eagle-core/eagle-alert-parent/eagle-alert/alert-common/src/test/java/org/apache/eagle/correlation/meta/LocalKafkaTest.java
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-alert-parent/eagle-alert/alert-common/src/test/java/org/apache/eagle/correlation/meta/LocalKafkaTest.java b/eagle-core/eagle-alert-parent/eagle-alert/alert-common/src/test/java/org/apache/eagle/correlation/meta/LocalKafkaTest.java
deleted file mode 100644
index d8bd942..0000000
--- a/eagle-core/eagle-alert-parent/eagle-alert/alert-common/src/test/java/org/apache/eagle/correlation/meta/LocalKafkaTest.java
+++ /dev/null
@@ -1,63 +0,0 @@
-package org.apache.eagle.correlation.meta;
-
-/*
- * 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.
- */
-import kafka.admin.AdminUtils;
-import kafka.utils.ZKStringSerializer$;
-import org.I0Itec.zkclient.ZkClient;
-import org.I0Itec.zkclient.ZkConnection;
-import org.apache.eagle.alert.utils.KafkaEmbedded;
-import org.junit.Ignore;
-
-import java.util.Properties;
-
-/**
- * @since Jun 3, 2016
- *
- */
-@Ignore
-public class LocalKafkaTest {
-
-    /**
-     * @param args
-     */
-    public static void main(String[] args) {
-        KafkaEmbedded kafka = new KafkaEmbedded(9092, 2181);
-
-        makeSureTopic("local_kafka_topic");
-
-        while (true) {
-            try {
-                Thread.sleep(3000);
-            } catch (Exception e) {
-                break;
-            }
-        }
-
-        kafka.shutdown();
-
-    }
-
-    public static void makeSureTopic(String topic) {
-        ZkClient zkClient = new ZkClient("localhost:2181", 10000, 10000, ZKStringSerializer$.MODULE$);
-        Properties topicConfiguration = new Properties();
-        ZkConnection zkConnection = new ZkConnection("localhost:2181");
-//        ZkUtils zkUtils = new ZkUtils(zkClient, zkConnection, false);
-        AdminUtils.createTopic(zkClient, topic, 1, 1, topicConfiguration);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/8de69a88/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/main/java/org/apache/eagle/alert/engine/spout/CreateTopicUtils.java
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/main/java/org/apache/eagle/alert/engine/spout/CreateTopicUtils.java b/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/main/java/org/apache/eagle/alert/engine/spout/CreateTopicUtils.java
index 5f43d11..22993b3 100644
--- a/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/main/java/org/apache/eagle/alert/engine/spout/CreateTopicUtils.java
+++ b/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/main/java/org/apache/eagle/alert/engine/spout/CreateTopicUtils.java
@@ -18,13 +18,6 @@
  */
 package org.apache.eagle.alert.engine.spout;
 
-import java.util.Properties;
-
-import kafka.admin.AdminUtils;
-import kafka.utils.ZKStringSerializer$;
-
-import org.I0Itec.zkclient.ZkClient;
-import org.I0Itec.zkclient.ZkConnection;
 import org.slf4j.Logger;
 
 /**
@@ -38,13 +31,13 @@ public class CreateTopicUtils {
     private static final int replicationFactor = 1;
 
     public static void ensureTopicReady(String zkQuorum, String topic) {
-        ZkConnection zkConnection = new ZkConnection(zkQuorum);
-        ZkClient zkClient = new ZkClient(zkQuorum, 10000, 10000, ZKStringSerializer$.MODULE$);
-//        ZkUtils zkUtils = new ZkUtils(zkClient, zkConnection, false);
-        if (!AdminUtils.topicExists(zkClient, topic)) {
-            LOG.info("create topic " + topic + " with partitions " + partitions + ", and replicationFactor "
-                    + replicationFactor);
-            AdminUtils.createTopic(zkClient, topic, partitions, replicationFactor, new Properties());
-        }
+//        ZkConnection zkConnection = new ZkConnection(zkQuorum);
+//        ZkClient zkClient = new ZkClient(zkQuorum, 10000, 10000, ZKStringSerializer$.MODULE$);
+////        ZkUtils zkUtils = new ZkUtils(zkClient, zkConnection, false);
+//        if (!AdminUtils.topicExists(zkClient, topic)) {
+//            LOG.info("create topic " + topic + " with partitions " + partitions + ", and replicationFactor "
+//                    + replicationFactor);
+//            AdminUtils.createTopic(zkClient, topic, partitions, replicationFactor, new Properties());
+//        }
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/8de69a88/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/test/java/org/apache/eagle/alert/engine/e2e/Integration1.java
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/test/java/org/apache/eagle/alert/engine/e2e/Integration1.java b/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/test/java/org/apache/eagle/alert/engine/e2e/Integration1.java
index d05b7ee..db8e0a2 100644
--- a/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/test/java/org/apache/eagle/alert/engine/e2e/Integration1.java
+++ b/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/test/java/org/apache/eagle/alert/engine/e2e/Integration1.java
@@ -23,11 +23,6 @@ import com.fasterxml.jackson.databind.type.CollectionType;
 import com.fasterxml.jackson.databind.type.SimpleType;
 import com.typesafe.config.Config;
 import com.typesafe.config.ConfigFactory;
-import kafka.admin.AdminUtils;
-import kafka.utils.ZKStringSerializer$;
-import org.I0Itec.zkclient.ZkClient;
-import org.apache.eagle.alert.config.ZKConfig;
-import org.apache.eagle.alert.config.ZKConfigBuilder;
 import org.apache.eagle.alert.coordination.model.Kafka2TupleMetadata;
 import org.apache.eagle.alert.coordination.model.internal.Topology;
 import org.apache.eagle.alert.engine.UnitTopologyMain;
@@ -42,7 +37,6 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import java.util.List;
-import java.util.Properties;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
 import java.util.concurrent.ThreadFactory;
@@ -139,21 +133,6 @@ public class Integration1 {
         }
     }
 
-    public static void makeSureTopic(String topic) {
-        System.setProperty("config.resource", SIMPLE_CONFIG);
-        ConfigFactory.invalidateCaches();
-        Config config = ConfigFactory.load();
-        ZKConfig zkconfig = ZKConfigBuilder.getZKConfig(config);
-
-        ZkClient zkClient = new ZkClient(zkconfig.zkQuorum, 10000, 10000, ZKStringSerializer$.MODULE$);
-        Properties topicConfiguration = new Properties();
-//        ZkUtils zkUtils = new ZkUtils(zkClient, zkConnection, false);
-        AdminUtils.createTopic(zkClient, topic, 1, 1, topicConfiguration);// RackAwareMode.Disabled$.MODULE$);
-
-        AdminUtils.createTopic(zkClient, topic, 1, 1, topicConfiguration);
-
-    }
-
     public static void proactive_schedule(Config config) throws Exception {
         try (CoordinatorClient cc = new CoordinatorClient(config)) {
             try {


[16/52] [abbrv] incubator-eagle git commit: [minor] Change resourceFetch to resourcefetch

Posted by yo...@apache.org.
[minor] Change resourceFetch to resourcefetch

Author: Zhao, Qingwen <qi...@apache.org>

Closes #386.


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

Branch: refs/heads/master
Commit: b5fd782c0968ce8dd499de1899464c8edbf50dfd
Parents: 6f5f972
Author: Qingwen Zhao <qi...@gmail.com>
Authored: Thu Aug 25 11:45:23 2016 +0800
Committer: Qingwen Zhao <qi...@gmail.com>
Committed: Thu Aug 25 13:00:44 2016 +0800

----------------------------------------------------------------------
 .../util/resourceFetch/RMResourceFetcher.java   | 189 ------------
 .../jpm/util/resourceFetch/ResourceFetcher.java |  27 --
 .../SparkHistoryServerResourceFetcher.java      |  88 ------
 .../connection/InputStreamUtils.java            |  68 -----
 .../util/resourceFetch/connection/JobUtils.java |  43 ---
 .../connection/URLConnectionUtils.java          | 101 -------
 .../resourceFetch/ha/AbstractURLSelector.java   | 110 -------
 .../util/resourceFetch/ha/HAURLSelector.java    |  28 --
 .../resourceFetch/ha/HAURLSelectorImpl.java     | 109 -------
 .../jpm/util/resourceFetch/model/AppInfo.java   | 210 --------------
 .../util/resourceFetch/model/Applications.java  |  38 ---
 .../util/resourceFetch/model/AppsWrapper.java   |  37 ---
 .../util/resourceFetch/model/ClusterInfo.java   | 119 --------
 .../resourceFetch/model/ClusterInfoWrapper.java |  35 ---
 .../resourceFetch/model/JobCounterGroup.java    |  46 ---
 .../resourceFetch/model/JobCounterItem.java     |  62 ----
 .../util/resourceFetch/model/JobCounters.java   |  46 ---
 .../resourceFetch/model/JobCountersWrapper.java |  35 ---
 .../jpm/util/resourceFetch/model/MRJob.java     | 289 -------------------
 .../util/resourceFetch/model/MRJobsWrapper.java |  37 ---
 .../jpm/util/resourceFetch/model/MRTask.java    | 109 -------
 .../util/resourceFetch/model/MRTaskAttempt.java | 136 ---------
 .../model/MRTaskAttemptWrapper.java             |  37 ---
 .../resourceFetch/model/MRTaskAttempts.java     |  39 ---
 .../jpm/util/resourceFetch/model/MRTasks.java   |  40 ---
 .../resourceFetch/model/MRTasksWrapper.java     |  37 ---
 .../jpm/util/resourceFetch/model/MrJobs.java    |  39 ---
 .../resourceFetch/model/SparkApplication.java   |  57 ----
 .../model/SparkApplicationAttempt.java          |  73 -----
 .../model/SparkApplicationWrapper.java          |  38 ---
 .../util/resourceFetch/model/SparkExecutor.java | 155 ----------
 .../jpm/util/resourceFetch/model/SparkJob.java  | 165 -----------
 .../util/resourceFetch/model/SparkStage.java    | 211 --------------
 .../jpm/util/resourceFetch/model/SparkTask.java | 111 -------
 .../model/SparkTaskInputMetrics.java            |  46 ---
 .../resourceFetch/model/SparkTaskMetrics.java   | 118 --------
 .../model/SparkTaskShuffleReadMetrics.java      |  82 ------
 .../model/SparkTaskShuffleWriteMetrics.java     |  55 ----
 .../resourceFetch/model/TaskCounterGroup.java   |  46 ---
 .../resourceFetch/model/TaskCounterItem.java    |  44 ---
 .../util/resourceFetch/model/TaskCounters.java  |  46 ---
 .../model/TaskCountersWrapper.java              |  35 ---
 .../url/JobListServiceURLBuilderImpl.java       |  52 ----
 .../resourceFetch/url/ServiceURLBuilder.java    |  21 --
 .../SparkCompleteJobServiceURLBuilderImpl.java  |  33 ---
 .../url/SparkJobServiceURLBuilderImpl.java      |  31 --
 .../jpm/util/resourceFetch/url/URLUtil.java     |  33 ---
 .../util/resourcefetch/RMResourceFetcher.java   | 189 ++++++++++++
 .../jpm/util/resourcefetch/ResourceFetcher.java |  27 ++
 .../SparkHistoryServerResourceFetcher.java      |  88 ++++++
 .../connection/InputStreamUtils.java            |  68 +++++
 .../util/resourcefetch/connection/JobUtils.java |  43 +++
 .../connection/URLConnectionUtils.java          | 101 +++++++
 .../resourcefetch/ha/AbstractURLSelector.java   | 110 +++++++
 .../util/resourcefetch/ha/HAURLSelector.java    |  28 ++
 .../resourcefetch/ha/HAURLSelectorImpl.java     | 109 +++++++
 .../jpm/util/resourcefetch/model/AppInfo.java   | 210 ++++++++++++++
 .../util/resourcefetch/model/Applications.java  |  38 +++
 .../util/resourcefetch/model/AppsWrapper.java   |  37 +++
 .../util/resourcefetch/model/ClusterInfo.java   | 119 ++++++++
 .../resourcefetch/model/ClusterInfoWrapper.java |  35 +++
 .../resourcefetch/model/JobCounterGroup.java    |  46 +++
 .../resourcefetch/model/JobCounterItem.java     |  62 ++++
 .../util/resourcefetch/model/JobCounters.java   |  46 +++
 .../resourcefetch/model/JobCountersWrapper.java |  35 +++
 .../jpm/util/resourcefetch/model/MRJob.java     | 289 +++++++++++++++++++
 .../util/resourcefetch/model/MRJobsWrapper.java |  37 +++
 .../jpm/util/resourcefetch/model/MRTask.java    | 109 +++++++
 .../util/resourcefetch/model/MRTaskAttempt.java | 136 +++++++++
 .../model/MRTaskAttemptWrapper.java             |  37 +++
 .../resourcefetch/model/MRTaskAttempts.java     |  39 +++
 .../jpm/util/resourcefetch/model/MRTasks.java   |  40 +++
 .../resourcefetch/model/MRTasksWrapper.java     |  37 +++
 .../jpm/util/resourcefetch/model/MrJobs.java    |  39 +++
 .../resourcefetch/model/SparkApplication.java   |  57 ++++
 .../model/SparkApplicationAttempt.java          |  73 +++++
 .../model/SparkApplicationWrapper.java          |  38 +++
 .../util/resourcefetch/model/SparkExecutor.java | 155 ++++++++++
 .../jpm/util/resourcefetch/model/SparkJob.java  | 165 +++++++++++
 .../util/resourcefetch/model/SparkStage.java    | 211 ++++++++++++++
 .../jpm/util/resourcefetch/model/SparkTask.java | 111 +++++++
 .../model/SparkTaskInputMetrics.java            |  46 +++
 .../resourcefetch/model/SparkTaskMetrics.java   | 118 ++++++++
 .../model/SparkTaskShuffleReadMetrics.java      |  82 ++++++
 .../model/SparkTaskShuffleWriteMetrics.java     |  55 ++++
 .../resourcefetch/model/TaskCounterGroup.java   |  46 +++
 .../resourcefetch/model/TaskCounterItem.java    |  44 +++
 .../util/resourcefetch/model/TaskCounters.java  |  46 +++
 .../model/TaskCountersWrapper.java              |  35 +++
 .../url/JobListServiceURLBuilderImpl.java       |  52 ++++
 .../resourcefetch/url/ServiceURLBuilder.java    |  21 ++
 .../SparkCompleteJobServiceURLBuilderImpl.java  |  33 +++
 .../url/SparkJobServiceURLBuilderImpl.java      |  31 ++
 .../jpm/util/resourcefetch/url/URLUtil.java     |  33 +++
 94 files changed, 3606 insertions(+), 3606 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/b5fd782c/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/RMResourceFetcher.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/RMResourceFetcher.java b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/RMResourceFetcher.java
deleted file mode 100644
index b1881ef..0000000
--- a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/RMResourceFetcher.java
+++ /dev/null
@@ -1,189 +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.eagle.jpm.util.resourcefetch;
-
-import org.apache.eagle.jpm.util.Constants;
-import org.apache.eagle.jpm.util.resourcefetch.connection.InputStreamUtils;
-import org.apache.eagle.jpm.util.resourcefetch.ha.HAURLSelector;
-import org.apache.eagle.jpm.util.resourcefetch.ha.HAURLSelectorImpl;
-import org.apache.eagle.jpm.util.resourcefetch.model.AppInfo;
-import org.apache.eagle.jpm.util.resourcefetch.model.AppsWrapper;
-import org.apache.eagle.jpm.util.resourcefetch.model.ClusterInfo;
-import org.apache.eagle.jpm.util.resourcefetch.model.ClusterInfoWrapper;
-import org.apache.eagle.jpm.util.resourcefetch.url.JobListServiceURLBuilderImpl;
-import org.apache.eagle.jpm.util.resourcefetch.url.ServiceURLBuilder;
-import org.apache.eagle.jpm.util.resourcefetch.url.SparkCompleteJobServiceURLBuilderImpl;
-import org.apache.eagle.jpm.util.resourcefetch.url.URLUtil;
-import org.codehaus.jackson.JsonParser;
-import org.codehaus.jackson.map.ObjectMapper;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-import java.io.InputStream;
-import java.util.List;
-
-public class RMResourceFetcher implements ResourceFetcher<AppInfo> {
-
-    private static final Logger LOG = LoggerFactory.getLogger(RMResourceFetcher.class);
-    private final HAURLSelector selector;
-    private final ServiceURLBuilder jobListServiceURLBuilder;
-    private final ServiceURLBuilder sparkCompleteJobServiceURLBuilder;
-    private static final ObjectMapper OBJ_MAPPER = new ObjectMapper();
-
-    static {
-        OBJ_MAPPER.configure(JsonParser.Feature.ALLOW_NON_NUMERIC_NUMBERS, true);
-    }
-
-    public RMResourceFetcher(String[] rmBasePaths) {
-        this.jobListServiceURLBuilder = new JobListServiceURLBuilderImpl();
-        this.sparkCompleteJobServiceURLBuilder = new SparkCompleteJobServiceURLBuilderImpl();
-
-        this.selector = new HAURLSelectorImpl(rmBasePaths, jobListServiceURLBuilder, Constants.CompressionType.GZIP);
-    }
-
-    private void checkUrl() throws IOException {
-        if (!selector.checkUrl(jobListServiceURLBuilder.build(selector.getSelectedUrl(), Constants.JobState.RUNNING.name()))) {
-            selector.reSelectUrl();
-        }
-    }
-
-    private List<AppInfo> doFetchFinishApplicationsList(String urlString, Constants.CompressionType compressionType) throws Exception {
-        List<AppInfo> result;
-        InputStream is = null;
-        try {
-            checkUrl();
-            LOG.info("Going to call yarn api to fetch finished application list: " + urlString);
-            is = InputStreamUtils.getInputStream(urlString, null, compressionType);
-            final AppsWrapper appWrapper = OBJ_MAPPER.readValue(is, AppsWrapper.class);
-            if (appWrapper != null && appWrapper.getApps() != null
-                && appWrapper.getApps().getApp() != null) {
-                result = appWrapper.getApps().getApp();
-                return result;
-            }
-            return null;
-        } finally {
-            if (is != null) {
-                try {
-                    is.close();
-                } catch (Exception e) {
-                    LOG.warn("{}", e);
-                }
-            }
-        }
-    }
-
-    private String getSparkRunningJobURL() {
-        return selector.getSelectedUrl()
-                + "/"
-                + Constants.V2_APPS_URL
-                + "?applicationTypes=SPARK&state=RUNNING&"
-                + Constants.ANONYMOUS_PARAMETER;
-    }
-
-    private String getMRRunningJobURL() {
-        return String.format("%s/%s?applicationTypes=MAPREDUCE&state=RUNNING&%s",
-            selector.getSelectedUrl(),
-            Constants.V2_APPS_URL,
-            Constants.ANONYMOUS_PARAMETER);
-    }
-
-    private String getMRFinishedJobURL(String lastFinishedTime) {
-        String url = URLUtil.removeTrailingSlash(selector.getSelectedUrl());
-        return url + "/" + "Constants.V2_APPS_URL"
-                + "?applicationTypes=MAPREDUCE&state=FINISHED&finishedTimeBegin="
-                + lastFinishedTime + "&" + Constants.ANONYMOUS_PARAMETER;
-    }
-
-    private List<AppInfo> doFetchRunningApplicationsList(String urlString, Constants.CompressionType compressionType) throws Exception {
-        List<AppInfo> result;
-        InputStream is = null;
-        try {
-            checkUrl();
-            LOG.info("Going to call yarn api to fetch running application list: " + urlString);
-            is = InputStreamUtils.getInputStream(urlString, null, compressionType);
-            final AppsWrapper appWrapper = OBJ_MAPPER.readValue(is, AppsWrapper.class);
-            if (appWrapper != null && appWrapper.getApps() != null && appWrapper.getApps().getApp() != null) {
-                result = appWrapper.getApps().getApp();
-                return result;
-            }
-            return null;
-        } finally {
-            if (is != null) {
-                try {
-                    is.close();
-                } catch (Exception e) {
-                    LOG.warn("{}", e);
-                }
-            }
-        }
-    }
-
-    private List<AppInfo> getResource(Constants.ResourceType resourceType, Constants.CompressionType compressionType, Object... parameter) throws Exception {
-        switch (resourceType) {
-            case COMPLETE_SPARK_JOB:
-                final String urlString = sparkCompleteJobServiceURLBuilder.build(selector.getSelectedUrl(), (String) parameter[0]);
-                return doFetchFinishApplicationsList(urlString, compressionType);
-            case RUNNING_SPARK_JOB:
-                return doFetchRunningApplicationsList(getSparkRunningJobURL(), compressionType);
-            case RUNNING_MR_JOB:
-                return doFetchRunningApplicationsList(getMRRunningJobURL(), compressionType);
-            case COMPLETE_MR_JOB:
-                return doFetchFinishApplicationsList(getMRFinishedJobURL((String) parameter[0]), compressionType);
-            default:
-                throw new Exception("Not support resourceType :" + resourceType);
-        }
-    }
-
-    public List<AppInfo> getResource(Constants.ResourceType resourceType, Object... parameter) throws Exception {
-        try {
-            return getResource(resourceType, Constants.CompressionType.GZIP, parameter);
-        } catch (java.util.zip.ZipException ex) {
-            return getResource(resourceType, Constants.CompressionType.NONE, parameter);
-        }
-    }
-
-    private String getClusterInfoURL() {
-        return selector.getSelectedUrl() + "/" + Constants.YARN_API_CLUSTER_INFO + "?" + Constants.ANONYMOUS_PARAMETER;
-    }
-
-    public ClusterInfo getClusterInfo() throws Exception {
-        InputStream is = null;
-        try {
-            checkUrl();
-            final String urlString = getClusterInfoURL();
-            LOG.info("Calling yarn api to fetch cluster info: " + urlString);
-            is = InputStreamUtils.getInputStream(urlString, null, Constants.CompressionType.GZIP);
-            final ClusterInfoWrapper clusterInfoWrapper = OBJ_MAPPER.readValue(is, ClusterInfoWrapper.class);
-            if (clusterInfoWrapper != null && clusterInfoWrapper.getClusterInfo() != null) {
-                return clusterInfoWrapper.getClusterInfo();
-            }
-            return null;
-        } finally {
-            if (is != null) {
-                try {
-                    is.close();
-                } catch (Exception e) {
-                    LOG.warn("{}", e);
-                }
-            }
-        }
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/b5fd782c/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/ResourceFetcher.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/ResourceFetcher.java b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/ResourceFetcher.java
deleted file mode 100644
index f920ddb..0000000
--- a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/ResourceFetcher.java
+++ /dev/null
@@ -1,27 +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.eagle.jpm.util.resourcefetch;
-
-import org.apache.eagle.jpm.util.Constants;
-
-import java.util.List;
-
-public interface ResourceFetcher<T> {
-    //continue to refactor later
-    List<T> getResource(Constants.ResourceType resoureType, Object... parameter) throws Exception;
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/b5fd782c/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/SparkHistoryServerResourceFetcher.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/SparkHistoryServerResourceFetcher.java b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/SparkHistoryServerResourceFetcher.java
deleted file mode 100644
index ce2d9b8..0000000
--- a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/SparkHistoryServerResourceFetcher.java
+++ /dev/null
@@ -1,88 +0,0 @@
-/*
- *
- *  * Licensed to the Apache Software Foundation (ASF) under one or more
- *  * contributor license agreements.  See the NOTICE file distributed with
- *  * this work for additional information regarding copyright ownership.
- *  * The ASF licenses this file to You under the Apache License, Version 2.0
- *  * (the "License"); you may not use this file except in compliance with
- *  * the License.  You may obtain a copy of the License at
- *  *
- *  *    http://www.apache.org/licenses/LICENSE-2.0
- *  *
- *  * Unless required by applicable law or agreed to in writing, software
- *  * distributed under the License is distributed on an "AS IS" BASIS,
- *  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- *  * See the License for the specific language governing permissions and
- *  * limitations under the License.
- *
- */
-
-package org.apache.eagle.jpm.util.resourcefetch;
-
-import org.apache.eagle.jpm.util.Constants;
-import org.apache.eagle.jpm.util.resourcefetch.connection.InputStreamUtils;
-import org.apache.eagle.jpm.util.resourcefetch.model.SparkApplication;
-import org.apache.eagle.jpm.util.resourcefetch.url.ServiceURLBuilder;
-import org.apache.eagle.jpm.util.resourcefetch.url.SparkJobServiceURLBuilderImpl;
-import org.apache.commons.codec.binary.Base64;
-import org.codehaus.jackson.JsonParser;
-import org.codehaus.jackson.map.ObjectMapper;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.FileNotFoundException;
-import java.io.InputStream;
-import java.util.Arrays;
-import java.util.List;
-
-public class SparkHistoryServerResourceFetcher implements ResourceFetcher<SparkApplication> {
-
-    private static final Logger LOG = LoggerFactory.getLogger(SparkHistoryServerResourceFetcher.class);
-
-    private static final ObjectMapper OBJ_MAPPER = new ObjectMapper();
-
-    private String historyServerURL;
-    private final ServiceURLBuilder sparkDetailJobServiceURLBuilder;
-    private String auth;
-
-    static {
-        OBJ_MAPPER.configure(JsonParser.Feature.ALLOW_NON_NUMERIC_NUMBERS, true);
-    }
-
-    public SparkHistoryServerResourceFetcher(String historyServerURL, String userName, String pwd) {
-        this.historyServerURL = historyServerURL;
-        this.sparkDetailJobServiceURLBuilder = new SparkJobServiceURLBuilderImpl();
-        this.auth = "Basic " + new String(new Base64().encode(String.format("%s:%s", userName, pwd).getBytes()));
-        ;
-    }
-
-    private List<SparkApplication> doFetchSparkApplicationDetail(String appId) throws Exception {
-        InputStream is = null;
-        try {
-            final String urlString = sparkDetailJobServiceURLBuilder.build(this.historyServerURL, appId);
-            LOG.info("Going to call spark history server api to fetch spark job: " + urlString);
-            is = InputStreamUtils.getInputStream(urlString, auth, Constants.CompressionType.NONE);
-            SparkApplication app = OBJ_MAPPER.readValue(is, SparkApplication.class);
-            return Arrays.asList(app);
-        } catch (FileNotFoundException e) {
-            return null;
-        } finally {
-            if (is != null) {
-                try {
-                    is.close();
-                } catch (Exception e) {
-                    LOG.warn("{}", e);
-                }
-            }
-        }
-    }
-
-    public List<SparkApplication> getResource(Constants.ResourceType resoureType, Object... parameter) throws Exception {
-        switch (resoureType) {
-            case SPARK_JOB_DETAIL:
-                return doFetchSparkApplicationDetail((String) parameter[0]);
-            default:
-                throw new Exception("Not support resourceType :" + resoureType);
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/b5fd782c/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/connection/InputStreamUtils.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/connection/InputStreamUtils.java b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/connection/InputStreamUtils.java
deleted file mode 100644
index e99e2aa..0000000
--- a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/connection/InputStreamUtils.java
+++ /dev/null
@@ -1,68 +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.eagle.jpm.util.resourcefetch.connection;
-
-import org.apache.eagle.jpm.util.Constants;
-
-import java.io.IOException;
-import java.io.InputStream;
-import java.net.URL;
-import java.net.URLConnection;
-import java.util.zip.GZIPInputStream;
-
-public class InputStreamUtils {
-
-    private static final int CONNECTION_TIMEOUT = 10 * 1000;
-    private static final int READ_TIMEOUT = 5 * 60 * 1000;
-    private static final String GZIP_HTTP_HEADER = "Accept-Encoding";
-    private static final String GZIP_COMPRESSION = "gzip";
-
-    private static InputStream openGZIPInputStream(URL url, String auth, int timeout) throws IOException {
-        final URLConnection connection = url.openConnection();
-        connection.setConnectTimeout(CONNECTION_TIMEOUT);
-        connection.setReadTimeout(timeout);
-        connection.addRequestProperty(GZIP_HTTP_HEADER, GZIP_COMPRESSION);
-        if (null != auth) {
-            connection.setRequestProperty("Authorization", auth);
-        }
-        return new GZIPInputStream(connection.getInputStream());
-    }
-
-    private static InputStream openInputStream(URL url, String auth, int timeout) throws IOException {
-        URLConnection connection = url.openConnection();
-        connection.setConnectTimeout(timeout);
-        if (null != auth) {
-            connection.setRequestProperty("Authorization", auth);
-        }
-
-        return connection.getInputStream();
-    }
-
-    public static InputStream getInputStream(String urlString, String auth, Constants.CompressionType compressionType, int timeout) throws Exception {
-        final URL url = URLConnectionUtils.getUrl(urlString);
-        if (compressionType.equals(Constants.CompressionType.GZIP)) {
-            return openGZIPInputStream(url, auth, timeout);
-        } else { // CompressionType.NONE
-            return openInputStream(url, auth, timeout);
-        }
-    }
-
-    public static InputStream getInputStream(String urlString, String auth, Constants.CompressionType compressionType) throws Exception {
-        return getInputStream(urlString, auth, compressionType, READ_TIMEOUT);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/b5fd782c/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/connection/JobUtils.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/connection/JobUtils.java b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/connection/JobUtils.java
deleted file mode 100644
index 44d27e8..0000000
--- a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/connection/JobUtils.java
+++ /dev/null
@@ -1,43 +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.eagle.jpm.util.resourcefetch.connection;
-
-import org.apache.eagle.jpm.util.Constants;
-
-public class JobUtils {
-
-    public static String checkAndAddLastSlash(String urlBase) {
-        if (!urlBase.endsWith("/")) {
-            return urlBase + "/";
-        }
-        return urlBase;
-    }
-
-    public static String getJobIDByAppID(String appID) {
-        if (appID.startsWith(Constants.APPLICATION_PREFIX)) {
-            return appID.replace(Constants.APPLICATION_PREFIX, Constants.JOB_PREFIX);
-        }
-        return null;
-    }
-
-    public static String getAppIDByJobID(String jobID) {
-        if (jobID.startsWith(Constants.JOB_PREFIX)) {
-            return jobID.replace(Constants.JOB_PREFIX, Constants.APPLICATION_PREFIX);
-        }
-        return null;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/b5fd782c/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/connection/URLConnectionUtils.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/connection/URLConnectionUtils.java b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/connection/URLConnectionUtils.java
deleted file mode 100644
index 2cd60ba..0000000
--- a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/connection/URLConnectionUtils.java
+++ /dev/null
@@ -1,101 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.eagle.jpm.util.resourcefetch.connection;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import java.io.IOException;
-import java.net.MalformedURLException;
-import java.net.URL;
-import java.net.URLConnection;
-import java.security.KeyManagementException;
-import java.security.NoSuchAlgorithmException;
-import java.security.cert.CertificateException;
-import javax.net.ssl.*;
-
-public final class URLConnectionUtils {
-    //TODO: change some public method to private
-    private static final Logger LOG = LoggerFactory.getLogger(URLConnectionUtils.class);
-
-    public static URLConnection getConnection(String url) throws Exception {
-        if (url.startsWith("https://")) {
-            return getHTTPSConnection(url);
-        } else if (url.startsWith("http://")) {
-            return getHTTPConnection(url);
-        }
-        throw new Exception("Invalid input argument url: " + url);
-    }
-
-    public static URLConnection getHTTPConnection(String urlString) throws Exception {
-        final URL url = new URL(urlString);
-        return url.openConnection();
-    }
-
-    public static URL getUrl(String urlString) throws Exception {
-        if (urlString.toLowerCase().contains("https")) {
-            return getHTTPSUrl(urlString);
-        } else if (urlString.toLowerCase().contains("http")) {
-            return getURL(urlString);
-        }
-        throw new Exception("Invalid input argument url: " + urlString);
-    }
-
-    public static URL getURL(String urlString) throws MalformedURLException {
-        return new URL(urlString);
-    }
-
-    public static URL getHTTPSUrl(String urlString) throws MalformedURLException, NoSuchAlgorithmException, KeyManagementException {
-        // Create a trust manager that does not validate certificate chains
-        final TrustManager[] trustAllCerts = new TrustManager[] {new TrustAllX509TrustManager()};
-        // Install the all-trusting trust manager   
-        final SSLContext sc = SSLContext.getInstance("SSL");
-        sc.init(null, trustAllCerts, new java.security.SecureRandom());
-        HttpsURLConnection.setDefaultSSLSocketFactory(sc.getSocketFactory());
-        // Create all-trusting host name verifier   
-        final HostnameVerifier allHostsValid = new HostnameVerifier() {
-            public boolean verify(String hostname, SSLSession session) {
-                return true;
-            }
-        };
-        HttpsURLConnection.setDefaultHostnameVerifier(allHostsValid);
-        return new URL(urlString);
-    }
-
-    public static URLConnection getHTTPSConnection(String urlString) throws IOException, KeyManagementException, NoSuchAlgorithmException {
-        final URL url = getHTTPSUrl(urlString);
-        return url.openConnection();
-    }
-
-    public static class TrustAllX509TrustManager implements X509TrustManager {
-        @Override
-        public void checkClientTrusted(
-            java.security.cert.X509Certificate[] chain, String authType)
-            throws CertificateException {
-        }
-
-        @Override
-        public void checkServerTrusted(
-            java.security.cert.X509Certificate[] chain, String authType)
-            throws CertificateException {
-        }
-
-        @Override
-        public java.security.cert.X509Certificate[] getAcceptedIssuers() {
-            return null;
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/b5fd782c/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/ha/AbstractURLSelector.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/ha/AbstractURLSelector.java b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/ha/AbstractURLSelector.java
deleted file mode 100644
index d25d05b..0000000
--- a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/ha/AbstractURLSelector.java
+++ /dev/null
@@ -1,110 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- *  Unless required by applicable law or agreed to in writing, software
- *  distributed under the License is distributed on an "AS IS" BASIS,
- *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- *  See the License for the specific language governing permissions and
- *  limitations under the License.
- *
- */
-
-package org.apache.eagle.jpm.util.resourcefetch.ha;
-
-import org.apache.eagle.jpm.util.Constants;
-import org.apache.eagle.jpm.util.resourcefetch.connection.InputStreamUtils;
-import org.apache.hadoop.util.StringUtils;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-import java.io.InputStream;
-import java.util.Arrays;
-
-public abstract class AbstractURLSelector implements HAURLSelector {
-    private final String[] urls;
-    private volatile String selectedUrl;
-
-    private volatile boolean reselectInProgress;
-    private final Constants.CompressionType compressionType;
-
-    private static final long MAX_RETRY_TIME = 3;
-    private static final Logger LOG = LoggerFactory.getLogger(HAURLSelectorImpl.class);
-
-    public AbstractURLSelector(String[] urls, Constants.CompressionType compressionType) {
-        this.urls = urls;
-        this.compressionType = compressionType;
-    }
-
-    public boolean checkUrl(String urlString) {
-        InputStream is = null;
-        try {
-            is = InputStreamUtils.getInputStream(urlString, null, compressionType);
-        } catch (Exception ex) {
-            LOG.info("get input stream from url: " + urlString + " failed. ");
-            return false;
-        } finally {
-            if (is != null) {
-                try {
-                    is.close();
-                } catch (IOException e) {
-                    LOG.warn("{}", e);
-                }
-            }
-        }
-        return true;
-    }
-
-    @Override
-    public String getSelectedUrl() {
-        if (selectedUrl == null) {
-            selectedUrl = urls[0];
-        }
-        return selectedUrl;
-    }
-
-    @Override
-    public void reSelectUrl() throws IOException {
-        if (reselectInProgress) {
-            return;
-        }
-        synchronized (this) {
-            if (reselectInProgress) {
-                return;
-            }
-            reselectInProgress = true;
-            try {
-                LOG.info("Going to reselect url");
-                for (int i = 0; i < urls.length; i++) {
-                    String urlToCheck = urls[i];
-                    LOG.info("Going to try url :" + urlToCheck);
-                    for (int time = 0; time < MAX_RETRY_TIME; time++) {
-                        if (checkUrl(buildTestURL(urlToCheck))) {
-                            selectedUrl = urls[i];
-                            LOG.info("Successfully switch to new url : " + selectedUrl);
-                            return;
-                        }
-                        LOG.info("try url " + urlToCheck + "fail for " + (time + 1) + " times, sleep 5 seconds before try again. ");
-                        try {
-                            Thread.sleep(5 * 1000);
-                        } catch (InterruptedException ex) {
-                            LOG.warn("{}", ex);
-                        }
-                    }
-                }
-                throw new IOException("No alive url found: " + StringUtils.join(";", Arrays.asList(this.urls)));
-            } finally {
-                reselectInProgress = false;
-            }
-        }
-    }
-
-    protected abstract String buildTestURL(String urlToCheck);
-}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/b5fd782c/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/ha/HAURLSelector.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/ha/HAURLSelector.java b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/ha/HAURLSelector.java
deleted file mode 100644
index fa9b52b..0000000
--- a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/ha/HAURLSelector.java
+++ /dev/null
@@ -1,28 +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.eagle.jpm.util.resourcefetch.ha;
-
-import java.io.IOException;
-
-public interface HAURLSelector {
-
-    boolean checkUrl(String url);
-
-    void reSelectUrl() throws IOException;
-
-    String getSelectedUrl();
-}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/b5fd782c/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/ha/HAURLSelectorImpl.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/ha/HAURLSelectorImpl.java b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/ha/HAURLSelectorImpl.java
deleted file mode 100644
index a083ef2..0000000
--- a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/ha/HAURLSelectorImpl.java
+++ /dev/null
@@ -1,109 +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.eagle.jpm.util.resourcefetch.ha;
-
-import org.apache.eagle.jpm.util.Constants;
-import org.apache.eagle.jpm.util.resourcefetch.connection.InputStreamUtils;
-import org.apache.eagle.jpm.util.resourcefetch.url.ServiceURLBuilder;
-import org.apache.hadoop.util.StringUtils;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-import java.io.InputStream;
-import java.util.Arrays;
-
-public class HAURLSelectorImpl implements HAURLSelector {
-
-    private final String[] urls;
-    private volatile String selectedUrl;
-    private final ServiceURLBuilder builder;
-
-    private volatile boolean reselectInProgress;
-    private final Constants.CompressionType compressionType;
-    private static final long MAX_RETRY_TIME = 2;
-    private static final Logger LOG = LoggerFactory.getLogger(HAURLSelectorImpl.class);
-
-    public HAURLSelectorImpl(String[] urls, ServiceURLBuilder builder, Constants.CompressionType compressionType) {
-        this.urls = urls;
-        this.compressionType = compressionType;
-        this.builder = builder;
-    }
-
-    public boolean checkUrl(String urlString) {
-        InputStream is = null;
-        try {
-            is = InputStreamUtils.getInputStream(urlString, null, compressionType);
-        } catch (Exception ex) {
-            LOG.info("get inputstream from url: " + urlString + " failed. ");
-            return false;
-        } finally {
-            if (is != null) {
-                try {
-                    is.close();
-                } catch (IOException e) {
-                    LOG.warn("{}", e);
-                }
-            }
-        }
-        return true;
-    }
-
-    @Override
-    public String getSelectedUrl() {
-        if (selectedUrl == null) {
-            selectedUrl = urls[0];
-        }
-        return selectedUrl;
-    }
-
-    @Override
-    public void reSelectUrl() throws IOException {
-        if (reselectInProgress) {
-            return;
-        }
-        synchronized (this) {
-            if (reselectInProgress) {
-                return;
-            }
-            reselectInProgress = true;
-            try {
-                LOG.info("Going to reselect url");
-                for (int i = 0; i < urls.length; i++) {
-                    String urlToCheck = urls[i];
-                    LOG.info("Going to try url :" + urlToCheck);
-                    for (int time = 0; time < MAX_RETRY_TIME; time++) {
-                        if (checkUrl(builder.build(urlToCheck, Constants.JobState.RUNNING.name()))) {
-                            selectedUrl = urls[i];
-                            LOG.info("Successfully switch to new url : " + selectedUrl);
-                            return;
-                        }
-                        LOG.info("try url " + urlToCheck + "fail for " + (time + 1) + " times, sleep 5 seconds before try again. ");
-                        try {
-                            Thread.sleep(1000);
-                        } catch (InterruptedException ex) {
-                            LOG.warn("{}", ex);
-                        }
-                    }
-                }
-                throw new IOException("No alive url found: " + StringUtils.join(";", Arrays.asList(this.urls)));
-            } finally {
-                reselectInProgress = false;
-            }
-        }
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/b5fd782c/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/AppInfo.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/AppInfo.java b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/AppInfo.java
deleted file mode 100644
index f769217..0000000
--- a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/AppInfo.java
+++ /dev/null
@@ -1,210 +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.eagle.jpm.util.resourcefetch.model;
-
-import org.codehaus.jackson.annotate.JsonIgnoreProperties;
-import org.codehaus.jackson.map.annotate.JsonSerialize;
-
-import java.io.Serializable;
-
-@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
-@JsonIgnoreProperties(ignoreUnknown = true)
-public class AppInfo implements Serializable {
-    private static final long serialVersionUID = 1L;
-
-    private String id;
-    private String user;
-    private String name;
-    private String queue;
-    private String state;
-    private String finalStatus;
-    private double progress;
-    private String trackingUI;
-    private String trackingUrl;
-    private String diagnostics;
-    private String clusterId;
-    private String applicationType;
-    private long startedTime;
-    private long finishedTime;
-    private long elapsedTime;
-    private String amContainerLogs;
-    private String amHostHttpAddress;
-    private long allocatedMB;
-    private int allocatedVCores;
-    private int runningContainers;
-
-    public String getId() {
-        return id;
-    }
-
-    public void setId(String id) {
-        this.id = id;
-    }
-
-    public String getUser() {
-        return user;
-    }
-
-    public void setUser(String user) {
-        this.user = user;
-    }
-
-    public String getName() {
-        return name;
-    }
-
-    public void setName(String name) {
-        this.name = name;
-    }
-
-    public String getQueue() {
-        return queue;
-    }
-
-    public void setQueue(String queue) {
-        this.queue = queue;
-    }
-
-    public String getState() {
-        return state;
-    }
-
-    public void setState(String state) {
-        this.state = state;
-    }
-
-    public String getFinalStatus() {
-        return finalStatus;
-    }
-
-    public void setFinalStatus(String finalStatus) {
-        this.finalStatus = finalStatus;
-    }
-
-    public double getProgress() {
-        return progress;
-    }
-
-    public void setProgress(double progress) {
-        this.progress = progress;
-    }
-
-    public String getTrackingUI() {
-        return trackingUI;
-    }
-
-    public void setTrackingUI(String trackingUI) {
-        this.trackingUI = trackingUI;
-    }
-
-    public String getTrackingUrl() {
-        return trackingUrl;
-    }
-
-    public void setTrackingUrl(String trackingUrl) {
-        this.trackingUrl = trackingUrl;
-    }
-
-    public String getDiagnostics() {
-        return diagnostics;
-    }
-
-    public void setDiagnostics(String diagnostics) {
-        this.diagnostics = diagnostics;
-    }
-
-    public String getClusterId() {
-        return clusterId;
-    }
-
-    public void setClusterId(String clusterId) {
-        this.clusterId = clusterId;
-    }
-
-    public String getApplicationType() {
-        return applicationType;
-    }
-
-    public void setApplicationType(String applicationType) {
-        this.applicationType = applicationType;
-    }
-
-    public long getStartedTime() {
-        return startedTime;
-    }
-
-    public void setStartedTime(long startedTime) {
-        this.startedTime = startedTime;
-    }
-
-    public long getFinishedTime() {
-        return finishedTime;
-    }
-
-    public void setFinishedTime(long finishedTime) {
-        this.finishedTime = finishedTime;
-    }
-
-    public long getElapsedTime() {
-        return elapsedTime;
-    }
-
-    public void setElapsedTime(long elapsedTime) {
-        this.elapsedTime = elapsedTime;
-    }
-
-    public String getAmContainerLogs() {
-        return amContainerLogs;
-    }
-
-    public void setAmContainerLogs(String amContainerLogs) {
-        this.amContainerLogs = amContainerLogs;
-    }
-
-    public String getAmHostHttpAddress() {
-        return amHostHttpAddress;
-    }
-
-    public void setAmHostHttpAddress(String amHostHttpAddress) {
-        this.amHostHttpAddress = amHostHttpAddress;
-    }
-
-    public long getAllocatedMB() {
-        return allocatedMB;
-    }
-
-    public void setAllocatedMB(long allocatedMB) {
-        this.allocatedMB = allocatedMB;
-    }
-
-    public int getAllocatedVCores() {
-        return allocatedVCores;
-    }
-
-    public void setAllocatedVCores(int allocatedVCores) {
-        this.allocatedVCores = allocatedVCores;
-    }
-
-    public int getRunningContainers() {
-        return runningContainers;
-    }
-
-    public void setRunningContainers(int runningContainers) {
-        this.runningContainers = runningContainers;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/b5fd782c/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/Applications.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/Applications.java b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/Applications.java
deleted file mode 100644
index 22ee2b9..0000000
--- a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/Applications.java
+++ /dev/null
@@ -1,38 +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.eagle.jpm.util.resourcefetch.model;
-
-import org.codehaus.jackson.annotate.JsonIgnoreProperties;
-import org.codehaus.jackson.map.annotate.JsonSerialize;
-
-import java.util.List;
-
-@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
-@JsonIgnoreProperties(ignoreUnknown = true)
-public class Applications {
-
-    private List<AppInfo> app;
-
-    public List<AppInfo> getApp() {
-        return app;
-    }
-
-    public void setApp(List<AppInfo> app) {
-        this.app = app;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/b5fd782c/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/AppsWrapper.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/AppsWrapper.java b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/AppsWrapper.java
deleted file mode 100644
index 63e37eb..0000000
--- a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/AppsWrapper.java
+++ /dev/null
@@ -1,37 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    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.eagle.jpm.util.resourcefetch.model;
-
-import org.codehaus.jackson.annotate.JsonIgnoreProperties;
-import org.codehaus.jackson.map.annotate.JsonSerialize;
-
-@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
-@JsonIgnoreProperties(ignoreUnknown = true)
-public class AppsWrapper {
-    private static final long serialVersionUID = 1L;
-
-    private Applications apps;
-
-    public Applications getApps() {
-        return apps;
-    }
-
-    public void setApps(Applications apps) {
-        this.apps = apps;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/b5fd782c/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/ClusterInfo.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/ClusterInfo.java b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/ClusterInfo.java
deleted file mode 100644
index 6e16b7f..0000000
--- a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/ClusterInfo.java
+++ /dev/null
@@ -1,119 +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.eagle.jpm.util.resourcefetch.model;
-
-import org.codehaus.jackson.annotate.JsonIgnoreProperties;
-import org.codehaus.jackson.map.annotate.JsonSerialize;
-
-import java.io.Serializable;
-
-@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
-@JsonIgnoreProperties(ignoreUnknown = true)
-public class ClusterInfo implements Serializable {
-    private static final long serialVersionUID = 1L;
-
-    private long id;
-    private long startedOn;
-    private String state;
-    private String haState;
-    private String resourceManagerVersion;
-    private String resourceManagerBuildVersion;
-    private String resourceManagerVersionBuiltOn;
-    private String hadoopVersion;
-    private String hadoopBuildVersion;
-    private String hadoopVersionBuiltOn;
-
-    public long getId() {
-        return id;
-    }
-
-    public void setId(long id) {
-        this.id = id;
-    }
-
-    public long getStartedOn() {
-        return startedOn;
-    }
-
-    public void setStartedOn(long startedOn) {
-        this.startedOn = startedOn;
-    }
-
-    public String getState() {
-        return state;
-    }
-
-    public void setState(String state) {
-        this.state = state;
-    }
-
-    public String getHaState() {
-        return haState;
-    }
-
-    public void setHaState(String haState) {
-        this.haState = haState;
-    }
-
-    public String getResourceManagerVersion() {
-        return resourceManagerVersion;
-    }
-
-    public void setResourceManagerVersion(String resourceManagerVersion) {
-        this.resourceManagerVersion = resourceManagerVersion;
-    }
-
-    public String getResourceManagerBuildVersion() {
-        return resourceManagerBuildVersion;
-    }
-
-    public void setResourceManagerBuildVersion(String resourceManagerBuildVersion) {
-        this.resourceManagerBuildVersion = resourceManagerBuildVersion;
-    }
-
-    public String getResourceManagerVersionBuiltOn() {
-        return resourceManagerVersionBuiltOn;
-    }
-
-    public void setResourceManagerVersionBuiltOn(String resourceManagerVersionBuiltOn) {
-        this.resourceManagerVersionBuiltOn = resourceManagerVersionBuiltOn;
-    }
-
-    public String getHadoopVersion() {
-        return hadoopVersion;
-    }
-
-    public void setHadoopVersion(String hadoopVersion) {
-        this.hadoopVersion = hadoopVersion;
-    }
-
-    public String getHadoopBuildVersion() {
-        return hadoopBuildVersion;
-    }
-
-    public void setHadoopBuildVersion(String hadoopBuildVersion) {
-        this.hadoopBuildVersion = hadoopBuildVersion;
-    }
-
-    public String getHadoopVersionBuiltOn() {
-        return hadoopVersionBuiltOn;
-    }
-
-    public void setHadoopVersionBuiltOn(String hadoopVersionBuiltOn) {
-        this.hadoopVersionBuiltOn = hadoopVersionBuiltOn;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/b5fd782c/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/ClusterInfoWrapper.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/ClusterInfoWrapper.java b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/ClusterInfoWrapper.java
deleted file mode 100644
index 4359e66..0000000
--- a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/ClusterInfoWrapper.java
+++ /dev/null
@@ -1,35 +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.eagle.jpm.util.resourcefetch.model;
-
-import org.codehaus.jackson.annotate.JsonIgnoreProperties;
-import org.codehaus.jackson.map.annotate.JsonSerialize;
-
-@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
-@JsonIgnoreProperties(ignoreUnknown = true)
-public class ClusterInfoWrapper {
-    private ClusterInfo clusterInfo;
-
-    public ClusterInfo getClusterInfo() {
-        return clusterInfo;
-    }
-
-    public void setClusterInfo(ClusterInfo clusterInfo) {
-        this.clusterInfo = clusterInfo;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/b5fd782c/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/JobCounterGroup.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/JobCounterGroup.java b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/JobCounterGroup.java
deleted file mode 100644
index 189767b..0000000
--- a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/JobCounterGroup.java
+++ /dev/null
@@ -1,46 +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.eagle.jpm.util.resourcefetch.model;
-
-import org.codehaus.jackson.annotate.JsonIgnoreProperties;
-import org.codehaus.jackson.map.annotate.JsonSerialize;
-
-import java.util.List;
-
-@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
-@JsonIgnoreProperties(ignoreUnknown = true)
-public class JobCounterGroup {
-    private String counterGroupName;
-    private List<JobCounterItem> counter;
-
-    public String getCounterGroupName() {
-        return counterGroupName;
-    }
-
-    public void setCounterGroupName(String counterGroupName) {
-        this.counterGroupName = counterGroupName;
-    }
-
-    public List<JobCounterItem> getCounter() {
-        return counter;
-    }
-
-    public void setCounter(List<JobCounterItem> counter) {
-        this.counter = counter;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/b5fd782c/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/JobCounterItem.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/JobCounterItem.java b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/JobCounterItem.java
deleted file mode 100644
index 001c5ff..0000000
--- a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/JobCounterItem.java
+++ /dev/null
@@ -1,62 +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.eagle.jpm.util.resourcefetch.model;
-
-import org.codehaus.jackson.annotate.JsonIgnoreProperties;
-import org.codehaus.jackson.map.annotate.JsonSerialize;
-
-@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
-@JsonIgnoreProperties(ignoreUnknown = true)
-public class JobCounterItem {
-    private String name;
-    private long totalCounterValue;
-    private long mapCounterValue;
-    private long reduceCounterValue;
-
-    public String getName() {
-        return name;
-    }
-
-    public void setName(String name) {
-        this.name = name;
-    }
-
-    public long getTotalCounterValue() {
-        return totalCounterValue;
-    }
-
-    public void setTotalCounterValue(long totalCounterValue) {
-        this.totalCounterValue = totalCounterValue;
-    }
-
-    public long getMapCounterValue() {
-        return mapCounterValue;
-    }
-
-    public void setMapCounterValue(long mapCounterValue) {
-        this.mapCounterValue = mapCounterValue;
-    }
-
-    public long getReduceCounterValue() {
-        return reduceCounterValue;
-    }
-
-    public void setReduceCounterValue(long reduceCounterValue) {
-        this.reduceCounterValue = reduceCounterValue;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/b5fd782c/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/JobCounters.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/JobCounters.java b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/JobCounters.java
deleted file mode 100644
index cc0709f..0000000
--- a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/JobCounters.java
+++ /dev/null
@@ -1,46 +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.eagle.jpm.util.resourcefetch.model;
-
-import org.codehaus.jackson.annotate.JsonIgnoreProperties;
-import org.codehaus.jackson.map.annotate.JsonSerialize;
-
-import java.util.List;
-
-@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
-@JsonIgnoreProperties(ignoreUnknown = true)
-public class JobCounters {
-    private String id;
-    private List<JobCounterGroup> counterGroup;
-
-    public String getId() {
-        return id;
-    }
-
-    public void setId(String id) {
-        this.id = id;
-    }
-
-    public List<JobCounterGroup> getCounterGroup() {
-        return counterGroup;
-    }
-
-    public void setCounterGroup(List<JobCounterGroup> counterGroup) {
-        this.counterGroup = counterGroup;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/b5fd782c/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/JobCountersWrapper.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/JobCountersWrapper.java b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/JobCountersWrapper.java
deleted file mode 100644
index f6bb29c..0000000
--- a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/JobCountersWrapper.java
+++ /dev/null
@@ -1,35 +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.eagle.jpm.util.resourcefetch.model;
-
-import org.codehaus.jackson.annotate.JsonIgnoreProperties;
-import org.codehaus.jackson.map.annotate.JsonSerialize;
-
-@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
-@JsonIgnoreProperties(ignoreUnknown = true)
-public class JobCountersWrapper {
-    public JobCounters getJobCounters() {
-        return jobCounters;
-    }
-
-    public void setJobCounters(JobCounters jobCounters) {
-        this.jobCounters = jobCounters;
-    }
-
-    private JobCounters jobCounters;
-}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/b5fd782c/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/MRJob.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/MRJob.java b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/MRJob.java
deleted file mode 100644
index f12a5d7..0000000
--- a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/MRJob.java
+++ /dev/null
@@ -1,289 +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.eagle.jpm.util.resourcefetch.model;
-
-import org.codehaus.jackson.annotate.JsonIgnoreProperties;
-import org.codehaus.jackson.map.annotate.JsonSerialize;
-
-@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
-@JsonIgnoreProperties(ignoreUnknown = true)
-public class MRJob {
-    private long startTime;
-    private long finishTime;
-    private long elapsedTime;
-    private String id;
-    private String name;
-    private String user;
-    private String state;
-    private int mapsTotal;
-    private int mapsCompleted;
-    private int reducesTotal;
-    private int reducesCompleted;
-    private double mapProgress;
-    private double reduceProgress;
-    private int mapsPending;
-    private int mapsRunning;
-    private int reducesPending;
-    private int reducesRunning;
-    private boolean uberized;
-    private String diagnostics;
-    private int newReduceAttempts;
-    private int runningReduceAttempts;
-    private int failedReduceAttempts;
-    private int killedReduceAttempts;
-    private int successfulReduceAttempts;
-    private int newMapAttempts;
-    private int runningMapAttempts;
-    private int failedMapAttempts;
-    private int killedMapAttempts;
-    private int successfulMapAttempts;
-
-    public long getStartTime() {
-        return startTime;
-    }
-
-    public void setStartTime(long startTime) {
-        this.startTime = startTime;
-    }
-
-    public long getFinishTime() {
-        return finishTime;
-    }
-
-    public void setFinishTime(long finishTime) {
-        this.finishTime = finishTime;
-    }
-
-    public long getElapsedTime() {
-        return elapsedTime;
-    }
-
-    public void setElapsedTime(long elapsedTime) {
-        this.elapsedTime = elapsedTime;
-    }
-
-    public String getId() {
-        return id;
-    }
-
-    public void setId(String id) {
-        this.id = id;
-    }
-
-    public String getName() {
-        return name;
-    }
-
-    public void setName(String name) {
-        this.name = name;
-    }
-
-    public String getUser() {
-        return user;
-    }
-
-    public void setUser(String user) {
-        this.user = user;
-    }
-
-    public String getState() {
-        return state;
-    }
-
-    public void setState(String state) {
-        this.state = state;
-    }
-
-    public int getMapsTotal() {
-        return mapsTotal;
-    }
-
-    public void setMapsTotal(int mapsTotal) {
-        this.mapsTotal = mapsTotal;
-    }
-
-    public int getMapsCompleted() {
-        return mapsCompleted;
-    }
-
-    public void setMapsCompleted(int mapsCompleted) {
-        this.mapsCompleted = mapsCompleted;
-    }
-
-    public int getReducesTotal() {
-        return reducesTotal;
-    }
-
-    public void setReducesTotal(int reducesTotal) {
-        this.reducesTotal = reducesTotal;
-    }
-
-    public int getReducesCompleted() {
-        return reducesCompleted;
-    }
-
-    public void setReducesCompleted(int reducesCompleted) {
-        this.reducesCompleted = reducesCompleted;
-    }
-
-    public double getMapProgress() {
-        return mapProgress;
-    }
-
-    public void setMapProgress(double mapProgress) {
-        this.mapProgress = mapProgress;
-    }
-
-    public double getReduceProgress() {
-        return reduceProgress;
-    }
-
-    public void setReduceProgress(double reduceProgress) {
-        this.reduceProgress = reduceProgress;
-    }
-
-    public int getMapsPending() {
-        return mapsPending;
-    }
-
-    public void setMapsPending(int mapsPending) {
-        this.mapsPending = mapsPending;
-    }
-
-    public int getMapsRunning() {
-        return mapsRunning;
-    }
-
-    public void setMapsRunning(int mapsRunning) {
-        this.mapsRunning = mapsRunning;
-    }
-
-    public int getReducesPending() {
-        return reducesPending;
-    }
-
-    public void setReducesPending(int reducesPending) {
-        this.reducesPending = reducesPending;
-    }
-
-    public int getReducesRunning() {
-        return reducesRunning;
-    }
-
-    public void setReducesRunning(int reducesRunning) {
-        this.reducesRunning = reducesRunning;
-    }
-
-    public boolean isUberized() {
-        return uberized;
-    }
-
-    public void setUberized(boolean uberized) {
-        this.uberized = uberized;
-    }
-
-    public String getDiagnostics() {
-        return diagnostics;
-    }
-
-    public void setDiagnostics(String diagnostics) {
-        this.diagnostics = diagnostics;
-    }
-
-    public int getNewReduceAttempts() {
-        return newReduceAttempts;
-    }
-
-    public void setNewReduceAttempts(int newReduceAttempts) {
-        this.newReduceAttempts = newReduceAttempts;
-    }
-
-    public int getRunningReduceAttempts() {
-        return runningReduceAttempts;
-    }
-
-    public void setRunningReduceAttempts(int runningReduceAttempts) {
-        this.runningReduceAttempts = runningReduceAttempts;
-    }
-
-    public int getFailedReduceAttempts() {
-        return failedReduceAttempts;
-    }
-
-    public void setFailedReduceAttempts(int failedReduceAttempts) {
-        this.failedReduceAttempts = failedReduceAttempts;
-    }
-
-    public int getKilledReduceAttempts() {
-        return killedReduceAttempts;
-    }
-
-    public void setKilledReduceAttempts(int killedReduceAttempts) {
-        this.killedReduceAttempts = killedReduceAttempts;
-    }
-
-    public int getSuccessfulReduceAttempts() {
-        return successfulReduceAttempts;
-    }
-
-    public void setSuccessfulReduceAttempts(int successfulReduceAttempts) {
-        this.successfulReduceAttempts = successfulReduceAttempts;
-    }
-
-    public int getNewMapAttempts() {
-        return newMapAttempts;
-    }
-
-    public void setNewMapAttempts(int newMapAttempts) {
-        this.newMapAttempts = newMapAttempts;
-    }
-
-    public int getRunningMapAttempts() {
-        return runningMapAttempts;
-    }
-
-    public void setRunningMapAttempts(int runningMapAttempts) {
-        this.runningMapAttempts = runningMapAttempts;
-    }
-
-    public int getFailedMapAttempts() {
-        return failedMapAttempts;
-    }
-
-    public void setFailedMapAttempts(int failedMapAttempts) {
-        this.failedMapAttempts = failedMapAttempts;
-    }
-
-    public int getKilledMapAttempts() {
-        return killedMapAttempts;
-    }
-
-    public void setKilledMapAttempts(int killedMapAttempts) {
-        this.killedMapAttempts = killedMapAttempts;
-    }
-
-    public int getSuccessfulMapAttempts() {
-        return successfulMapAttempts;
-    }
-
-    public void setSuccessfulMapAttempts(int successfulMapAttempts) {
-        this.successfulMapAttempts = successfulMapAttempts;
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/b5fd782c/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/MRJobsWrapper.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/MRJobsWrapper.java b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/MRJobsWrapper.java
deleted file mode 100644
index acd316f..0000000
--- a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/MRJobsWrapper.java
+++ /dev/null
@@ -1,37 +0,0 @@
-/*
- *
- *  * Licensed to the Apache Software Foundation (ASF) under one or more
- *  * contributor license agreements.  See the NOTICE file distributed with
- *  * this work for additional information regarding copyright ownership.
- *  * The ASF licenses this file to You under the Apache License, Version 2.0
- *  * (the "License"); you may not use this file except in compliance with
- *  * the License.  You may obtain a copy of the License at
- *  *
- *  *    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.eagle.jpm.util.resourcefetch.model;
-
-import org.codehaus.jackson.annotate.JsonIgnoreProperties;
-import org.codehaus.jackson.map.annotate.JsonSerialize;
-
-@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
-@JsonIgnoreProperties(ignoreUnknown = true)
-public class MRJobsWrapper {
-    public MrJobs getJobs() {
-        return jobs;
-    }
-
-    public void setJobs(MrJobs jobs) {
-        this.jobs = jobs;
-    }
-
-    private MrJobs jobs;
-}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/b5fd782c/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/MRTask.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/MRTask.java b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/MRTask.java
deleted file mode 100644
index 4a549de..0000000
--- a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/MRTask.java
+++ /dev/null
@@ -1,109 +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.eagle.jpm.util.resourcefetch.model;
-
-import org.codehaus.jackson.annotate.JsonIgnoreProperties;
-import org.codehaus.jackson.map.annotate.JsonSerialize;
-
-@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
-@JsonIgnoreProperties(ignoreUnknown = true)
-public class MRTask {
-    private long startTime;
-    private long finishTime;
-    private long elapsedTime;
-    private double progress;
-    private String id;
-    private String state;
-    private String type;
-    private String successfulAttempt;
-    private String status;
-
-    public long getStartTime() {
-        return startTime;
-    }
-
-    public void setStartTime(long startTime) {
-        this.startTime = startTime;
-    }
-
-    public long getFinishTime() {
-        return finishTime;
-    }
-
-    public void setFinishTime(long finishTime) {
-        this.finishTime = finishTime;
-    }
-
-    public long getElapsedTime() {
-        return elapsedTime;
-    }
-
-    public void setElapsedTime(long elapsedTime) {
-        this.elapsedTime = elapsedTime;
-    }
-
-    public double getProgress() {
-        return progress;
-    }
-
-    public void setProgress(double progress) {
-        this.progress = progress;
-    }
-
-    public String getId() {
-        return id;
-    }
-
-    public void setId(String id) {
-        this.id = id;
-    }
-
-    public String getState() {
-        return state;
-    }
-
-    public void setState(String state) {
-        this.state = state;
-    }
-
-    public String getType() {
-        return type;
-    }
-
-    public void setType(String type) {
-        this.type = type;
-    }
-
-    public String getSuccessfulAttempt() {
-        return successfulAttempt;
-    }
-
-    public void setSuccessfulAttempt(String successfulAttempt) {
-        this.successfulAttempt = successfulAttempt;
-    }
-
-    public String getStatus() {
-        return status;
-    }
-
-    public void setStatus(String status) {
-        this.status = status;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/b5fd782c/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/MRTaskAttempt.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/MRTaskAttempt.java b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/MRTaskAttempt.java
deleted file mode 100644
index 218d021..0000000
--- a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/MRTaskAttempt.java
+++ /dev/null
@@ -1,136 +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.eagle.jpm.util.resourcefetch.model;
-
-import org.codehaus.jackson.annotate.JsonIgnoreProperties;
-import org.codehaus.jackson.map.annotate.JsonSerialize;
-
-@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
-@JsonIgnoreProperties(ignoreUnknown = true)
-public class MRTaskAttempt {
-    private long startTime;
-    private long finishTime;
-    private long elapsedTime;
-    private double progress;
-    private String id;
-    private String rack;
-    private String state;
-    private String status;
-    private String nodeHttpAddress;
-    private String diagnostics;
-    private String type;
-    private String assignedContainerId;
-
-    public long getStartTime() {
-        return startTime;
-    }
-
-    public void setStartTime(long startTime) {
-        this.startTime = startTime;
-    }
-
-    public long getFinishTime() {
-        return finishTime;
-    }
-
-    public void setFinishTime(long finishTime) {
-        this.finishTime = finishTime;
-    }
-
-    public long getElapsedTime() {
-        return elapsedTime;
-    }
-
-    public void setElapsedTime(long elapsedTime) {
-        this.elapsedTime = elapsedTime;
-    }
-
-    public double getProgress() {
-        return progress;
-    }
-
-    public void setProgress(double progress) {
-        this.progress = progress;
-    }
-
-    public String getId() {
-        return id;
-    }
-
-    public void setId(String id) {
-        this.id = id;
-    }
-
-    public String getRack() {
-        return rack;
-    }
-
-    public void setRack(String rack) {
-        this.rack = rack;
-    }
-
-    public String getState() {
-        return state;
-    }
-
-    public void setState(String state) {
-        this.state = state;
-    }
-
-    public String getStatus() {
-        return status;
-    }
-
-    public void setStatus(String status) {
-        this.status = status;
-    }
-
-    public String getNodeHttpAddress() {
-        return nodeHttpAddress;
-    }
-
-    public void setNodeHttpAddress(String nodeHttpAddress) {
-        this.nodeHttpAddress = nodeHttpAddress;
-    }
-
-    public String getDiagnostics() {
-        return diagnostics;
-    }
-
-    public void setDiagnostics(String diagnostics) {
-        this.diagnostics = diagnostics;
-    }
-
-    public String getType() {
-        return type;
-    }
-
-    public void setType(String type) {
-        this.type = type;
-    }
-
-    public String getAssignedContainerId() {
-        return assignedContainerId;
-    }
-
-    public void setAssignedContainerId(String assignedContainerId) {
-        this.assignedContainerId = assignedContainerId;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/b5fd782c/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/MRTaskAttemptWrapper.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/MRTaskAttemptWrapper.java b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/MRTaskAttemptWrapper.java
deleted file mode 100644
index 005b73a..0000000
--- a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/MRTaskAttemptWrapper.java
+++ /dev/null
@@ -1,37 +0,0 @@
-/*
- *
- *  * Licensed to the Apache Software Foundation (ASF) under one or more
- *  * contributor license agreements.  See the NOTICE file distributed with
- *  * this work for additional information regarding copyright ownership.
- *  * The ASF licenses this file to You under the Apache License, Version 2.0
- *  * (the "License"); you may not use this file except in compliance with
- *  * the License.  You may obtain a copy of the License at
- *  *
- *  *    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.eagle.jpm.util.resourcefetch.model;
-
-import org.codehaus.jackson.annotate.JsonIgnoreProperties;
-import org.codehaus.jackson.map.annotate.JsonSerialize;
-
-@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
-@JsonIgnoreProperties(ignoreUnknown = true)
-public class MRTaskAttemptWrapper {
-    public MRTaskAttempts getTaskAttempts() {
-        return taskAttempts;
-    }
-
-    public void setTaskAttempts(MRTaskAttempts taskAttempts) {
-        this.taskAttempts = taskAttempts;
-    }
-
-    private MRTaskAttempts taskAttempts;
-}



[35/52] [abbrv] incubator-eagle git commit: Update eagle-alert pom.xml for source jar release to maven repository

Posted by yo...@apache.org.
Update eagle-alert pom.xml for source jar release to maven repository

Author: ralphsu


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

Branch: refs/heads/master
Commit: 4a15e18f9c90f50fabdbd78f8d79264824962176
Parents: ebbaad0
Author: Ralph, Su <su...@gmail.com>
Authored: Sun Sep 4 18:39:08 2016 +0800
Committer: Ralph, Su <su...@gmail.com>
Committed: Sun Sep 4 18:39:08 2016 +0800

----------------------------------------------------------------------
 .../eagle-alert/alert-common/pom.xml            | 14 ++++++++
 .../engine/coordinator/PolicyDefinition.java    | 13 +++++--
 .../eagle-alert/alert-coordinator/pom.xml       | 14 ++++++++
 .../eagle-alert/alert-engine/pom.xml            | 14 ++++++++
 .../publisher/impl/JsonEventSerializer.java     | 12 +++----
 .../alert-metadata-service/pom.xml              | 14 ++++++++
 .../alert-metadata/pom.xml                      | 19 ++++++++++
 .../eagle-alert/alert-service/pom.xml           | 37 ++++++++++++++++++++
 .../eagle-alert-parent/eagle-alert/pom.xml      | 14 ++++++++
 9 files changed, 142 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/4a15e18f/eagle-core/eagle-alert-parent/eagle-alert/alert-common/pom.xml
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-alert-parent/eagle-alert/alert-common/pom.xml b/eagle-core/eagle-alert-parent/eagle-alert/alert-common/pom.xml
index ae30921..b56e27f 100644
--- a/eagle-core/eagle-alert-parent/eagle-alert/alert-common/pom.xml
+++ b/eagle-core/eagle-alert-parent/eagle-alert/alert-common/pom.xml
@@ -132,6 +132,20 @@
                     </execution>
                 </executions>
             </plugin>
+            <plugin>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-source-plugin</artifactId>
+                <version>2.1.2</version>
+                <executions>
+                    <execution>
+                        <id>attach-sources</id>
+                        <phase>verify</phase>
+                        <goals>
+                            <goal>jar-no-fork</goal>
+                        </goals>
+                    </execution>
+                </executions>
+            </plugin>
         </plugins>
     </build>
 </project>

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/4a15e18f/eagle-core/eagle-alert-parent/eagle-alert/alert-common/src/main/java/org/apache/eagle/alert/engine/coordinator/PolicyDefinition.java
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-alert-parent/eagle-alert/alert-common/src/main/java/org/apache/eagle/alert/engine/coordinator/PolicyDefinition.java b/eagle-core/eagle-alert-parent/eagle-alert/alert-common/src/main/java/org/apache/eagle/alert/engine/coordinator/PolicyDefinition.java
index bc389a2..b154514 100644
--- a/eagle-core/eagle-alert-parent/eagle-alert/alert-common/src/main/java/org/apache/eagle/alert/engine/coordinator/PolicyDefinition.java
+++ b/eagle-core/eagle-alert-parent/eagle-alert/alert-common/src/main/java/org/apache/eagle/alert/engine/coordinator/PolicyDefinition.java
@@ -22,9 +22,7 @@ import org.apache.commons.lang3.builder.HashCodeBuilder;
 import org.codehaus.jackson.annotate.JsonIgnoreProperties;
 
 import java.io.Serializable;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Objects;
+import java.util.*;
 
 /**
  * @since Apr 5, 2016
@@ -165,6 +163,7 @@ public class PolicyDefinition implements Serializable{
         public String type;
         public String value;
         public String handlerClass;
+        public Map<String, Object> properties = new HashMap<>();
 
         private List<String> inputStreams = new ArrayList<String>();
         private List<String> outputStreams = new ArrayList<String>();
@@ -239,6 +238,14 @@ public class PolicyDefinition implements Serializable{
             this.handlerClass = handlerClass;
         }
 
+        public Map<String, Object> getProperties() {
+            return properties;
+        }
+
+        public void setProperties(Map<String, Object> properties) {
+            this.properties = properties;
+        }
+
         @Override
         public String toString() {
             return String.format("{type=\"%s\",value=\"%s\", inputStreams=\"%s\", outputStreams=\"%s\" }",type,value, inputStreams, outputStreams);

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/4a15e18f/eagle-core/eagle-alert-parent/eagle-alert/alert-coordinator/pom.xml
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-alert-parent/eagle-alert/alert-coordinator/pom.xml b/eagle-core/eagle-alert-parent/eagle-alert/alert-coordinator/pom.xml
index 19352ba..38abebd 100644
--- a/eagle-core/eagle-alert-parent/eagle-alert/alert-coordinator/pom.xml
+++ b/eagle-core/eagle-alert-parent/eagle-alert/alert-coordinator/pom.xml
@@ -103,6 +103,20 @@
                     </execution>
                 </executions>
             </plugin>
+            <plugin>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-source-plugin</artifactId>
+                <version>2.1.2</version>
+                <executions>
+                    <execution>
+                        <id>attach-sources</id>
+                        <phase>verify</phase>
+                        <goals>
+                            <goal>jar-no-fork</goal>
+                        </goals>
+                    </execution>
+                </executions>
+            </plugin>
         </plugins>
     </build>
 </project>

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/4a15e18f/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/pom.xml
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/pom.xml b/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/pom.xml
index 64d7493..e6244ab 100644
--- a/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/pom.xml
+++ b/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/pom.xml
@@ -149,6 +149,20 @@
                     </execution>
                 </executions>
             </plugin>
+            <plugin>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-source-plugin</artifactId>
+                <version>2.1.2</version>
+                <executions>
+                    <execution>
+                        <id>attach-sources</id>
+                        <phase>verify</phase>
+                        <goals>
+                            <goal>jar-no-fork</goal>
+                        </goals>
+                    </execution>
+                </executions>
+            </plugin>
         </plugins>
     </build>
 </project>

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/4a15e18f/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/main/java/org/apache/eagle/alert/engine/publisher/impl/JsonEventSerializer.java
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/main/java/org/apache/eagle/alert/engine/publisher/impl/JsonEventSerializer.java b/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/main/java/org/apache/eagle/alert/engine/publisher/impl/JsonEventSerializer.java
index f30bf8f..a448703 100644
--- a/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/main/java/org/apache/eagle/alert/engine/publisher/impl/JsonEventSerializer.java
+++ b/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/main/java/org/apache/eagle/alert/engine/publisher/impl/JsonEventSerializer.java
@@ -16,10 +16,6 @@
  */
 package org.apache.eagle.alert.engine.publisher.impl;
 
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
 import org.apache.eagle.alert.engine.codec.IEventSerializer;
 import org.apache.eagle.alert.engine.coordinator.StreamColumn;
 import org.apache.eagle.alert.engine.model.AlertStreamEvent;
@@ -27,6 +23,10 @@ import org.apache.eagle.alert.utils.JsonUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
 /**
  * @since Jul 9, 2016
  *
@@ -59,8 +59,8 @@ public class JsonEventSerializer implements IEventSerializer {
         List<StreamColumn> columns = event.getSchema().getColumns();
         for (int i = 0; i < size; i++) {
             if (columns.size() < i) {
-                // redudant check to log inconsistency
-                LOG.error(" strema event data have different lenght compare to column definition! ");
+                // redundant check to log inconsistency
+                LOG.error(" stream event data have different lenght compare to column definition! ");
             } else {
                 jsonMap.put(columns.get(i).getName(), event.getData()[i]);
             }

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/4a15e18f/eagle-core/eagle-alert-parent/eagle-alert/alert-metadata-parent/alert-metadata-service/pom.xml
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-alert-parent/eagle-alert/alert-metadata-parent/alert-metadata-service/pom.xml b/eagle-core/eagle-alert-parent/eagle-alert/alert-metadata-parent/alert-metadata-service/pom.xml
index 2f7e4a7..9467f1b 100644
--- a/eagle-core/eagle-alert-parent/eagle-alert/alert-metadata-parent/alert-metadata-service/pom.xml
+++ b/eagle-core/eagle-alert-parent/eagle-alert/alert-metadata-parent/alert-metadata-service/pom.xml
@@ -108,6 +108,20 @@
                     <scanIntervalSeconds>5</scanIntervalSeconds>
                 </configuration>
             </plugin>
+            <plugin>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-source-plugin</artifactId>
+                <version>2.1.2</version>
+                <executions>
+                    <execution>
+                        <id>attach-sources</id>
+                        <phase>verify</phase>
+                        <goals>
+                            <goal>jar-no-fork</goal>
+                        </goals>
+                    </execution>
+                </executions>
+            </plugin>
         </plugins>
     </build>
 </project>

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/4a15e18f/eagle-core/eagle-alert-parent/eagle-alert/alert-metadata-parent/alert-metadata/pom.xml
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-alert-parent/eagle-alert/alert-metadata-parent/alert-metadata/pom.xml b/eagle-core/eagle-alert-parent/eagle-alert/alert-metadata-parent/alert-metadata/pom.xml
index 7203e7d..877a6c7 100644
--- a/eagle-core/eagle-alert-parent/eagle-alert/alert-metadata-parent/alert-metadata/pom.xml
+++ b/eagle-core/eagle-alert-parent/eagle-alert/alert-metadata-parent/alert-metadata/pom.xml
@@ -64,4 +64,23 @@
             <version>3.0</version>
         </dependency>
     </dependencies>
+
+    <build>
+        <plugins>
+            <plugin>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-source-plugin</artifactId>
+                <version>2.1.2</version>
+                <executions>
+                    <execution>
+                        <id>attach-sources</id>
+                        <phase>verify</phase>
+                        <goals>
+                            <goal>jar-no-fork</goal>
+                        </goals>
+                    </execution>
+                </executions>
+            </plugin>
+        </plugins>
+    </build>
 </project>

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/4a15e18f/eagle-core/eagle-alert-parent/eagle-alert/alert-service/pom.xml
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-alert-parent/eagle-alert/alert-service/pom.xml b/eagle-core/eagle-alert-parent/eagle-alert/alert-service/pom.xml
index 1cab431..18c054e 100644
--- a/eagle-core/eagle-alert-parent/eagle-alert/alert-service/pom.xml
+++ b/eagle-core/eagle-alert-parent/eagle-alert/alert-service/pom.xml
@@ -164,6 +164,43 @@
 				</executions>
 			</plugin>
 
+			<plugin>
+				<groupId>org.codehaus.mojo</groupId>
+				<artifactId>build-helper-maven-plugin</artifactId>
+				<version>1.12</version>
+				<executions>
+					<execution>
+						<id>attach-artifacts</id>
+						<phase>package</phase>
+						<goals>
+							<goal>attach-artifact</goal>
+						</goals>
+						<configuration>
+							<artifacts>
+								<artifact>
+									<file>target/alert-service-${project.version}.war</file>
+									<type>war</type>
+								</artifact>
+							</artifacts>
+						</configuration>
+					</execution>
+				</executions>
+			</plugin>
+
+			<plugin>
+				<groupId>org.apache.maven.plugins</groupId>
+				<artifactId>maven-source-plugin</artifactId>
+				<version>2.1.2</version>
+				<executions>
+					<execution>
+						<id>attach-sources</id>
+						<phase>verify</phase>
+						<goals>
+							<goal>jar-no-fork</goal>
+						</goals>
+					</execution>
+				</executions>
+			</plugin>
 		</plugins>
 	</build>
 </project>

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/4a15e18f/eagle-core/eagle-alert-parent/eagle-alert/pom.xml
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-alert-parent/eagle-alert/pom.xml b/eagle-core/eagle-alert-parent/eagle-alert/pom.xml
index af59563..c5ed464 100644
--- a/eagle-core/eagle-alert-parent/eagle-alert/pom.xml
+++ b/eagle-core/eagle-alert-parent/eagle-alert/pom.xml
@@ -352,6 +352,20 @@
                     <artifactId>jetty-maven-plugin</artifactId>
                     <version>${jetty.version}</version>
                 </plugin>
+                <plugin>
+                    <groupId>org.apache.maven.plugins</groupId>
+                    <artifactId>maven-source-plugin</artifactId>
+                    <version>2.1.2</version>
+                    <executions>
+                        <execution>
+                            <id>attach-sources</id>
+                            <phase>verify</phase>
+                            <goals>
+                                <goal>jar-no-fork</goal>
+                            </goals>
+                        </execution>
+                    </executions>
+                </plugin>
             </plugins>
         </pluginManagement>
     </build>


[41/52] [abbrv] incubator-eagle git commit: add jobTypes list in response

Posted by yo...@apache.org.
add jobTypes list in response

Author: Qingwen Zhao <qi...@gmail.com>

Closes #421 from qingwen220/jobAPIUpdate.


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

Branch: refs/heads/master
Commit: fce6ae3f0c33ded811e78ca4bf8ff4c0a7ff3935
Parents: 3110c72
Author: Qingwen Zhao <qi...@gmail.com>
Authored: Wed Sep 7 11:07:17 2016 +0800
Committer: Qingwen Zhao <qi...@gmail.com>
Committed: Wed Sep 7 11:07:17 2016 +0800

----------------------------------------------------------------------
 .../eagle/service/jpm/MRJobCountHelper.java     | 31 ++++++++++++++++----
 .../service/jpm/MRJobExecutionResource.java     | 16 +++++++---
 .../service/jpm/MRJobTaskCountResponse.java     |  1 +
 .../jpm/TestJobCountPerBucketHelper.java        |  7 +++--
 4 files changed, 43 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/fce6ae3f/eagle-jpm/eagle-jpm-service/src/main/java/org/apache/eagle/service/jpm/MRJobCountHelper.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-service/src/main/java/org/apache/eagle/service/jpm/MRJobCountHelper.java b/eagle-jpm/eagle-jpm-service/src/main/java/org/apache/eagle/service/jpm/MRJobCountHelper.java
index 93c6c00..2fa5c04 100644
--- a/eagle-jpm/eagle-jpm-service/src/main/java/org/apache/eagle/service/jpm/MRJobCountHelper.java
+++ b/eagle-jpm/eagle-jpm-service/src/main/java/org/apache/eagle/service/jpm/MRJobCountHelper.java
@@ -20,45 +20,61 @@ package org.apache.eagle.service.jpm;
 
 import org.apache.eagle.common.DateTimeUtil;
 import org.apache.eagle.jpm.mr.historyentity.JobExecutionAPIEntity;
+import org.apache.eagle.jpm.util.Constants;
 import org.apache.eagle.jpm.util.MRJobTagName;
 import org.apache.eagle.service.jpm.MRJobTaskCountResponse.JobCountResponse;
 import org.apache.eagle.service.jpm.MRJobTaskCountResponse.UnitJobCount;
 
 import java.text.ParseException;
 import java.util.ArrayList;
+import java.util.HashSet;
 import java.util.List;
+import java.util.Set;
 
 public class MRJobCountHelper {
 
     public void initJobCountList(List<UnitJobCount> jobCounts, long startTime, long endTime, long intervalInSecs) {
         for (long i = startTime / intervalInSecs; i * intervalInSecs <= endTime; i++) {
-            jobCounts.add(new UnitJobCount(i * intervalInSecs));
+            jobCounts.add(new UnitJobCount(i * intervalInSecs * DateTimeUtil.ONESECOND));
         }
     }
 
-    public String moveTimeforwardOneDay(String startTime) throws ParseException {
+    public String moveTimeForwardOneDay(String startTime) throws ParseException {
         long timeInSecs = DateTimeUtil.humanDateToSeconds(startTime);
         timeInSecs -= 24L * 60L * 60L;
         return DateTimeUtil.secondsToHumanDate(timeInSecs);
     }
 
     public JobCountResponse getRunningJobCount(List<JobExecutionAPIEntity> jobDurations,
+                                               List<org.apache.eagle.jpm.mr.runningentity.JobExecutionAPIEntity> runningJobs,
                                         long startTimeInSecs,
                                         long endTimeInSecs,
                                         long intervalInSecs) {
-        JobCountResponse response = new JobCountResponse();
         List<UnitJobCount> jobCounts = new ArrayList<>();
+        Set<String> jobTypes = new HashSet<>();
         initJobCountList(jobCounts, startTimeInSecs, endTimeInSecs, intervalInSecs);
         for (JobExecutionAPIEntity jobDuration: jobDurations) {
-            countJob(jobCounts, jobDuration.getStartTime() / 1000, jobDuration.getEndTime() / 1000, intervalInSecs, jobDuration.getTags().get(MRJobTagName.JOB_TYPE.toString()));
+            String jobType = jobDuration.getTags().get(MRJobTagName.JOB_TYPE.toString());
+            jobTypes.add(jobType);
+            countJob(jobCounts, jobDuration.getStartTime() / 1000, jobDuration.getEndTime() / 1000, intervalInSecs, jobType);
+        }
+        for (org.apache.eagle.jpm.mr.runningentity.JobExecutionAPIEntity job : runningJobs) {
+            if (job.getInternalState() != null && !job.getInternalState().equalsIgnoreCase(Constants.JobState.FINISHED.toString())) {
+                String jobType = job.getTags().get(MRJobTagName.JOB_TYPE.toString());
+                jobTypes.add(jobType);
+                countJob(jobCounts, job.getStartTime() / 1000, endTimeInSecs, intervalInSecs, jobType);
+            }
         }
+        JobCountResponse response = new JobCountResponse();
         response.jobCounts = jobCounts;
+        response.jobTypes = jobTypes;
         return response;
     }
 
     public JobCountResponse getHistoryJobCount(List<JobExecutionAPIEntity> jobDurations, String timeList) {
         JobCountResponse response = new JobCountResponse();
         List<UnitJobCount> jobCounts = new ArrayList<>();
+        Set<String> jobTypes = new HashSet<>();
         List<Long> times = TaskCountByDurationHelper.parseTimeList(timeList);
         for (int i = 0; i < times.size(); i++) {
             jobCounts.add(new UnitJobCount(times.get(i)));
@@ -66,9 +82,12 @@ public class MRJobCountHelper {
         for (JobExecutionAPIEntity job : jobDurations) {
             int jobIndex = TaskCountByDurationHelper.getPosition(times, job.getDurationTime());
             UnitJobCount counter = jobCounts.get(jobIndex);
-            countJob(counter, job.getTags().get(MRJobTagName.JOB_TYPE.toString()));
+            String jobType = job.getTags().get(MRJobTagName.JOB_TYPE.toString());
+            jobTypes.add(jobType);
+            countJob(counter, jobType);
         }
         response.jobCounts = jobCounts;
+        response.jobTypes = jobTypes;
         return response;
     }
 
@@ -85,7 +104,7 @@ public class MRJobCountHelper {
     }
 
     public void countJob(List<UnitJobCount> jobCounts, long jobStartTimeSecs, long jobEndTimeSecs, long intervalInSecs, String jobType) {
-        long startCountPoint = jobCounts.get(0).timeBucket;
+        long startCountPoint = jobCounts.get(0).timeBucket / DateTimeUtil.ONESECOND;
         if (jobEndTimeSecs < startCountPoint) {
             return;
         }

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/fce6ae3f/eagle-jpm/eagle-jpm-service/src/main/java/org/apache/eagle/service/jpm/MRJobExecutionResource.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-service/src/main/java/org/apache/eagle/service/jpm/MRJobExecutionResource.java b/eagle-jpm/eagle-jpm-service/src/main/java/org/apache/eagle/service/jpm/MRJobExecutionResource.java
index 5af9811..e6041f2 100644
--- a/eagle-jpm/eagle-jpm-service/src/main/java/org/apache/eagle/service/jpm/MRJobExecutionResource.java
+++ b/eagle-jpm/eagle-jpm-service/src/main/java/org/apache/eagle/service/jpm/MRJobExecutionResource.java
@@ -276,7 +276,7 @@ public class MRJobExecutionResource {
         String searchEndTime = endTime;
         try {
             startTimeInMills = DateTimeUtil.humanDateToSeconds(startTime) * DateTimeUtil.ONESECOND;
-            searchStartTime = helper.moveTimeforwardOneDay(searchStartTime);
+            searchStartTime = helper.moveTimeForwardOneDay(searchStartTime);
         } catch (Exception e) {
             response.errMessage = e.getMessage();
             return response;
@@ -285,15 +285,23 @@ public class MRJobExecutionResource {
         GenericServiceAPIResponseEntity<JobExecutionAPIEntity> historyRes =
             resource.search(query, searchStartTime, searchEndTime, Integer.MAX_VALUE, null, false, true, 0L, 0, true, 0, null, false);
         if (!historyRes.isSuccess() || historyRes.getObj() == null) {
-            response.errMessage = String.format("Catch an exception: %s with query=%s", historyRes.getException(), query);
+            response.errMessage = String.format("Catch an exception during fetch history jobs: %s with query=%s", historyRes.getException(), query);
+            return response;
+        }
+        query = String.format("%s[@site=\"%s\"]{@startTime,@endTime,@jobType}", Constants.JPA_RUNNING_JOB_EXECUTION_SERVICE_NAME, site);
+        GenericServiceAPIResponseEntity<org.apache.eagle.jpm.mr.runningentity.JobExecutionAPIEntity> runningRes =
+            resource.search(query, searchStartTime, searchEndTime, Integer.MAX_VALUE, null, false, true, 0L, 0, true, 0, null, false);
+        if (!runningRes.isSuccess() || runningRes.getObj() == null) {
+            response.errMessage = String.format("Catch an exception during fetch running jobs: %s with query=%s", runningRes.getException(), query);
             return response;
         }
 
         try {
             long startTimeInSecs = DateTimeUtil.humanDateToSeconds(startTime);
             long endTimeInSecs = DateTimeUtil.humanDateToSeconds(endTime);
-            return helper.getRunningJobCount(historyRes.getObj(), startTimeInSecs, endTimeInSecs, intervalInSecs);
+            return helper.getRunningJobCount(historyRes.getObj(), runningRes.getObj(), startTimeInSecs, endTimeInSecs, intervalInSecs);
         } catch (Exception e) {
+            e.printStackTrace();
             response.errMessage = e.getMessage();
             return response;
         }
@@ -344,7 +352,7 @@ public class MRJobExecutionResource {
         String searchEndTime = timePoint;
         try {
             timePointsInMills = DateTimeUtil.humanDateToSeconds(timePoint) * DateTimeUtil.ONESECOND;
-            searchStartTime = helper.moveTimeforwardOneDay(searchStartTime);
+            searchStartTime = helper.moveTimeForwardOneDay(searchStartTime);
         } catch (ParseException e) {
             response.setException(e);
             response.setSuccess(false);

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/fce6ae3f/eagle-jpm/eagle-jpm-service/src/main/java/org/apache/eagle/service/jpm/MRJobTaskCountResponse.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-service/src/main/java/org/apache/eagle/service/jpm/MRJobTaskCountResponse.java b/eagle-jpm/eagle-jpm-service/src/main/java/org/apache/eagle/service/jpm/MRJobTaskCountResponse.java
index c546198..170533c 100644
--- a/eagle-jpm/eagle-jpm-service/src/main/java/org/apache/eagle/service/jpm/MRJobTaskCountResponse.java
+++ b/eagle-jpm/eagle-jpm-service/src/main/java/org/apache/eagle/service/jpm/MRJobTaskCountResponse.java
@@ -30,6 +30,7 @@ public class MRJobTaskCountResponse {
     }
 
     public static class JobCountResponse extends MRJobTaskCountResponse {
+        public Set<String> jobTypes;
         public List<UnitJobCount> jobCounts;
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/fce6ae3f/eagle-jpm/eagle-jpm-service/src/test/java/org/apache/eagle/service/jpm/TestJobCountPerBucketHelper.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-service/src/test/java/org/apache/eagle/service/jpm/TestJobCountPerBucketHelper.java b/eagle-jpm/eagle-jpm-service/src/test/java/org/apache/eagle/service/jpm/TestJobCountPerBucketHelper.java
index 718f068..c8d8869 100644
--- a/eagle-jpm/eagle-jpm-service/src/test/java/org/apache/eagle/service/jpm/TestJobCountPerBucketHelper.java
+++ b/eagle-jpm/eagle-jpm-service/src/test/java/org/apache/eagle/service/jpm/TestJobCountPerBucketHelper.java
@@ -40,8 +40,11 @@ public class TestJobCountPerBucketHelper {
         String timeString2 = DateTimeUtil.secondsToHumanDate(timestamp);
         Assert.assertTrue(timeString2.equals(timeString));
 
-        String timeString3 = helper.moveTimeforwardOneDay(timeString);
+        String timeString3 = helper.moveTimeForwardOneDay(timeString);
         Assert.assertTrue(timeString3.equals("2016-08-21 20:13:00"));
+
+        String timeString4 = helper.moveTimeForwardOneDay(timeString3);
+        Assert.assertTrue(timeString4.equals("2016-08-20 20:13:00"));
     }
 
     @Test
@@ -53,7 +56,7 @@ public class TestJobCountPerBucketHelper {
         /*for (MRJobTaskCountResponse.UnitJobCount jobCount : jobCounts) {
             LOG.info(DateTimeUtil.secondsToHumanDate(jobCount.timeBucket));
         }*/
-        Assert.assertTrue(DateTimeUtil.secondsToHumanDate(jobCounts.get(1).timeBucket).equals("2016-08-22 20:15:00"));
+        Assert.assertTrue(DateTimeUtil.millisecondsToHumanDateWithSeconds(jobCounts.get(1).timeBucket).equals("2016-08-22 20:15:00"));
     }
 
     @Test


[22/52] [abbrv] incubator-eagle git commit: Add test case for MapComparator function

Posted by yo...@apache.org.
Add test case for MapComparator function

- Add test case for getAdded/getRemoved/getModified method

Author: joe-hj <jo...@gmail.com>

Closes #397 from joe-hj/ut.


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

Branch: refs/heads/master
Commit: a10eeb7019b8eff964316b084c0b25b68bf58b44
Parents: 000b746
Author: joe-hj <jo...@gmail.com>
Authored: Mon Aug 29 10:16:38 2016 +0800
Committer: Hao Chen <ha...@apache.org>
Committed: Mon Aug 29 10:16:38 2016 +0800

----------------------------------------------------------------------
 .../engine/router/TestAlertPublisherBolt.java   | 53 ++++++++++++++++++++
 .../src/test/resources/testPublishForAdd0.json  | 22 ++++++++
 .../src/test/resources/testPublishForAdd1.json  | 32 ++++++++++++
 .../test/resources/testPublishForMdyValue.json  | 22 ++++++++
 4 files changed, 129 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/a10eeb70/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/test/java/org/apache/eagle/alert/engine/router/TestAlertPublisherBolt.java
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/test/java/org/apache/eagle/alert/engine/router/TestAlertPublisherBolt.java b/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/test/java/org/apache/eagle/alert/engine/router/TestAlertPublisherBolt.java
index 9989068..1854c41 100644
--- a/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/test/java/org/apache/eagle/alert/engine/router/TestAlertPublisherBolt.java
+++ b/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/test/java/org/apache/eagle/alert/engine/router/TestAlertPublisherBolt.java
@@ -78,6 +78,59 @@ public class TestAlertPublisherBolt {
         return alert;
     }
 
+
+    @Test
+    public void testMapComparatorAdded() {
+
+        PublishSpec spec1 = MetadataSerDeser.deserialize(getClass().getResourceAsStream("/testPublishForAdd1.json"), PublishSpec.class);
+        PublishSpec spec2 = MetadataSerDeser.deserialize(getClass().getResourceAsStream("/testPublishForAdd0.json"), PublishSpec.class);
+
+        Map<String, Publishment> map1 = new HashMap<>();
+        Map<String, Publishment> map2 = new HashMap<>();
+        spec1.getPublishments().forEach(p -> map1.put(p.getName(), p));
+        spec2.getPublishments().forEach(p -> map2.put(p.getName(), p));
+
+        MapComparator<String, Publishment> comparator = new MapComparator<>(map1, map2);
+        comparator.compare();
+        Assert.assertTrue(comparator.getAdded().size() == 1);
+
+    }
+
+    @Test
+    public void testMapComparatorRemoved() {
+
+        PublishSpec spec1 = MetadataSerDeser.deserialize(getClass().getResourceAsStream("/testPublishForAdd0.json"), PublishSpec.class);
+        PublishSpec spec2 = MetadataSerDeser.deserialize(getClass().getResourceAsStream("/testPublishForAdd1.json"), PublishSpec.class);
+
+        Map<String, Publishment> map1 = new HashMap<>();
+        Map<String, Publishment> map2 = new HashMap<>();
+        spec1.getPublishments().forEach(p -> map1.put(p.getName(), p));
+        spec2.getPublishments().forEach(p -> map2.put(p.getName(), p));
+
+        MapComparator<String, Publishment> comparator = new MapComparator<>(map1, map2);
+        comparator.compare();
+        Assert.assertTrue(comparator.getRemoved().size() == 1);
+
+    }
+
+    @Test
+    public void testMapComparatorModified() {
+
+        PublishSpec spec1 = MetadataSerDeser.deserialize(getClass().getResourceAsStream("/testPublishForAdd0.json"), PublishSpec.class);
+        PublishSpec spec2 = MetadataSerDeser.deserialize(getClass().getResourceAsStream("/testPublishForMdyValue.json"), PublishSpec.class);
+
+        Map<String, Publishment> map1 = new HashMap<>();
+        Map<String, Publishment> map2 = new HashMap<>();
+        spec1.getPublishments().forEach(p -> map1.put(p.getName(), p));
+        spec2.getPublishments().forEach(p -> map2.put(p.getName(), p));
+
+        MapComparator<String, Publishment> comparator = new MapComparator<>(map1, map2);
+        comparator.compare();
+        Assert.assertTrue(comparator.getModified().size() == 1);
+
+    }
+
+
     @Test
     public void testMapComparator() {
         PublishSpec spec1 = MetadataSerDeser.deserialize(getClass().getResourceAsStream("/testPublishSpec.json"), PublishSpec.class);

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/a10eeb70/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/test/resources/testPublishForAdd0.json
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/test/resources/testPublishForAdd0.json b/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/test/resources/testPublishForAdd0.json
new file mode 100644
index 0000000..9ba7adb
--- /dev/null
+++ b/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/test/resources/testPublishForAdd0.json
@@ -0,0 +1,22 @@
+{
+  "version": "version1",
+  "topologyName": "testTopology",
+  "boltId": "alertPublishBolt",
+  "publishments": [
+    {
+      "type": "org.apache.eagle.alert.engine.publisher.impl.AlertEmailPublisher",
+      "name":"email-testAlertStream",
+      "policyIds": ["policy1", "policy2", "policy3"],
+      "dedupIntervalMin": "PT1M",
+      "properties":{
+        "subject":"UMP Test Alert",
+        "template":"",
+	  "sender": "sender@corp.com",
+	  "recipients": "receiver@corp.com",
+	  "smtp.server":"mailhost.com",
+        "connection": "plaintext",
+        "smtp.port": "25"
+      }
+    }
+  ]
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/a10eeb70/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/test/resources/testPublishForAdd1.json
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/test/resources/testPublishForAdd1.json b/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/test/resources/testPublishForAdd1.json
new file mode 100644
index 0000000..3974584
--- /dev/null
+++ b/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/test/resources/testPublishForAdd1.json
@@ -0,0 +1,32 @@
+{
+  "version": "version1",
+  "topologyName": "testTopology",
+  "boltId": "alertPublishBolt",
+  "publishments": [
+    {
+      "type": "org.apache.eagle.alert.engine.publisher.impl.AlertEmailPublisher",
+      "name":"email-testAlertStream",
+      "policyIds": ["policy1", "policy2", "policy3"],
+      "dedupIntervalMin": "PT1M",
+      "properties":{
+        "subject":"UMP Test Alert",
+        "template":"",
+	  "sender": "sender@corp.com",
+	  "recipients": "receiver@corp.com",
+	  "smtp.server":"mailhost.com",
+        "connection": "plaintext",
+        "smtp.port": "25"
+      }
+    },
+    {
+      "type": "org.apache.eagle.alert.engine.publisher.impl.AlertKafkaPublisher",
+      "name":"kafka-testAlertStream",
+      "policyIds": ["testPolicy"],
+      "dedupIntervalMin": "PT1M",
+      "properties":{
+        "kafka_broker":"sandbox.hortonworks.com:6667",
+        "topic":"test_kafka"
+      }
+    }
+  ]
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/a10eeb70/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/test/resources/testPublishForMdyValue.json
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/test/resources/testPublishForMdyValue.json b/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/test/resources/testPublishForMdyValue.json
new file mode 100644
index 0000000..4e1df83
--- /dev/null
+++ b/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/test/resources/testPublishForMdyValue.json
@@ -0,0 +1,22 @@
+{
+  "version": "version1",
+  "topologyName": "testTopology",
+  "boltId": "alertPublishBolt",
+  "publishments": [
+    {
+      "type": "org.apache.eagle.alert.engine.publisher.impl.AlertEmailPublisher",
+      "name":"email-testAlertStream",
+      "policyIds": ["policy1", "policy2", "policy3", "policy_MdyValue"],
+      "dedupIntervalMin": "PT1M",
+      "properties":{
+        "subject":"UMP Test Alert",
+        "template":"",
+	  "sender": "sender@corp.com",
+	  "recipients": "receiver@corp.com",
+	  "smtp.server":"mailhost.com",
+        "connection": "plaintext",
+        "smtp.port": "25"
+      }
+    }
+  ]
+}
\ No newline at end of file


[39/52] [abbrv] incubator-eagle git commit: Update spark history job feeder config & refactor the code

Posted by yo...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/3110c72e/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/spark/running/entities/SparkJobEntity.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/spark/running/entities/SparkJobEntity.java b/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/spark/running/entities/SparkJobEntity.java
new file mode 100644
index 0000000..211d6b7
--- /dev/null
+++ b/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/spark/running/entities/SparkJobEntity.java
@@ -0,0 +1,191 @@
+/*
+ *  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.eagle.jpm.spark.running.entities;
+
+import org.apache.eagle.jpm.util.Constants;
+import org.apache.eagle.log.base.taggedlog.TaggedLogAPIEntity;
+import org.apache.eagle.log.entity.meta.*;
+import org.codehaus.jackson.annotate.JsonIgnoreProperties;
+
+import java.util.List;
+
+@Table("eagleSparkRunningJobs")
+@ColumnFamily("f")
+@Prefix("sparkJob")
+@Service(Constants.RUNNING_SPARK_JOB_SERVICE_ENDPOINT_NAME)
+@JsonIgnoreProperties(ignoreUnknown = true)
+@TimeSeries(true)
+@Tags({"site","sparkAppId", "sparkAppAttemptId", "sparkAppName", "jobId","user", "queue"})
+@Partition({"site"})
+public class SparkJobEntity extends TaggedLogAPIEntity {
+    @Column("a")
+    private long  submissionTime;
+    @Column("b")
+    private long completionTime;
+    @Column("c")
+    private int numStages = 0;
+    @Column("d")
+    private String status;
+    @Column("e")
+    private int numTask = 0;
+    @Column("f")
+    private int numActiveTasks = 0;
+    @Column("g")
+    private int numCompletedTasks = 0;
+    @Column("h")
+    private int numSkippedTasks = 0;
+    @Column("i")
+    private int numFailedTasks = 0;
+    @Column("j")
+    private int numActiveStages = 0;
+    @Column("k")
+    private int numCompletedStages = 0;
+    @Column("l")
+    private int numSkippedStages = 0;
+    @Column("m")
+    private int numFailedStages = 0;
+    @Column("n")
+    private List<Integer> stages;
+
+    public List<Integer> getStages() {
+        return stages;
+    }
+
+    public void setStages(List<Integer> stages) {
+        this.stages = stages;
+        this.valueChanged("stages");
+    }
+
+    public long getSubmissionTime() {
+        return submissionTime;
+    }
+
+    public long getCompletionTime() {
+        return completionTime;
+    }
+
+    public int getNumStages() {
+        return numStages;
+    }
+
+    public String getStatus() {
+        return status;
+    }
+
+    public int getNumTask() {
+        return numTask;
+    }
+
+    public int getNumActiveTasks() {
+        return numActiveTasks;
+    }
+
+    public int getNumCompletedTasks() {
+        return numCompletedTasks;
+    }
+
+    public int getNumSkippedTasks() {
+        return numSkippedTasks;
+    }
+
+    public int getNumFailedTasks() {
+        return numFailedTasks;
+    }
+
+    public int getNumActiveStages() {
+        return numActiveStages;
+    }
+
+    public int getNumCompletedStages() {
+        return numCompletedStages;
+    }
+
+    public int getNumSkippedStages() {
+        return numSkippedStages;
+    }
+
+    public int getNumFailedStages() {
+        return numFailedStages;
+    }
+
+    public void setSubmissionTime(long submissionTime) {
+        this.submissionTime = submissionTime;
+        this.valueChanged("submissionTime");
+    }
+
+    public void setCompletionTime(long completionTime) {
+        this.completionTime = completionTime;
+        this.valueChanged("completionTime");
+    }
+
+    public void setNumStages(int numStages) {
+        this.numStages = numStages;
+        this.valueChanged("numStages");
+    }
+
+    public void setStatus(String status) {
+        this.status = status;
+        this.valueChanged("status");
+    }
+
+    public void setNumTask(int numTask) {
+        this.numTask = numTask;
+        this.valueChanged("numTask");
+    }
+
+    public void setNumActiveTasks(int numActiveTasks) {
+        this.numActiveTasks = numActiveTasks;
+        this.valueChanged("numActiveTasks");
+    }
+
+    public void setNumCompletedTasks(int numCompletedTasks) {
+        this.numCompletedTasks = numCompletedTasks;
+        this.valueChanged("numCompletedTasks");
+    }
+
+    public void setNumSkippedTasks(int numSkippedTasks) {
+        this.numSkippedTasks = numSkippedTasks;
+        this.valueChanged("numSkippedTasks");
+    }
+
+    public void setNumFailedTasks(int numFailedTasks) {
+        this.numFailedTasks = numFailedTasks;
+        this.valueChanged("numFailedTasks");
+    }
+
+    public void setNumActiveStages(int numActiveStages) {
+        this.numActiveStages = numActiveStages;
+        this.valueChanged("numActiveStages");
+    }
+
+    public void setNumCompletedStages(int numCompletedStages) {
+        this.numCompletedStages = numCompletedStages;
+        this.valueChanged("numCompletedStages");
+    }
+
+    public void setNumSkippedStages(int numSkippedStages) {
+        this.numSkippedStages = numSkippedStages;
+        this.valueChanged("numSkippedStages");
+    }
+
+    public void setNumFailedStages(int numFailedStages) {
+        this.numFailedStages = numFailedStages;
+        this.valueChanged("numFailedStages");
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/3110c72e/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/spark/running/entities/SparkStageEntity.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/spark/running/entities/SparkStageEntity.java b/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/spark/running/entities/SparkStageEntity.java
new file mode 100644
index 0000000..0194132
--- /dev/null
+++ b/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/spark/running/entities/SparkStageEntity.java
@@ -0,0 +1,299 @@
+/*
+ *  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.eagle.jpm.spark.running.entities;
+
+import org.apache.eagle.jpm.util.Constants;
+import org.apache.eagle.log.base.taggedlog.TaggedLogAPIEntity;
+import org.apache.eagle.log.entity.meta.*;
+import org.codehaus.jackson.annotate.JsonIgnoreProperties;
+
+@Table("eagleSparkRunningStages")
+@ColumnFamily("f")
+@Prefix("sparkStage")
+@Service(Constants.RUNNING_SPARK_STAGE_SERVICE_ENDPOINT_NAME)
+@JsonIgnoreProperties(ignoreUnknown = true)
+@TimeSeries(true)
+@Tags({"site","sparkAppId", "sparkAppAttemptId", "sparkAppName", "jobId", "stageId","stageAttemptId","user", "queue"})
+@Partition({"site"})
+public class SparkStageEntity extends TaggedLogAPIEntity {
+    @Column("a")
+    private String status;
+    @Column("b")
+    private int numActiveTasks = 0;
+    @Column("c")
+    private int numCompletedTasks = 0;
+    @Column("d")
+    private int numFailedTasks = 0;
+    @Column("e")
+    private long executorRunTime = 0L;
+    @Column("f")
+    private long inputBytes = 0L;
+    @Column("g")
+    private long inputRecords = 0L;
+    @Column("h")
+    private long outputBytes = 0L;
+    @Column("i")
+    private long outputRecords = 0L;
+    @Column("j")
+    private long shuffleReadBytes = 0L;
+    @Column("k")
+    private long shuffleReadRecords = 0L;
+    @Column("l")
+    private long shuffleWriteBytes = 0L;
+    @Column("m")
+    private long shuffleWriteRecords = 0L;
+    @Column("n")
+    private long memoryBytesSpilled = 0L;
+    @Column("o")
+    private long diskBytesSpilled = 0L;
+    @Column("p")
+    private String name;
+    @Column("q")
+    private String schedulingPool;
+    @Column("r")
+    private long submitTime;
+    @Column("s")
+    private long completeTime;
+    @Column("t")
+    private int numTasks;
+    @Column("u")
+    private long executorDeserializeTime;
+    @Column("v")
+    private long resultSize;
+    @Column("w")
+    private long jvmGcTime;
+    @Column("x")
+    private long resultSerializationTime;
+
+    public String getStatus() {
+        return status;
+    }
+
+    public int getNumActiveTasks() {
+        return numActiveTasks;
+    }
+
+    public int getNumCompletedTasks() {
+        return numCompletedTasks;
+    }
+
+    public int getNumFailedTasks() {
+        return numFailedTasks;
+    }
+
+    public long getExecutorRunTime() {
+        return executorRunTime;
+    }
+
+    public long getInputBytes() {
+        return inputBytes;
+    }
+
+    public long getInputRecords() {
+        return inputRecords;
+    }
+
+    public long getOutputBytes() {
+        return outputBytes;
+    }
+
+    public long getOutputRecords() {
+        return outputRecords;
+    }
+
+    public long getShuffleReadBytes() {
+        return shuffleReadBytes;
+    }
+
+    public long getShuffleReadRecords() {
+        return shuffleReadRecords;
+    }
+
+    public long getShuffleWriteBytes() {
+        return shuffleWriteBytes;
+    }
+
+    public long getShuffleWriteRecords() {
+        return shuffleWriteRecords;
+    }
+
+    public long getMemoryBytesSpilled() {
+        return memoryBytesSpilled;
+    }
+
+    public long getDiskBytesSpilled() {
+        return diskBytesSpilled;
+    }
+
+    public String getName() {
+        return name;
+    }
+
+    public String getSchedulingPool() {
+        return schedulingPool;
+    }
+
+    public long getSubmitTime() {
+        return submitTime;
+    }
+
+    public long getCompleteTime() {
+        return completeTime;
+    }
+
+    public int getNumTasks() {
+        return numTasks;
+    }
+
+    public long getExecutorDeserializeTime() {
+        return executorDeserializeTime;
+    }
+
+    public long getResultSize() {
+        return resultSize;
+    }
+
+    public long getJvmGcTime() {
+        return jvmGcTime;
+    }
+
+    public long getResultSerializationTime() {
+        return resultSerializationTime;
+    }
+
+    public void setStatus(String status) {
+        this.status = status;
+        this.valueChanged("status");
+    }
+
+    public void setNumActiveTasks(int numActiveTasks) {
+        this.numActiveTasks = numActiveTasks;
+        this.valueChanged("numActiveTasks");
+    }
+
+    public void setNumCompletedTasks(int numCompletedTasks) {
+        this.numCompletedTasks = numCompletedTasks;
+        this.valueChanged("numCompletedTasks");
+    }
+
+    public void setNumFailedTasks(int numFailedTasks) {
+        this.numFailedTasks = numFailedTasks;
+        this.valueChanged("numFailedTasks");
+    }
+
+    public void setExecutorRunTime(long executorRunTime) {
+        this.executorRunTime = executorRunTime;
+        this.valueChanged("executorRunTime");
+    }
+
+    public void setInputBytes(long inputBytes) {
+        this.inputBytes = inputBytes;
+        this.valueChanged("inputBytes");
+    }
+
+    public void setInputRecords(long inputRecords) {
+        this.inputRecords = inputRecords;
+        this.valueChanged("inputRecords");
+    }
+
+    public void setOutputBytes(long outputBytes) {
+        this.outputBytes = outputBytes;
+        this.valueChanged("outputBytes");
+    }
+
+    public void setOutputRecords(long outputRecords) {
+        this.outputRecords = outputRecords;
+        this.valueChanged("outputRecords");
+    }
+
+    public void setShuffleReadBytes(long shuffleReadBytes) {
+        this.shuffleReadBytes = shuffleReadBytes;
+        this.valueChanged("shuffleReadBytes");
+    }
+
+    public void setShuffleReadRecords(long shuffleReadRecords) {
+        this.shuffleReadRecords = shuffleReadRecords;
+        this.valueChanged("shuffleReadRecords");
+    }
+
+    public void setShuffleWriteBytes(long shuffleWriteBytes) {
+        this.shuffleWriteBytes = shuffleWriteBytes;
+        this.valueChanged("shuffleWriteBytes");
+    }
+
+    public void setShuffleWriteRecords(long shuffleWriteRecords) {
+        this.shuffleWriteRecords = shuffleWriteRecords;
+        this.valueChanged("shuffleWriteRecords");
+    }
+
+    public void setMemoryBytesSpilled(long memoryBytesSpilled) {
+        this.memoryBytesSpilled = memoryBytesSpilled;
+        this.valueChanged("memoryBytesSpilled");
+    }
+
+    public void setDiskBytesSpilled(long diskBytesSpilled) {
+        this.diskBytesSpilled = diskBytesSpilled;
+        this.valueChanged("diskBytesSpilled");
+    }
+
+    public void setName(String name) {
+        this.name = name;
+        this.valueChanged("name");
+    }
+
+    public void setSchedulingPool(String schedulingPool) {
+        this.schedulingPool = schedulingPool;
+        this.valueChanged("schedulingPool");
+    }
+
+    public void setSubmitTime(long submitTime) {
+        this.submitTime = submitTime;
+        this.valueChanged("submitTime");
+    }
+
+    public void setCompleteTime(long completeTime) {
+        this.completeTime = completeTime;
+        this.valueChanged("completeTime");
+    }
+
+    public void setNumTasks(int numTasks) {
+        this.numTasks = numTasks;
+        valueChanged("numTasks");
+    }
+
+    public void setExecutorDeserializeTime(long executorDeserializeTime) {
+        this.executorDeserializeTime = executorDeserializeTime;
+        valueChanged("executorDeserializeTime");
+    }
+
+    public void setResultSize(long resultSize) {
+        this.resultSize = resultSize;
+        valueChanged("resultSize");
+    }
+
+    public void setJvmGcTime(long jvmGcTime) {
+        this.jvmGcTime = jvmGcTime;
+        valueChanged("jvmGcTime");
+    }
+
+    public void setResultSerializationTime(long resultSerializationTime) {
+        this.resultSerializationTime = resultSerializationTime;
+        valueChanged("resultSerializationTime");
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/3110c72e/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/spark/running/entities/SparkTaskEntity.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/spark/running/entities/SparkTaskEntity.java b/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/spark/running/entities/SparkTaskEntity.java
new file mode 100644
index 0000000..6522c3c
--- /dev/null
+++ b/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/spark/running/entities/SparkTaskEntity.java
@@ -0,0 +1,290 @@
+/*
+ *  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.eagle.jpm.spark.running.entities;
+
+import org.apache.eagle.jpm.util.Constants;
+import org.apache.eagle.log.base.taggedlog.TaggedLogAPIEntity;
+import org.apache.eagle.log.entity.meta.*;
+import org.codehaus.jackson.annotate.JsonIgnoreProperties;
+
+@Table("eagleSparkRunningTasks")
+@ColumnFamily("f")
+@Prefix("sparkTask")
+@Service(Constants.RUNNING_SPARK_TASK_SERVICE_ENDPOINT_NAME)
+@JsonIgnoreProperties(ignoreUnknown = true)
+@TimeSeries(true)
+@Tags({"site","sparkAppId", "sparkAppAttemptId", "sparkAppName", "jobId", "jobName", "stageId","stageAttemptId","taskIndex","taskAttemptId","user", "queue"})
+@Partition({"site"})
+public class SparkTaskEntity extends TaggedLogAPIEntity {
+    @Column("a")
+    private int taskId;
+    @Column("b")
+    private long launchTime;
+    @Column("c")
+    private String executorId;
+    @Column("d")
+    private String host;
+    @Column("e")
+    private String taskLocality;
+    @Column("f")
+    private boolean speculative;
+    @Column("g")
+    private long executorDeserializeTime;
+    @Column("h")
+    private long executorRunTime;
+    @Column("i")
+    private long resultSize;
+    @Column("j")
+    private long jvmGcTime;
+    @Column("k")
+    private long resultSerializationTime;
+    @Column("l")
+    private long memoryBytesSpilled;
+    @Column("m")
+    private long diskBytesSpilled;
+    @Column("n")
+    private long inputBytes;
+    @Column("o")
+    private long inputRecords;
+    @Column("p")
+    private long outputBytes;
+    @Column("q")
+    private long outputRecords;
+    @Column("r")
+    private long shuffleReadRemoteBytes;
+    @Column("x")
+    private long shuffleReadLocalBytes;
+    @Column("s")
+    private long shuffleReadRecords;
+    @Column("t")
+    private long shuffleWriteBytes;
+    @Column("u")
+    private long shuffleWriteRecords;
+    @Column("v")
+    private boolean failed;
+
+    public int getTaskId() {
+        return taskId;
+    }
+
+    public long getLaunchTime() {
+        return launchTime;
+    }
+
+    public String getExecutorId() {
+        return executorId;
+    }
+
+    public String getHost() {
+        return host;
+    }
+
+    public String getTaskLocality() {
+        return taskLocality;
+    }
+
+    public boolean isSpeculative() {
+        return speculative;
+    }
+
+    public long getExecutorDeserializeTime() {
+        return executorDeserializeTime;
+    }
+
+    public long getExecutorRunTime() {
+        return executorRunTime;
+    }
+
+    public long getResultSize() {
+        return resultSize;
+    }
+
+    public long getJvmGcTime() {
+        return jvmGcTime;
+    }
+
+    public long getResultSerializationTime() {
+        return resultSerializationTime;
+    }
+
+    public long getMemoryBytesSpilled() {
+        return memoryBytesSpilled;
+    }
+
+    public long getDiskBytesSpilled() {
+        return diskBytesSpilled;
+    }
+
+    public long getInputBytes() {
+        return inputBytes;
+    }
+
+    public long getInputRecords() {
+        return inputRecords;
+    }
+
+    public long getOutputBytes() {
+        return outputBytes;
+    }
+
+    public long getOutputRecords() {
+        return outputRecords;
+    }
+
+    public long getShuffleReadRecords() {
+        return shuffleReadRecords;
+    }
+
+    public long getShuffleWriteBytes() {
+        return shuffleWriteBytes;
+    }
+
+    public long getShuffleWriteRecords() {
+        return shuffleWriteRecords;
+    }
+
+    public boolean isFailed() {
+        return failed;
+    }
+
+    public long getShuffleReadRemoteBytes() {
+        return shuffleReadRemoteBytes;
+    }
+
+    public long getShuffleReadLocalBytes() {
+        return shuffleReadLocalBytes;
+    }
+
+    public void setFailed(boolean failed) {
+        this.failed = failed;
+        valueChanged("failed");
+    }
+
+    public void setTaskId(int taskId) {
+        this.taskId = taskId;
+        valueChanged("taskId");
+    }
+
+    public void setLaunchTime(long launchTime) {
+        this.launchTime = launchTime;
+        valueChanged("launchTime");
+    }
+
+    public void setExecutorId(String executorId) {
+        this.executorId = executorId;
+        valueChanged("executorId");
+    }
+
+    public void setHost(String host) {
+        this.host = host;
+        this.valueChanged("host");
+    }
+
+    public void setTaskLocality(String taskLocality) {
+        this.taskLocality = taskLocality;
+        this.valueChanged("taskLocality");
+    }
+
+    public void setSpeculative(boolean speculative) {
+        this.speculative = speculative;
+        this.valueChanged("speculative");
+    }
+
+    public void setExecutorDeserializeTime(long executorDeserializeTime) {
+        this.executorDeserializeTime = executorDeserializeTime;
+        this.valueChanged("executorDeserializeTime");
+    }
+
+    public void setExecutorRunTime(long executorRunTime) {
+        this.executorRunTime = executorRunTime;
+        this.valueChanged("executorRunTime");
+    }
+
+    public void setResultSize(long resultSize) {
+        this.resultSize = resultSize;
+        this.valueChanged("resultSize");
+    }
+
+    public void setJvmGcTime(long jvmGcTime) {
+        this.jvmGcTime = jvmGcTime;
+        this.valueChanged("jvmGcTime");
+    }
+
+    public void setResultSerializationTime(long resultSerializationTime) {
+        this.resultSerializationTime = resultSerializationTime;
+        this.valueChanged("resultSerializationTime");
+    }
+
+    public void setMemoryBytesSpilled(long memoryBytesSpilled) {
+        this.memoryBytesSpilled = memoryBytesSpilled;
+        this.valueChanged("memoryBytesSpilled");
+    }
+
+    public void setDiskBytesSpilled(long diskBytesSpilled) {
+        this.diskBytesSpilled = diskBytesSpilled;
+        this.valueChanged("diskBytesSpilled");
+    }
+
+    public void setInputBytes(long inputBytes) {
+        this.inputBytes = inputBytes;
+        this.valueChanged("inputBytes");
+    }
+
+    public void setInputRecords(long inputRecords) {
+        this.inputRecords = inputRecords;
+        this.valueChanged("inputRecords");
+    }
+
+    public void setOutputBytes(long outputBytes) {
+        this.outputBytes = outputBytes;
+        this.valueChanged("outputBytes");
+    }
+
+    public void setOutputRecords(long outputRecords) {
+        this.outputRecords = outputRecords;
+        this.valueChanged("outputRecords");
+    }
+
+
+
+    public void setShuffleReadRecords(long shuffleReadRecords) {
+        this.shuffleReadRecords = shuffleReadRecords;
+        this.valueChanged("shuffleReadRecords");
+    }
+
+    public void setShuffleWriteBytes(long shuffleWriteBytes) {
+        this.shuffleWriteBytes = shuffleWriteBytes;
+        this.valueChanged("shuffleWriteBytes");
+    }
+
+    public void setShuffleWriteRecords(long shuffleWriteRecords) {
+        this.shuffleWriteRecords = shuffleWriteRecords;
+        this.valueChanged("shuffleWriteRecords");
+    }
+
+    public void setShuffleReadRemoteBytes(long shuffleReadRemoteBytes) {
+        this.shuffleReadRemoteBytes = shuffleReadRemoteBytes;
+        this.valueChanged("shuffleReadRemoteBytes");
+    }
+
+    public void setShuffleReadLocalBytes(long shuffleReadLocalBytes) {
+        this.shuffleReadLocalBytes = shuffleReadLocalBytes;
+        this.valueChanged("shuffleReadLocalBytes");
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/3110c72e/eagle-jpm/eagle-jpm-spark-history/src/main/java/org/apache/eagle/jpm/spark/history/SparkHistoryJobAppConfig.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-spark-history/src/main/java/org/apache/eagle/jpm/spark/history/SparkHistoryJobAppConfig.java b/eagle-jpm/eagle-jpm-spark-history/src/main/java/org/apache/eagle/jpm/spark/history/SparkHistoryJobAppConfig.java
index 0fc74d7..284eeee 100644
--- a/eagle-jpm/eagle-jpm-spark-history/src/main/java/org/apache/eagle/jpm/spark/history/SparkHistoryJobAppConfig.java
+++ b/eagle-jpm/eagle-jpm-spark-history/src/main/java/org/apache/eagle/jpm/spark/history/SparkHistoryJobAppConfig.java
@@ -82,8 +82,6 @@ public class SparkHistoryJobAppConfig implements Serializable {
         this.eagleInfo.host = config.getString("eagleProps.eagle.service.host");
         this.eagleInfo.port = config.getInt("eagleProps.eagle.service.port");
 
-        this.stormConfig.topologyName = config.getString("storm.name");
-        this.stormConfig.workerNo = config.getInt("storm.worker.num");
         this.stormConfig.timeoutSec = config.getInt("storm.messageTimeoutSec");
         this.stormConfig.spoutPending = config.getInt("storm.pendingSpout");
         this.stormConfig.spoutCrawlInterval = config.getInt("storm.spoutCrawlInterval");
@@ -117,9 +115,7 @@ public class SparkHistoryJobAppConfig implements Serializable {
     }
 
     public static class StormConfig implements Serializable {
-        public int workerNo;
         public int timeoutSec;
-        public String topologyName;
         public int spoutPending;
         public int spoutCrawlInterval;
     }

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/3110c72e/eagle-jpm/eagle-jpm-spark-history/src/main/java/org/apache/eagle/jpm/spark/history/crawl/JHFInputStreamReader.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-spark-history/src/main/java/org/apache/eagle/jpm/spark/history/crawl/JHFInputStreamReader.java b/eagle-jpm/eagle-jpm-spark-history/src/main/java/org/apache/eagle/jpm/spark/history/crawl/JHFInputStreamReader.java
new file mode 100644
index 0000000..b73b52e
--- /dev/null
+++ b/eagle-jpm/eagle-jpm-spark-history/src/main/java/org/apache/eagle/jpm/spark/history/crawl/JHFInputStreamReader.java
@@ -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.
+ */
+
+package org.apache.eagle.jpm.spark.history.crawl;
+
+import java.io.InputStream;
+
+public interface JHFInputStreamReader {
+    void read(InputStream is) throws Exception;
+}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/3110c72e/eagle-jpm/eagle-jpm-spark-history/src/main/java/org/apache/eagle/jpm/spark/history/crawl/JHFParserBase.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-spark-history/src/main/java/org/apache/eagle/jpm/spark/history/crawl/JHFParserBase.java b/eagle-jpm/eagle-jpm-spark-history/src/main/java/org/apache/eagle/jpm/spark/history/crawl/JHFParserBase.java
new file mode 100644
index 0000000..047e2d5
--- /dev/null
+++ b/eagle-jpm/eagle-jpm-spark-history/src/main/java/org/apache/eagle/jpm/spark/history/crawl/JHFParserBase.java
@@ -0,0 +1,29 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.eagle.jpm.spark.history.crawl;
+
+import java.io.InputStream;
+
+public interface JHFParserBase {
+    /**
+     * this method will ensure to close the inputStream.
+     * @param is
+     * @throws Exception
+     */
+    void parse(InputStream is) throws Exception;
+}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/3110c72e/eagle-jpm/eagle-jpm-spark-history/src/main/java/org/apache/eagle/jpm/spark/history/crawl/JHFSparkEventReader.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-spark-history/src/main/java/org/apache/eagle/jpm/spark/history/crawl/JHFSparkEventReader.java b/eagle-jpm/eagle-jpm-spark-history/src/main/java/org/apache/eagle/jpm/spark/history/crawl/JHFSparkEventReader.java
new file mode 100644
index 0000000..571620a
--- /dev/null
+++ b/eagle-jpm/eagle-jpm-spark-history/src/main/java/org/apache/eagle/jpm/spark/history/crawl/JHFSparkEventReader.java
@@ -0,0 +1,713 @@
+/*
+ * 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.eagle.jpm.spark.history.crawl;
+
+import com.typesafe.config.Config;
+import com.typesafe.config.ConfigFactory;
+import org.apache.commons.lang.ArrayUtils;
+import org.apache.eagle.jpm.spark.entity.*;
+import org.apache.eagle.jpm.util.*;
+import org.apache.eagle.log.base.taggedlog.TaggedLogAPIEntity;
+import org.apache.eagle.service.client.EagleServiceClientException;
+import org.apache.eagle.service.client.impl.EagleServiceBaseClient;
+import org.apache.eagle.service.client.impl.EagleServiceClientImpl;
+import org.json.simple.JSONArray;
+import org.json.simple.JSONObject;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.*;
+
+public class JHFSparkEventReader {
+    private static final Logger LOG = LoggerFactory.getLogger(JHFSparkEventReader.class);
+
+    private static final int FLUSH_LIMIT = 500;
+    private long firstTaskLaunchTime;
+    private long lastEventTime;
+
+    private Map<String, SparkExecutor> executors;
+    private SparkApp app;
+    private Map<Integer, SparkJob> jobs;
+    private Map<String, SparkStage> stages;
+    private Map<Integer, Set<String>> jobStageMap;
+    private Map<Long, SparkTask> tasks;
+    private EagleServiceClientImpl client;
+    private Map<String, Map<Integer, Boolean>> stageTaskStatusMap;
+
+    private List<TaggedLogAPIEntity> createEntities;
+
+    private Config conf;
+
+    public JHFSparkEventReader(Map<String, String> baseTags, SparkApplicationInfo info) {
+        app = new SparkApp();
+        app.setTags(new HashMap<String, String>(baseTags));
+        app.setYarnState(info.getState());
+        app.setYarnStatus(info.getFinalStatus());
+        createEntities = new ArrayList<>();
+        jobs = new HashMap<Integer, SparkJob>();
+        stages = new HashMap<String, SparkStage>();
+        jobStageMap = new HashMap<Integer, Set<String>>();
+        tasks = new HashMap<Long, SparkTask>();
+        executors = new HashMap<String, SparkExecutor>();
+        stageTaskStatusMap = new HashMap<>();
+        conf = ConfigFactory.load();
+        this.initiateClient();
+    }
+
+    public SparkApp getApp() {
+        return this.app;
+    }
+
+    public void read(JSONObject eventObj) {
+        String eventType = (String) eventObj.get("Event");
+        if (eventType.equalsIgnoreCase(SparkEventType.SparkListenerApplicationStart.toString())) {
+            handleAppStarted(eventObj);
+        } else if (eventType.equalsIgnoreCase(SparkEventType.SparkListenerEnvironmentUpdate.toString())) {
+            handleEnvironmentSet(eventObj);
+        } else if (eventType.equalsIgnoreCase(SparkEventType.SparkListenerExecutorAdded.toString())) {
+            handleExecutorAdd(eventObj);
+        } else if (eventType.equalsIgnoreCase(SparkEventType.SparkListenerBlockManagerAdded.toString())) {
+            handleBlockManagerAdd(eventObj);
+        } else if (eventType.equalsIgnoreCase(SparkEventType.SparkListenerJobStart.toString())) {
+            handleJobStart(eventObj);
+        } else if (eventType.equalsIgnoreCase(SparkEventType.SparkListenerStageSubmitted.toString())) {
+            handleStageSubmit(eventObj);
+        } else if (eventType.equalsIgnoreCase(SparkEventType.SparkListenerTaskStart.toString())) {
+            handleTaskStart(eventObj);
+        } else if (eventType.equalsIgnoreCase(SparkEventType.SparkListenerTaskEnd.toString())) {
+            handleTaskEnd(eventObj);
+        } else if (eventType.equalsIgnoreCase(SparkEventType.SparkListenerStageCompleted.toString())) {
+            handleStageComplete(eventObj);
+        } else if (eventType.equalsIgnoreCase(SparkEventType.SparkListenerJobEnd.toString())) {
+            handleJobEnd(eventObj);
+        } else if (eventType.equalsIgnoreCase(SparkEventType.SparkListenerExecutorRemoved.toString())) {
+            handleExecutorRemoved(eventObj);
+        } else if (eventType.equalsIgnoreCase(SparkEventType.SparkListenerApplicationEnd.toString())) {
+            handleAppEnd(eventObj);
+        } else if (eventType.equalsIgnoreCase(SparkEventType.SparkListenerBlockManagerRemoved.toString())) {
+            //nothing to do now
+        } else {
+            LOG.info("Not registered event type:" + eventType);
+        }
+
+    }
+
+    private void handleEnvironmentSet(JSONObject event) {
+        app.setConfig(new JobConfig());
+        JSONObject sparkProps = (JSONObject) event.get("Spark Properties");
+
+        String[] additionalJobConf = conf.getString("basic.jobConf.additional.info").split(",\\s*");
+        String[] props = {"spark.yarn.app.id", "spark.executor.memory", "spark.driver.host", "spark.driver.port",
+            "spark.driver.memory", "spark.scheduler.pool", "spark.executor.cores", "spark.yarn.am.memory",
+            "spark.yarn.am.cores", "spark.yarn.executor.memoryOverhead", "spark.yarn.driver.memoryOverhead", "spark.yarn.am.memoryOverhead", "spark.master"};
+        String[] jobConf = (String[])ArrayUtils.addAll(additionalJobConf, props);
+        for (String prop : jobConf) {
+            if (sparkProps.containsKey(prop)) {
+                app.getConfig().getConfig().put(prop, (String) sparkProps.get(prop));
+            }
+        }
+    }
+
+    private Object getConfigVal(JobConfig config, String configName, String type) {
+        if (config.getConfig().containsKey(configName)) {
+            Object val = config.getConfig().get(configName);
+            if (type.equalsIgnoreCase(Integer.class.getName())) {
+                return Integer.parseInt((String) val);
+            } else {
+                return val;
+            }
+        } else {
+            if (type.equalsIgnoreCase(Integer.class.getName())) {
+                return conf.getInt("spark.defaultVal." + configName);
+            } else {
+                return conf.getString("spark.defaultVal." + configName);
+            }
+        }
+    }
+
+    private boolean isClientMode(JobConfig config) {
+        return config.getConfig().get("spark.master").equalsIgnoreCase("yarn-client");
+    }
+
+    private void handleAppStarted(JSONObject event) {
+        //need update all entities tag before app start
+        List<TaggedLogAPIEntity> entities = new ArrayList<TaggedLogAPIEntity>();
+        entities.addAll(this.executors.values());
+        entities.add(this.app);
+
+        long appStartTime = JSONUtils.getLong(event, "Timestamp", lastEventTime);
+        for (TaggedLogAPIEntity entity : entities) {
+            entity.getTags().put(SparkJobTagName.SPARK_APP_ID.toString(), JSONUtils.getString(event, "App ID"));
+            entity.getTags().put(SparkJobTagName.SPARK_APP_NAME.toString(), JSONUtils.getString(event, "App Name"));
+            // In yarn-client mode, attemptId is not available in the log, so we set attemptId = 1.
+            String attemptId = isClientMode(this.app.getConfig()) ? "1" : JSONUtils.getString(event, "App Attempt ID");
+            entity.getTags().put(SparkJobTagName.SPARK_APP_ATTEMPT_ID.toString(), attemptId);
+            // the second argument of getNormalizeName() is changed to null because the original code contains sensitive text
+            // original second argument looks like: this.app.getConfig().getConfig().get("xxx"), "xxx" is the sensitive text
+            entity.getTags().put(SparkJobTagName.SPARK_APP_NORM_NAME.toString(), this.getNormalizedName(JSONUtils.getString(event, "App Name"), null));
+            entity.getTags().put(SparkJobTagName.SPARK_USER.toString(), JSONUtils.getString(event, "User"));
+
+            entity.setTimestamp(appStartTime);
+        }
+
+        this.app.setStartTime(appStartTime);
+        this.lastEventTime = appStartTime;
+    }
+
+    private void handleExecutorAdd(JSONObject event) {
+        String executorID = (String) event.get("Executor ID");
+        long executorAddTime = JSONUtils.getLong(event, "Timestamp", lastEventTime);
+        this.lastEventTime = executorAddTime;
+        SparkExecutor executor = this.initiateExecutor(executorID, executorAddTime);
+
+        JSONObject executorInfo = JSONUtils.getJSONObject(event, "Executor Info");
+
+    }
+
+    private void handleBlockManagerAdd(JSONObject event) {
+        long maxMemory = JSONUtils.getLong(event, "Maximum Memory");
+        long timestamp = JSONUtils.getLong(event, "Timestamp", lastEventTime);
+        this.lastEventTime = timestamp;
+        JSONObject blockInfo = JSONUtils.getJSONObject(event, "Block Manager ID");
+        String executorID = JSONUtils.getString(blockInfo, "Executor ID");
+        String hostAndPort = JSONUtils.getString(blockInfo, "Host") + ":" + JSONUtils.getLong(blockInfo, "Port");
+
+        SparkExecutor executor = this.initiateExecutor(executorID, timestamp);
+        executor.setMaxMemory(maxMemory);
+        executor.setHostPort(hostAndPort);
+    }
+
+    private void handleTaskStart(JSONObject event) {
+        this.initializeTask(event);
+    }
+
+    private void handleTaskEnd(JSONObject event) {
+        JSONObject taskInfo = JSONUtils.getJSONObject(event, "Task Info");
+        long taskId = JSONUtils.getLong(taskInfo, "Task ID");
+        SparkTask task = tasks.get(taskId);
+        if (task == null) {
+            return;
+        }
+
+        task.setFailed(JSONUtils.getBoolean(taskInfo, "Failed"));
+        JSONObject taskMetrics = JSONUtils.getJSONObject(event, "Task Metrics");
+        if (null != taskMetrics) {
+            task.setExecutorDeserializeTime(JSONUtils.getLong(taskMetrics, "Executor Deserialize Time", lastEventTime));
+            task.setExecutorRunTime(JSONUtils.getLong(taskMetrics, "Executor Run Time", lastEventTime));
+            task.setJvmGcTime(JSONUtils.getLong(taskMetrics, "JVM GC Time", lastEventTime));
+            task.setResultSize(JSONUtils.getLong(taskMetrics, "Result Size"));
+            task.setResultSerializationTime(JSONUtils.getLong(taskMetrics, "Result Serialization Time", lastEventTime));
+            task.setMemoryBytesSpilled(JSONUtils.getLong(taskMetrics, "Memory Bytes Spilled"));
+            task.setDiskBytesSpilled(JSONUtils.getLong(taskMetrics, "Disk Bytes Spilled"));
+
+            JSONObject inputMetrics = JSONUtils.getJSONObject(taskMetrics, "Input Metrics");
+            if (null != inputMetrics) {
+                task.setInputBytes(JSONUtils.getLong(inputMetrics, "Bytes Read"));
+                task.setInputRecords(JSONUtils.getLong(inputMetrics, "Records Read"));
+            }
+
+            JSONObject outputMetrics = JSONUtils.getJSONObject(taskMetrics, "Output Metrics");
+            if (null != outputMetrics) {
+                task.setOutputBytes(JSONUtils.getLong(outputMetrics, "Bytes Written"));
+                task.setOutputRecords(JSONUtils.getLong(outputMetrics, "Records Written"));
+            }
+
+            JSONObject shuffleWriteMetrics = JSONUtils.getJSONObject(taskMetrics, "Shuffle Write Metrics");
+            if (null != shuffleWriteMetrics) {
+                task.setShuffleWriteBytes(JSONUtils.getLong(shuffleWriteMetrics, "Shuffle Bytes Written"));
+                task.setShuffleWriteRecords(JSONUtils.getLong(shuffleWriteMetrics, "Shuffle Records Written"));
+            }
+
+            JSONObject shuffleReadMetrics = JSONUtils.getJSONObject(taskMetrics, "Shuffle Read Metrics");
+            if (null != shuffleReadMetrics) {
+                task.setShuffleReadLocalBytes(JSONUtils.getLong(shuffleReadMetrics, "Local Bytes Read"));
+                task.setShuffleReadRemoteBytes(JSONUtils.getLong(shuffleReadMetrics, "Remote Bytes Read"));
+                task.setShuffleReadRecords(JSONUtils.getLong(shuffleReadMetrics, "Total Records Read"));
+            }
+        } else {
+            //for tasks success without task metrics, save in the end if no other information
+            if (!task.isFailed()) {
+                return;
+            }
+        }
+
+        aggregateToStage(task);
+        aggregateToExecutor(task);
+        tasks.remove(taskId);
+        this.flushEntities(task, false);
+    }
+
+
+    private SparkTask initializeTask(JSONObject event) {
+        SparkTask task = new SparkTask();
+        task.setTags(new HashMap<>(this.app.getTags()));
+        task.setTimestamp(app.getTimestamp());
+
+        task.getTags().put(SparkJobTagName.SPARK_SATGE_ID.toString(), Long.toString(JSONUtils.getLong(event, "Stage ID")));
+        task.getTags().put(SparkJobTagName.SPARK_STAGE_ATTEMPT_ID.toString(), Long.toString(JSONUtils.getLong(event, "Stage Attempt ID")));
+
+        JSONObject taskInfo = JSONUtils.getJSONObject(event, "Task Info");
+        long taskId = JSONUtils.getLong(taskInfo, "Task ID");
+        task.setTaskId(taskId);
+
+        task.getTags().put(SparkJobTagName.SPARK_TASK_INDEX.toString(), Long.toString(JSONUtils.getLong(taskInfo, "Index")));
+        task.getTags().put(SparkJobTagName.SPARK_TASK_ATTEMPT_ID.toString(), Integer.toString(JSONUtils.getInt(taskInfo, "Attempt")));
+        long launchTime = JSONUtils.getLong(taskInfo, "Launch Time", lastEventTime);
+        this.lastEventTime = launchTime;
+        if (taskId == 0) {
+            this.setFirstTaskLaunchTime(launchTime);
+        }
+        task.setLaunchTime(launchTime);
+        task.setExecutorId(JSONUtils.getString(taskInfo, "Executor ID"));
+        task.setHost(JSONUtils.getString(taskInfo, "Host"));
+        task.setTaskLocality(JSONUtils.getString(taskInfo, "Locality"));
+        task.setSpeculative(JSONUtils.getBoolean(taskInfo, "Speculative"));
+
+        tasks.put(task.getTaskId(), task);
+        return task;
+    }
+
+    private void setFirstTaskLaunchTime(long launchTime) {
+        this.firstTaskLaunchTime = launchTime;
+    }
+
+    private void handleJobStart(JSONObject event) {
+        SparkJob job = new SparkJob();
+        job.setTags(new HashMap<>(this.app.getTags()));
+        job.setTimestamp(app.getTimestamp());
+
+        int jobId = JSONUtils.getInt(event, "Job ID");
+        job.getTags().put(SparkJobTagName.SPARK_JOB_ID.toString(), Integer.toString(jobId));
+        long submissionTime = JSONUtils.getLong(event, "Submission Time", lastEventTime);
+        job.setSubmissionTime(submissionTime);
+        this.lastEventTime = submissionTime;
+
+        //for complete application, no active stages/tasks
+        job.setNumActiveStages(0);
+        job.setNumActiveTasks(0);
+
+        this.jobs.put(jobId, job);
+        this.jobStageMap.put(jobId, new HashSet<String>());
+
+        JSONArray stages = JSONUtils.getJSONArray(event, "Stage Infos");
+        int stagesSize = (stages == null ? 0 : stages.size());
+        job.setNumStages(stagesSize);
+        for (int i = 0; i < stagesSize; i++) {
+            JSONObject stageInfo = (JSONObject) stages.get(i);
+            int stageId = JSONUtils.getInt(stageInfo, "Stage ID");
+            int stageAttemptId = JSONUtils.getInt(stageInfo, "Stage Attempt ID");
+            String stageName = JSONUtils.getString(stageInfo, "Stage Name");
+            int numTasks = JSONUtils.getInt(stageInfo, "Number of Tasks");
+            this.initiateStage(jobId, stageId, stageAttemptId, stageName, numTasks);
+        }
+    }
+
+    private void handleStageSubmit(JSONObject event) {
+        JSONObject stageInfo = JSONUtils.getJSONObject(event, "Stage Info");
+        int stageId = JSONUtils.getInt(stageInfo, "Stage ID");
+        int stageAttemptId = JSONUtils.getInt(stageInfo, "Stage Attempt ID");
+        String key = this.generateStageKey(Integer.toString(stageId), Integer.toString(stageAttemptId));
+        stageTaskStatusMap.put(key, new HashMap<Integer, Boolean>());
+
+        if (!stages.containsKey(key)) {
+            //may be further attempt for one stage
+            String baseAttempt = this.generateStageKey(Integer.toString(stageId), "0");
+            if (stages.containsKey(baseAttempt)) {
+                SparkStage stage = stages.get(baseAttempt);
+                String jobId = stage.getTags().get(SparkJobTagName.SPARK_JOB_ID.toString());
+
+                String stageName = JSONUtils.getString(event, "Stage Name");
+                int numTasks = JSONUtils.getInt(stageInfo, "Number of Tasks");
+                this.initiateStage(Integer.parseInt(jobId), stageId, stageAttemptId, stageName, numTasks);
+            }
+        }
+    }
+
+    private void handleStageComplete(JSONObject event) {
+        JSONObject stageInfo = JSONUtils.getJSONObject(event, "Stage Info");
+        int stageId = JSONUtils.getInt(stageInfo, "Stage ID");
+        int stageAttemptId = JSONUtils.getInt(stageInfo, "Stage Attempt ID");
+        String key = this.generateStageKey(Integer.toString(stageId), Integer.toString(stageAttemptId));
+        SparkStage stage = stages.get(key);
+
+        // If "Submission Time" is not available, use the "Launch Time" of "Task ID" = 0.
+        Long submissionTime = JSONUtils.getLong(stageInfo, "Submission Time", firstTaskLaunchTime);
+
+        stage.setSubmitTime(submissionTime);
+
+        long completeTime = JSONUtils.getLong(stageInfo, "Completion Time", lastEventTime);
+        stage.setCompleteTime(completeTime);
+        this.lastEventTime = completeTime;
+
+        if (stageInfo != null && stageInfo.containsKey("Failure Reason")) {
+            stage.setStatus(SparkEntityConstant.SparkStageStatus.FAILED.toString());
+        } else {
+            stage.setStatus(SparkEntityConstant.SparkStageStatus.COMPLETE.toString());
+        }
+    }
+
+    private void handleExecutorRemoved(JSONObject event) {
+        String executorID = JSONUtils.getString(event, "Executor ID");
+        SparkExecutor executor = executors.get(executorID);
+        long removedTime = JSONUtils.getLong(event, "Timestamp", lastEventTime);
+        executor.setEndTime(removedTime);
+        this.lastEventTime = removedTime;
+    }
+
+    private void handleJobEnd(JSONObject event) {
+        int jobId = JSONUtils.getInt(event, "Job ID");
+        SparkJob job = jobs.get(jobId);
+
+        long completionTime = JSONUtils.getLong(event, "Completion Time", lastEventTime);
+        job.setCompletionTime(completionTime);
+        this.lastEventTime = completionTime;
+
+        JSONObject jobResult = JSONUtils.getJSONObject(event, "Job Result");
+        String result = JSONUtils.getString(jobResult, "Result");
+        if (result.equalsIgnoreCase("JobSucceeded")) {
+            job.setStatus(SparkEntityConstant.SparkJobStatus.SUCCEEDED.toString());
+        } else {
+            job.setStatus(SparkEntityConstant.SparkJobStatus.FAILED.toString());
+        }
+    }
+
+    private void handleAppEnd(JSONObject event) {
+        long endTime = JSONUtils.getLong(event, "Timestamp", lastEventTime);
+        app.setEndTime(endTime);
+        this.lastEventTime = endTime;
+    }
+
+    public void clearReader() throws Exception {
+        //clear tasks
+        for (SparkTask task : tasks.values()) {
+            LOG.info("Task {} does not have result or no task metrics.", task.getTaskId());
+            task.setFailed(true);
+            aggregateToStage(task);
+            aggregateToExecutor(task);
+            this.flushEntities(task, false);
+        }
+
+        List<SparkStage> needStoreStages = new ArrayList<>();
+        for (SparkStage stage : this.stages.values()) {
+            int jobId = Integer.parseInt(stage.getTags().get(SparkJobTagName.SPARK_JOB_ID.toString()));
+            if (stage.getSubmitTime() == 0 || stage.getCompleteTime() == 0) {
+                SparkJob job = this.jobs.get(jobId);
+                job.setNumSkippedStages(job.getNumSkippedStages() + 1);
+                job.setNumSkippedTasks(job.getNumSkippedTasks() + stage.getNumTasks());
+            } else {
+                this.aggregateToJob(stage);
+                this.aggregateStageToApp(stage);
+                needStoreStages.add(stage);
+            }
+            String stageId = stage.getTags().get(SparkJobTagName.SPARK_SATGE_ID.toString());
+            String stageAttemptId = stage.getTags().get(SparkJobTagName.SPARK_STAGE_ATTEMPT_ID.toString());
+            this.jobStageMap.get(jobId).remove(this.generateStageKey(stageId, stageAttemptId));
+        }
+
+        this.flushEntities(needStoreStages, false);
+        for (SparkJob job : jobs.values()) {
+            this.aggregateJobToApp(job);
+        }
+        this.flushEntities(jobs.values(), false);
+
+        app.setExecutors(executors.values().size());
+
+        long executorMemory = Utils.parseMemory((String) this.getConfigVal(this.app.getConfig(), "spark.executor.memory", String.class.getName()));
+        long driverMemory = Utils.parseMemory(this.isClientMode(app.getConfig())
+            ? (String) this.getConfigVal(this.app.getConfig(), "spark.yarn.am.memory", String.class.getName())
+            : (String) this.getConfigVal(app.getConfig(), "spark.driver.memory", String.class.getName()));
+
+        int executorCore = (Integer) this.getConfigVal(app.getConfig(), "spark.executor.cores", Integer.class.getName());
+        int driverCore = this.isClientMode(app.getConfig())
+            ? (Integer) this.getConfigVal(app.getConfig(), "spark.yarn.am.cores", Integer.class.getName())
+            : (Integer) this.getConfigVal(app.getConfig(), "spark.driver.cores", Integer.class.getName());
+
+        long executorMemoryOverhead = this.getMemoryOverhead(app.getConfig(), executorMemory, "spark.yarn.executor.memoryOverhead");
+        long driverMemoryOverhead = this.isClientMode(app.getConfig())
+            ? this.getMemoryOverhead(app.getConfig(), driverMemory, "spark.yarn.am.memoryOverhead")
+            : this.getMemoryOverhead(app.getConfig(), driverMemory, "spark.yarn.driver.memoryOverhead");
+
+        app.setExecMemoryBytes(executorMemory);
+        app.setDriveMemoryBytes(driverMemory);
+        app.setExecutorCores(executorCore);
+        app.setDriverCores(driverCore);
+        app.setExecutorMemoryOverhead(executorMemoryOverhead);
+        app.setDriverMemoryOverhead(driverMemoryOverhead);
+
+        for (SparkExecutor executor : executors.values()) {
+            String executorID = executor.getTags().get(SparkJobTagName.SPARK_EXECUTOR_ID.toString());
+            if (executorID.equalsIgnoreCase("driver")) {
+                executor.setExecMemoryBytes(driverMemory);
+                executor.setCores(driverCore);
+                executor.setMemoryOverhead(driverMemoryOverhead);
+            } else {
+                executor.setExecMemoryBytes(executorMemory);
+                executor.setCores(executorCore);
+                executor.setMemoryOverhead(executorMemoryOverhead);
+            }
+            if (app.getEndTime() <= 0L) {
+                app.setEndTime(this.lastEventTime);
+            }
+            if (executor.getEndTime() <= 0L) {
+                executor.setEndTime(app.getEndTime());
+            }
+            this.aggregateExecutorToApp(executor);
+        }
+        this.flushEntities(executors.values(), false);
+        //spark code...tricky
+        app.setSkippedTasks(app.getCompleteTasks());
+        this.flushEntities(app, true);
+    }
+
+    private long getMemoryOverhead(JobConfig config, long executorMemory, String fieldName) {
+        long result = 0L;
+        String fieldValue = config.getConfig().get(fieldName);
+        if (fieldValue != null) {
+            result = Utils.parseMemory(fieldValue + "m");
+            if (result == 0L) {
+               result = Utils.parseMemory(fieldValue);
+            }
+        }
+
+        if (result == 0L) {
+            result = Math.max(
+                    Utils.parseMemory(conf.getString("spark.defaultVal.spark.yarn.overhead.min")),
+                    executorMemory * conf.getInt("spark.defaultVal." + fieldName + ".factor") / 100);
+        }
+        return result;
+    }
+
+    private void aggregateExecutorToApp(SparkExecutor executor) {
+        long totalExecutorTime = app.getTotalExecutorTime() + executor.getEndTime() - executor.getStartTime();
+        if (totalExecutorTime < 0L) {
+            totalExecutorTime = 0L;
+        }
+        app.setTotalExecutorTime(totalExecutorTime);
+    }
+
+    private void aggregateJobToApp(SparkJob job) {
+        //aggregate job level metrics
+        app.setNumJobs(app.getNumJobs() + 1);
+        app.setTotalTasks(app.getTotalTasks() + job.getNumTask());
+        app.setCompleteTasks(app.getCompleteTasks() + job.getNumCompletedTasks());
+        app.setSkippedTasks(app.getSkippedTasks() + job.getNumSkippedTasks());
+        app.setFailedTasks(app.getFailedTasks() + job.getNumFailedTasks());
+        app.setTotalStages(app.getTotalStages() + job.getNumStages());
+        app.setFailedStages(app.getFailedStages() + job.getNumFailedStages());
+        app.setSkippedStages(app.getSkippedStages() + job.getNumSkippedStages());
+    }
+
+    private void aggregateStageToApp(SparkStage stage) {
+        //aggregate task level metrics
+        app.setDiskBytesSpilled(app.getDiskBytesSpilled() + stage.getDiskBytesSpilled());
+        app.setMemoryBytesSpilled(app.getMemoryBytesSpilled() + stage.getMemoryBytesSpilled());
+        app.setExecutorRunTime(app.getExecutorRunTime() + stage.getExecutorRunTime());
+        app.setJvmGcTime(app.getJvmGcTime() + stage.getJvmGcTime());
+        app.setExecutorDeserializeTime(app.getExecutorDeserializeTime() + stage.getExecutorDeserializeTime());
+        app.setResultSerializationTime(app.getResultSerializationTime() + stage.getResultSerializationTime());
+        app.setResultSize(app.getResultSize() + stage.getResultSize());
+        app.setInputRecords(app.getInputRecords() + stage.getInputRecords());
+        app.setInputBytes(app.getInputBytes() + stage.getInputBytes());
+        app.setOutputRecords(app.getOutputRecords() + stage.getOutputRecords());
+        app.setOutputBytes(app.getOutputBytes() + stage.getOutputBytes());
+        app.setShuffleWriteRecords(app.getShuffleWriteRecords() + stage.getShuffleWriteRecords());
+        app.setShuffleWriteBytes(app.getShuffleWriteBytes() + stage.getShuffleWriteBytes());
+        app.setShuffleReadRecords(app.getShuffleReadRecords() + stage.getShuffleReadRecords());
+        app.setShuffleReadBytes(app.getShuffleReadBytes() + stage.getShuffleReadBytes());
+    }
+
+    private void aggregateToStage(SparkTask task) {
+        String stageId = task.getTags().get(SparkJobTagName.SPARK_SATGE_ID.toString());
+        String stageAttemptId = task.getTags().get(SparkJobTagName.SPARK_STAGE_ATTEMPT_ID.toString());
+        String key = this.generateStageKey(stageId, stageAttemptId);
+        SparkStage stage = stages.get(key);
+
+        stage.setDiskBytesSpilled(stage.getDiskBytesSpilled() + task.getDiskBytesSpilled());
+        stage.setMemoryBytesSpilled(stage.getMemoryBytesSpilled() + task.getMemoryBytesSpilled());
+        stage.setExecutorRunTime(stage.getExecutorRunTime() + task.getExecutorRunTime());
+        stage.setJvmGcTime(stage.getJvmGcTime() + task.getJvmGcTime());
+        stage.setExecutorDeserializeTime(stage.getExecutorDeserializeTime() + task.getExecutorDeserializeTime());
+        stage.setResultSerializationTime(stage.getResultSerializationTime() + task.getResultSerializationTime());
+        stage.setResultSize(stage.getResultSize() + task.getResultSize());
+        stage.setInputRecords(stage.getInputRecords() + task.getInputRecords());
+        stage.setInputBytes(stage.getInputBytes() + task.getInputBytes());
+        stage.setOutputRecords(stage.getOutputRecords() + task.getOutputRecords());
+        stage.setOutputBytes(stage.getOutputBytes() + task.getOutputBytes());
+        stage.setShuffleWriteRecords(stage.getShuffleWriteRecords() + task.getShuffleWriteRecords());
+        stage.setShuffleWriteBytes(stage.getShuffleWriteBytes() + task.getShuffleWriteBytes());
+        stage.setShuffleReadRecords(stage.getShuffleReadRecords() + task.getShuffleReadRecords());
+        long taskShuffleReadBytes = task.getShuffleReadLocalBytes() + task.getShuffleReadRemoteBytes();
+        stage.setShuffleReadBytes(stage.getShuffleReadBytes() + taskShuffleReadBytes);
+
+        boolean success = !task.isFailed();
+
+        Integer taskIndex = Integer.parseInt(task.getTags().get(SparkJobTagName.SPARK_TASK_INDEX.toString()));
+        if (stageTaskStatusMap.get(key).containsKey(taskIndex)) {
+            //has previous task attempt, retrieved from task index in one stage
+            boolean previousResult = stageTaskStatusMap.get(key).get(taskIndex);
+            success = previousResult || success;
+            if (previousResult != success) {
+                stage.setNumFailedTasks(stage.getNumFailedTasks() - 1);
+                stage.setNumCompletedTasks(stage.getNumCompletedTasks() + 1);
+                stageTaskStatusMap.get(key).put(taskIndex, success);
+            }
+        } else {
+            if (success) {
+                stage.setNumCompletedTasks(stage.getNumCompletedTasks() + 1);
+            } else {
+                stage.setNumFailedTasks(stage.getNumFailedTasks() + 1);
+            }
+            stageTaskStatusMap.get(key).put(taskIndex, success);
+        }
+
+    }
+
+    private void aggregateToExecutor(SparkTask task) {
+        String executorId = task.getExecutorId();
+        SparkExecutor executor = executors.get(executorId);
+
+        if (null != executor) {
+            executor.setTotalTasks(executor.getTotalTasks() + 1);
+            if (task.isFailed()) {
+                executor.setFailedTasks(executor.getFailedTasks() + 1);
+            } else {
+                executor.setCompletedTasks(executor.getCompletedTasks() + 1);
+            }
+            long taskShuffleReadBytes = task.getShuffleReadLocalBytes() + task.getShuffleReadRemoteBytes();
+            executor.setTotalShuffleRead(executor.getTotalShuffleRead() + taskShuffleReadBytes);
+            executor.setTotalDuration(executor.getTotalDuration() + task.getExecutorRunTime());
+            executor.setTotalInputBytes(executor.getTotalInputBytes() + task.getInputBytes());
+            executor.setTotalShuffleWrite(executor.getTotalShuffleWrite() + task.getShuffleWriteBytes());
+            executor.setTotalDuration(executor.getTotalDuration() + task.getExecutorRunTime());
+        }
+
+    }
+
+    private void aggregateToJob(SparkStage stage) {
+        int jobId = Integer.parseInt(stage.getTags().get(SparkJobTagName.SPARK_JOB_ID.toString()));
+        SparkJob job = jobs.get(jobId);
+        job.setNumCompletedTasks(job.getNumCompletedTasks() + stage.getNumCompletedTasks());
+        job.setNumFailedTasks(job.getNumFailedTasks() + stage.getNumFailedTasks());
+        job.setNumTask(job.getNumTask() + stage.getNumTasks());
+
+
+        if (stage.getStatus().equalsIgnoreCase(SparkEntityConstant.SparkStageStatus.COMPLETE.toString())) {
+            //if multiple attempts succeed, just count one
+            if (!hasStagePriorAttemptSuccess(stage)) {
+                job.setNumCompletedStages(job.getNumCompletedStages() + 1);
+            }
+        } else {
+            job.setNumFailedStages(job.getNumFailedStages() + 1);
+        }
+    }
+
+    private boolean hasStagePriorAttemptSuccess(SparkStage stage) {
+        int stageAttemptId = Integer.parseInt(stage.getTags().get(SparkJobTagName.SPARK_STAGE_ATTEMPT_ID.toString()));
+        for (int i = 0; i < stageAttemptId; i++) {
+            SparkStage previousStage = stages.get(this.generateStageKey(
+                    stage.getTags().get(SparkJobTagName.SPARK_SATGE_ID.toString()), Integer.toString(i)));
+            if (previousStage.getStatus().equalsIgnoreCase(SparkEntityConstant.SparkStageStatus.COMPLETE.toString())) {
+                return true;
+            }
+        }
+        return false;
+    }
+
+
+    private String generateStageKey(String stageId, String stageAttemptId) {
+        return stageId + "-" + stageAttemptId;
+    }
+
+    private void initiateStage(int jobId, int stageId, int stageAttemptId, String name, int numTasks) {
+        SparkStage stage = new SparkStage();
+        stage.setTags(new HashMap<>(this.app.getTags()));
+        stage.setTimestamp(app.getTimestamp());
+        stage.getTags().put(SparkJobTagName.SPARK_JOB_ID.toString(), Integer.toString(jobId));
+        stage.getTags().put(SparkJobTagName.SPARK_SATGE_ID.toString(), Integer.toString(stageId));
+        stage.getTags().put(SparkJobTagName.SPARK_STAGE_ATTEMPT_ID.toString(), Integer.toString(stageAttemptId));
+        stage.setName(name);
+        stage.setNumActiveTasks(0);
+        stage.setNumTasks(numTasks);
+        stage.setSchedulingPool(this.app.getConfig().getConfig().get("spark.scheduler.pool") == null ?
+                "default" : this.app.getConfig().getConfig().get("spark.scheduler.pool"));
+
+        String stageKey = this.generateStageKey(Integer.toString(stageId), Integer.toString(stageAttemptId));
+        stages.put(stageKey, stage);
+        this.jobStageMap.get(jobId).add(stageKey);
+    }
+
+
+    private SparkExecutor initiateExecutor(String executorID, long startTime) {
+        if (!executors.containsKey(executorID)) {
+            SparkExecutor executor = new SparkExecutor();
+            executor.setTags(new HashMap<>(this.app.getTags()));
+            executor.getTags().put(SparkJobTagName.SPARK_EXECUTOR_ID.toString(), executorID);
+            executor.setStartTime(startTime);
+            executor.setTimestamp(app.getTimestamp());
+
+            this.executors.put(executorID, executor);
+        }
+
+        return this.executors.get(executorID);
+    }
+
+    private String getNormalizedName(String jobName, String assignedName) {
+        if (null != assignedName) {
+            return assignedName;
+        } else {
+            return JobNameNormalization.getInstance().normalize(jobName);
+        }
+    }
+
+    private void flushEntities(Object entity, boolean forceFlush) {
+        this.flushEntities(Collections.singletonList(entity), forceFlush);
+    }
+
+    private void flushEntities(Collection entities, boolean forceFlush) {
+        this.createEntities.addAll(entities);
+
+        if (forceFlush || this.createEntities.size() >= FLUSH_LIMIT) {
+            try {
+                this.doFlush(this.createEntities);
+                this.createEntities.clear();
+            } catch (Exception e) {
+                LOG.error("Fail to flush entities", e);
+            }
+
+        }
+    }
+
+    private EagleServiceBaseClient initiateClient() {
+        String host = conf.getString("eagleProps.eagle.service.host");
+        int port = conf.getInt("eagleProps.eagle.service.port");
+        String userName = conf.getString("eagleProps.eagle.service.username");
+        String pwd = conf.getString("eagleProps.eagle.service.password");
+        client = new EagleServiceClientImpl(host, port, userName, pwd);
+        int timeout = conf.getInt("eagleProps.eagle.service.read.timeout");
+        client.getJerseyClient().setReadTimeout(timeout * 1000);
+
+        return client;
+    }
+
+    private void doFlush(List entities) throws IOException, EagleServiceClientException {
+        client.create(entities);
+        int size = (entities == null ? 0 : entities.size());
+        LOG.info("finish flushing entities of total number " + size);
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/3110c72e/eagle-jpm/eagle-jpm-spark-history/src/main/java/org/apache/eagle/jpm/spark/history/crawl/JHFSparkParser.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-spark-history/src/main/java/org/apache/eagle/jpm/spark/history/crawl/JHFSparkParser.java b/eagle-jpm/eagle-jpm-spark-history/src/main/java/org/apache/eagle/jpm/spark/history/crawl/JHFSparkParser.java
new file mode 100644
index 0000000..b1dd09c
--- /dev/null
+++ b/eagle-jpm/eagle-jpm-spark-history/src/main/java/org/apache/eagle/jpm/spark/history/crawl/JHFSparkParser.java
@@ -0,0 +1,73 @@
+/*
+ * 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.eagle.jpm.spark.history.crawl;
+
+
+import org.json.simple.JSONObject;
+import org.json.simple.parser.JSONParser;
+import org.json.simple.parser.ParseException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.BufferedReader;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+
+public class JHFSparkParser implements JHFParserBase {
+
+    private static final Logger logger = LoggerFactory.getLogger(JHFSparkParser.class);
+
+    private boolean isValidJson;
+
+    private JHFSparkEventReader eventReader;
+
+    public JHFSparkParser(JHFSparkEventReader reader) {
+        this.eventReader = reader;
+    }
+
+    @Override
+    public void parse(InputStream is) throws Exception {
+        try (BufferedReader reader = new BufferedReader(new InputStreamReader(is))) {
+            for (String line = reader.readLine(); line != null; line = reader.readLine()) {
+                isValidJson = true;
+                JSONObject eventObj = parseAndValidateJSON(line);
+                if (isValidJson) {
+                    try {
+                        this.eventReader.read(eventObj);
+                    } catch (Exception e) {
+                        e.printStackTrace();
+                    }
+                }
+            }
+
+            this.eventReader.clearReader();
+        }
+    }
+
+    private JSONObject parseAndValidateJSON(String line) {
+        JSONObject eventObj = null;
+        JSONParser parser = new JSONParser();
+        try {
+            eventObj = (JSONObject) parser.parse(line);
+        } catch (ParseException ex) {
+            isValidJson = false;
+            logger.error(String.format("Invalid json string. Fail to parse %s.", line), ex);
+        }
+        return eventObj;
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/3110c72e/eagle-jpm/eagle-jpm-spark-history/src/main/java/org/apache/eagle/jpm/spark/history/crawl/SparkApplicationInfo.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-spark-history/src/main/java/org/apache/eagle/jpm/spark/history/crawl/SparkApplicationInfo.java b/eagle-jpm/eagle-jpm-spark-history/src/main/java/org/apache/eagle/jpm/spark/history/crawl/SparkApplicationInfo.java
new file mode 100644
index 0000000..c206b71
--- /dev/null
+++ b/eagle-jpm/eagle-jpm-spark-history/src/main/java/org/apache/eagle/jpm/spark/history/crawl/SparkApplicationInfo.java
@@ -0,0 +1,69 @@
+/*
+ *
+ *  * 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.eagle.jpm.spark.history.crawl;
+
+public class SparkApplicationInfo {
+
+    private String state;
+    private String finalStatus;
+    private String queue;
+    private String name;
+    private String user;
+
+    public String getState() {
+        return state;
+    }
+
+    public void setState(String state) {
+        this.state = state;
+    }
+
+    public String getFinalStatus() {
+        return finalStatus;
+    }
+
+    public void setFinalStatus(String finalStatus) {
+        this.finalStatus = finalStatus;
+    }
+
+    public String getQueue() {
+        return queue;
+    }
+
+    public void setQueue(String queue) {
+        this.queue = queue;
+    }
+
+    public String getName() {
+        return name;
+    }
+
+    public void setName(String name) {
+        this.name = name;
+    }
+
+    public String getUser() {
+        return user;
+    }
+
+    public void setUser(String user) {
+        this.user = user;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/3110c72e/eagle-jpm/eagle-jpm-spark-history/src/main/java/org/apache/eagle/jpm/spark/history/crawl/SparkFilesystemInputStreamReaderImpl.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-spark-history/src/main/java/org/apache/eagle/jpm/spark/history/crawl/SparkFilesystemInputStreamReaderImpl.java b/eagle-jpm/eagle-jpm-spark-history/src/main/java/org/apache/eagle/jpm/spark/history/crawl/SparkFilesystemInputStreamReaderImpl.java
new file mode 100644
index 0000000..0144410
--- /dev/null
+++ b/eagle-jpm/eagle-jpm-spark-history/src/main/java/org/apache/eagle/jpm/spark/history/crawl/SparkFilesystemInputStreamReaderImpl.java
@@ -0,0 +1,53 @@
+/*
+ * 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.eagle.jpm.spark.history.crawl;
+
+import org.apache.eagle.jpm.util.SparkJobTagName;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.InputStream;
+import java.util.HashMap;
+import java.util.Map;
+
+public class SparkFilesystemInputStreamReaderImpl implements JHFInputStreamReader {
+
+    private String site;
+    private SparkApplicationInfo app;
+
+
+    public SparkFilesystemInputStreamReaderImpl(String site, SparkApplicationInfo app) {
+        this.site = site;
+        this.app = app;
+    }
+
+    @Override
+    public void read(InputStream is) throws Exception {
+        Map<String, String> baseTags = new HashMap<>();
+        baseTags.put(SparkJobTagName.SITE.toString(), site);
+        baseTags.put(SparkJobTagName.SPARK_QUEUE.toString(), app.getQueue());
+        JHFParserBase parser = new JHFSparkParser(new JHFSparkEventReader(baseTags, this.app));
+        parser.parse(is);
+    }
+
+    public static void main(String[] args) throws Exception {
+        SparkFilesystemInputStreamReaderImpl impl = new SparkFilesystemInputStreamReaderImpl("apollo-phx", new SparkApplicationInfo());
+        impl.read(new FileInputStream(new File("E:\\eagle\\application_1459803563374_535667_1")));
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/3110c72e/eagle-jpm/eagle-jpm-spark-history/src/main/java/org/apache/eagle/jpm/spark/history/status/JobHistoryZKStateManager.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-spark-history/src/main/java/org/apache/eagle/jpm/spark/history/status/JobHistoryZKStateManager.java b/eagle-jpm/eagle-jpm-spark-history/src/main/java/org/apache/eagle/jpm/spark/history/status/JobHistoryZKStateManager.java
index 9fafc1f..0bb65df 100644
--- a/eagle-jpm/eagle-jpm-spark-history/src/main/java/org/apache/eagle/jpm/spark/history/status/JobHistoryZKStateManager.java
+++ b/eagle-jpm/eagle-jpm-spark-history/src/main/java/org/apache/eagle/jpm/spark/history/status/JobHistoryZKStateManager.java
@@ -19,7 +19,7 @@
 
 package org.apache.eagle.jpm.spark.history.status;
 
-import org.apache.eagle.jpm.spark.crawl.SparkApplicationInfo;
+import org.apache.eagle.jpm.spark.history.crawl.SparkApplicationInfo;
 import org.apache.eagle.jpm.spark.history.SparkHistoryJobAppConfig;
 import org.apache.curator.framework.CuratorFramework;
 import org.apache.curator.framework.CuratorFrameworkFactory;
@@ -233,10 +233,11 @@ public class JobHistoryZKStateManager {
                     curator.setData().forPath(path, status.toString().getBytes("UTF-8"));
                 }
             } else {
-                LOG.error("Failed to update for application with path: " + path);
+                LOG.warn("failed to update with status {} due to path {} not existing ", status, path);
+                //throw new RuntimeException("Failed to update for application with path: " + path);
             }
         } catch (Exception e) {
-            LOG.error("fail to update application status", e);
+            LOG.error("fail to update application status as {}", status, e);
             throw new RuntimeException(e);
         } finally {
             try {

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/3110c72e/eagle-jpm/eagle-jpm-spark-history/src/main/java/org/apache/eagle/jpm/spark/history/storm/SparkHistoryJobParseBolt.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-spark-history/src/main/java/org/apache/eagle/jpm/spark/history/storm/SparkHistoryJobParseBolt.java b/eagle-jpm/eagle-jpm-spark-history/src/main/java/org/apache/eagle/jpm/spark/history/storm/SparkHistoryJobParseBolt.java
index e88c62f..0351de3 100644
--- a/eagle-jpm/eagle-jpm-spark-history/src/main/java/org/apache/eagle/jpm/spark/history/storm/SparkHistoryJobParseBolt.java
+++ b/eagle-jpm/eagle-jpm-spark-history/src/main/java/org/apache/eagle/jpm/spark/history/storm/SparkHistoryJobParseBolt.java
@@ -19,9 +19,9 @@
 
 package org.apache.eagle.jpm.spark.history.storm;
 
-import org.apache.eagle.jpm.spark.crawl.JHFInputStreamReader;
-import org.apache.eagle.jpm.spark.crawl.SparkApplicationInfo;
-import org.apache.eagle.jpm.spark.crawl.SparkFilesystemInputStreamReaderImpl;
+import org.apache.eagle.jpm.spark.history.crawl.JHFInputStreamReader;
+import org.apache.eagle.jpm.spark.history.crawl.SparkApplicationInfo;
+import org.apache.eagle.jpm.spark.history.crawl.SparkFilesystemInputStreamReaderImpl;
 import org.apache.eagle.jpm.spark.history.SparkHistoryJobAppConfig;
 import org.apache.eagle.jpm.spark.history.status.JobHistoryZKStateManager;
 import org.apache.eagle.jpm.spark.history.status.ZKStateConstant;
@@ -106,9 +106,12 @@ public class SparkHistoryJobParseBolt extends BaseRichBolt {
             zkState.updateApplicationStatus(appId, ZKStateConstant.AppStatus.FINISHED);
             LOG.info("Successfully parse application {}", appId);
             collector.ack(tuple);
+        } catch (RuntimeException e) {
+            LOG.warn("fail to process application {} due to RuntimeException, ignore it", appId, e);
+            zkState.updateApplicationStatus(appId, ZKStateConstant.AppStatus.FINISHED);
+            collector.ack(tuple);
         } catch (Exception e) {
-            LOG.error("Fail to process application {}", appId, e);
-            zkState.updateApplicationStatus(appId, ZKStateConstant.AppStatus.FAILED);
+            LOG.error("Fail to process application {}, and retry", appId, e);
             collector.fail(tuple);
         }
     }

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/3110c72e/eagle-jpm/eagle-jpm-spark-history/src/main/java/org/apache/eagle/jpm/spark/history/storm/SparkHistoryJobSpout.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-spark-history/src/main/java/org/apache/eagle/jpm/spark/history/storm/SparkHistoryJobSpout.java b/eagle-jpm/eagle-jpm-spark-history/src/main/java/org/apache/eagle/jpm/spark/history/storm/SparkHistoryJobSpout.java
index 5602b4c..4c50607 100644
--- a/eagle-jpm/eagle-jpm-spark-history/src/main/java/org/apache/eagle/jpm/spark/history/storm/SparkHistoryJobSpout.java
+++ b/eagle-jpm/eagle-jpm-spark-history/src/main/java/org/apache/eagle/jpm/spark/history/storm/SparkHistoryJobSpout.java
@@ -91,10 +91,11 @@ public class SparkHistoryJobSpout extends BaseRichSpout {
                 LOG.info("emit " + appId);
                 zkState.updateApplicationStatus(appId, ZKStateConstant.AppStatus.SENT_FOR_PARSE);
             }
-            LOG.info("{} apps sent.", appIds.size());
 
             if (appIds.isEmpty()) {
-                this.takeRest(60);
+                this.takeRest(10);
+            } else {
+                LOG.info("{} apps sent.", appIds.size());
             }
         } catch (Exception e) {
             LOG.error("Fail to run next tuple", e);

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/3110c72e/eagle-jpm/eagle-jpm-spark-history/src/main/resources/META-INF/providers/org.apache.eagle.jpm.spark.history.SparkHistoryJobAppProvider.xml
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-spark-history/src/main/resources/META-INF/providers/org.apache.eagle.jpm.spark.history.SparkHistoryJobAppProvider.xml b/eagle-jpm/eagle-jpm-spark-history/src/main/resources/META-INF/providers/org.apache.eagle.jpm.spark.history.SparkHistoryJobAppProvider.xml
index 26842b8..b94c603 100644
--- a/eagle-jpm/eagle-jpm-spark-history/src/main/resources/META-INF/providers/org.apache.eagle.jpm.spark.history.SparkHistoryJobAppProvider.xml
+++ b/eagle-jpm/eagle-jpm-spark-history/src/main/resources/META-INF/providers/org.apache.eagle.jpm.spark.history.SparkHistoryJobAppProvider.xml
@@ -153,24 +153,6 @@
             <value>http://sandbox.hortonworks.com:8088</value>
         </property>
         <property>
-            <name>storm.mode</name>
-            <displayName>mode</displayName>
-            <description>Storm Mode: local or cluster</description>
-            <value>local</value>
-        </property>
-        <property>
-            <name>storm.worker.num</name>
-            <displayName>worker.num</displayName>
-            <description>The number of workers</description>
-            <value>2</value>
-        </property>
-        <property>
-            <name>name</name>
-            <displayName>name</displayName>
-            <description>Name of the topology</description>
-            <value>sparkHistoryJob</value>
-        </property>
-        <property>
             <name>storm.messageTimeoutSec</name>
             <displayName>messageTimeoutSec</displayName>
             <description>Message timeout (in seconds)</description>

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/3110c72e/eagle-jpm/eagle-jpm-spark-history/src/main/resources/application.conf
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-spark-history/src/main/resources/application.conf b/eagle-jpm/eagle-jpm-spark-history/src/main/resources/application.conf
index 58dd552..4c22b15 100644
--- a/eagle-jpm/eagle-jpm-spark-history/src/main/resources/application.conf
+++ b/eagle-jpm/eagle-jpm-spark-history/src/main/resources/application.conf
@@ -15,6 +15,9 @@
 
 
 {
+  "appId": "sparkHistoryJob",
+  "mode": "CLUSTER",
+  "workers" : 3,
   "basic":{
     "cluster":"sandbox",
     "dataCenter":"sandbox",
@@ -45,8 +48,6 @@
     }
   },
   "storm":{
-    worker.num: 2,
-    "name":"sparkHistoryJob",
     "messageTimeoutSec": 3000,
     "pendingSpout": 1000,
     "spoutCrawlInterval": 10000,#in ms
@@ -72,7 +73,5 @@
       spark.yarn.am.memoryOverhead.factor: 10,
       spark.yarn.overhead.min: "384m"
     }
-  },
-  "appId": "sparkHistoryJob",
-  "mode": "CLUSTER"
+  }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/3110c72e/eagle-jpm/eagle-jpm-spark-running/src/main/java/org/apache/eagle/jpm/spark/running/entities/JPMEntityRepository.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-spark-running/src/main/java/org/apache/eagle/jpm/spark/running/entities/JPMEntityRepository.java b/eagle-jpm/eagle-jpm-spark-running/src/main/java/org/apache/eagle/jpm/spark/running/entities/JPMEntityRepository.java
deleted file mode 100644
index 5d1cfaa..0000000
--- a/eagle-jpm/eagle-jpm-spark-running/src/main/java/org/apache/eagle/jpm/spark/running/entities/JPMEntityRepository.java
+++ /dev/null
@@ -1,30 +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.eagle.jpm.spark.running.entities;
-
-import org.apache.eagle.log.entity.repo.EntityRepository;
-
-public class JPMEntityRepository extends EntityRepository {
-    public JPMEntityRepository() {
-        entitySet.add(SparkAppEntity.class);
-        entitySet.add(SparkJobEntity.class);
-        entitySet.add(SparkStageEntity.class);
-        entitySet.add(SparkTaskEntity.class);
-        entitySet.add(SparkExecutorEntity.class);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/3110c72e/eagle-jpm/eagle-jpm-spark-running/src/main/java/org/apache/eagle/jpm/spark/running/entities/JobConfig.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-spark-running/src/main/java/org/apache/eagle/jpm/spark/running/entities/JobConfig.java b/eagle-jpm/eagle-jpm-spark-running/src/main/java/org/apache/eagle/jpm/spark/running/entities/JobConfig.java
deleted file mode 100644
index e18f1e7..0000000
--- a/eagle-jpm/eagle-jpm-spark-running/src/main/java/org/apache/eagle/jpm/spark/running/entities/JobConfig.java
+++ /dev/null
@@ -1,25 +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.eagle.jpm.spark.running.entities;
-
-import java.io.Serializable;
-import java.util.HashMap;
-
-public class JobConfig extends HashMap<String, String> implements Serializable {
-}
-


[06/52] [abbrv] incubator-eagle git commit: [EAGLE-496] fix code style of jpm

Posted by yo...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/0b852cbc/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/HDFSUtil.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/HDFSUtil.java b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/HDFSUtil.java
index 325a92a..b263c25 100644
--- a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/HDFSUtil.java
+++ b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/HDFSUtil.java
@@ -33,12 +33,12 @@ public class HDFSUtil {
     }
 
     public static void login(Configuration kConfig) throws IOException {
-        if(kConfig.get("hdfs.kerberos.principal") == null || kConfig.get("hdfs.kerberos.principal").isEmpty()){
+        if (kConfig.get("hdfs.kerberos.principal") == null || kConfig.get("hdfs.kerberos.principal").isEmpty()) {
             return;
         }
-       kConfig.setBoolean("hadoop.security.authorization", true);
-       kConfig.set("hadoop.security.authentication", "kerberos");
-       UserGroupInformation.setConfiguration(kConfig);
-       UserGroupInformation.loginUserFromKeytab(kConfig.get("hdfs.kerberos.principal"), kConfig.get("hdfs.keytab.file"));
-     }
+        kConfig.setBoolean("hadoop.security.authorization", true);
+        kConfig.set("hadoop.security.authentication", "kerberos");
+        UserGroupInformation.setConfiguration(kConfig);
+        UserGroupInformation.loginUserFromKeytab(kConfig.get("hdfs.kerberos.principal"), kConfig.get("hdfs.keytab.file"));
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/0b852cbc/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/JSONUtil.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/JSONUtil.java b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/JSONUtil.java
index 8080147..9804a3b 100644
--- a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/JSONUtil.java
+++ b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/JSONUtil.java
@@ -29,37 +29,37 @@ public class JSONUtil {
         return null;
     }
 
-    public static Integer getInt(JSONObject obj, String field){
-        if(obj.containsKey(field)){
-            return ((Long)obj.get(field)).intValue();
+    public static Integer getInt(JSONObject obj, String field) {
+        if (obj.containsKey(field)) {
+            return ((Long) obj.get(field)).intValue();
         }
         return null;
     }
 
-    public static Long getLong(JSONObject obj, String field){
-        if(obj.containsKey(field)){
-            return (Long)obj.get(field);
+    public static Long getLong(JSONObject obj, String field) {
+        if (obj.containsKey(field)) {
+            return (Long) obj.get(field);
         }
         return null;
     }
 
-    public static Boolean getBoolean(JSONObject obj, String field){
-        if(obj.containsKey(field)){
-            return (Boolean)obj.get(field);
+    public static Boolean getBoolean(JSONObject obj, String field) {
+        if (obj.containsKey(field)) {
+            return (Boolean) obj.get(field);
         }
         return null;
     }
 
-    public static JSONObject getJSONObject(JSONObject obj, String field){
-        if(obj.containsKey(field)){
-            return (JSONObject)obj.get(field);
+    public static JSONObject getJSONObject(JSONObject obj, String field) {
+        if (obj.containsKey(field)) {
+            return (JSONObject) obj.get(field);
         }
         return null;
     }
 
-    public static JSONArray getJSONArray(JSONObject obj, String field){
-        if(obj.containsKey(field)){
-            return (JSONArray)obj.get(field);
+    public static JSONArray getJSONArray(JSONObject obj, String field) {
+        if (obj.containsKey(field)) {
+            return (JSONArray) obj.get(field);
         }
         return null;
     }

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/0b852cbc/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/JobNameNormalization.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/JobNameNormalization.java b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/JobNameNormalization.java
index c5cc82f..4e67f89 100644
--- a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/JobNameNormalization.java
+++ b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/JobNameNormalization.java
@@ -28,91 +28,95 @@ import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 
 public class JobNameNormalization {
-	private static Logger logger = LoggerFactory.getLogger(JobNameNormalization.class);
-	private static JobNameNormalization instance = new JobNameNormalization();
-	private static final String JOB_NAME_NORMALIZATION_RULES_KEY = "job.name.normalization.rules.key";
-	private static final String PARAMETERIZED_PREFIX = "\\$";
-	private static final String MULTIPLE_RULE_DILIMITER = ";";
-	/**
-	 * map from source string to target string
-	 * source string is regular expression, for example ^(.*)[0-9]{4}/[0-9]{2}/[0-9]{2}/[0-9]{2}$
-	 * target string is parameterized string, for example $1, $2
-	 */
-	private List<JobNameNormalizationRule> _rules = new ArrayList<JobNameNormalizationRule>();
-	
-	private enum NormalizationOp{
-		REPLACE("=>");
-		private String value;
-		private NormalizationOp(String value){
-			this.value = value;
-		}
-		public String toString(){
-			return value;
-		}
-	}
-	
-	static class JobNameNormalizationRule{
-		Pattern pattern;
-		NormalizationOp op;
-		String target;
-	}
-	
-	private JobNameNormalization(){
-		try{
-			// load normalization rules
-			Config conf = ConfigFactory.load();
-			String key = JOB_NAME_NORMALIZATION_RULES_KEY.toLowerCase();
-			String value = conf.getString(key);
-			if(value == null){
-				logger.info("no job name normalization rules are loaded");
-				return;
-			}
-			// multiple rules are concatenated with semicolon, i.e. ;
-			String rules[] = value.split(MULTIPLE_RULE_DILIMITER);
-			for(String rule : rules){
-				rule = rule.trim();
-				logger.info("jobNormalizationRule is loaded " + rule);
-				addRule(rule);
-			}
-		}catch(Exception ex){
-			logger.error("fail loading job name normalization rules", ex);
-			throw new RuntimeException(ex);
-		}
-	}
-	
-	public static JobNameNormalization getInstance(){
-		return instance;
-	}
-	
-	private void addRule(String rule){
-		for(NormalizationOp op : NormalizationOp.values()){
-			// split the rule to be source and target string
-			String elements[] = rule.split(op.toString());
-			if(elements == null || elements.length != 2) return;
-			JobNameNormalizationRule r = new JobNameNormalizationRule();
-			r.pattern = Pattern.compile(elements[0].trim());
-			r.op = op;
-			r.target = elements[1].trim();
-			_rules.add(r);
-			break;  //once one Op is matched, exit
-		}
-	
-	}
-	
-	public String normalize(String jobName){
-		String normalizedJobName = jobName;
-		// go through each rules and do actions
-		for(JobNameNormalizationRule rule : _rules){
-			Pattern p = rule.pattern;
-			Matcher m = p.matcher(jobName);
-			if(m.find()){
-				normalizedJobName = rule.target;
-				int c = m.groupCount();
-				for(int i=1; i<c+1; i++){
-					normalizedJobName = normalizedJobName.replaceAll(PARAMETERIZED_PREFIX+String.valueOf(i), m.group(i));
-				}
-			}
-		}
-		return normalizedJobName;
-	}
+    private static Logger LOG = LoggerFactory.getLogger(JobNameNormalization.class);
+    private static JobNameNormalization instance = new JobNameNormalization();
+    private static final String JOB_NAME_NORMALIZATION_RULES_KEY = "job.name.normalization.rules.key";
+    private static final String PARAMETERIZED_PREFIX = "\\$";
+    private static final String MULTIPLE_RULE_DILIMITER = ";";
+    /**
+     * map from source string to target string.
+     * source string is regular expression, for example ^(.*)[0-9]{4}/[0-9]{2}/[0-9]{2}/[0-9]{2}$
+     * target string is parameterized string, for example $1, $2
+     */
+    private List<JobNameNormalizationRule> _rules = new ArrayList<JobNameNormalizationRule>();
+
+    private enum NormalizationOp {
+        REPLACE("=>");
+        private String value;
+
+        private NormalizationOp(String value) {
+            this.value = value;
+        }
+
+        public String toString() {
+            return value;
+        }
+    }
+
+    static class JobNameNormalizationRule {
+        Pattern pattern;
+        NormalizationOp op;
+        String target;
+    }
+
+    private JobNameNormalization() {
+        try {
+            // load normalization rules
+            Config conf = ConfigFactory.load();
+            String key = JOB_NAME_NORMALIZATION_RULES_KEY.toLowerCase();
+            String value = conf.getString(key);
+            if (value == null) {
+                LOG.info("no job name normalization rules are loaded");
+                return;
+            }
+            // multiple rules are concatenated with semicolon, i.e. ;
+            String[] rules = value.split(MULTIPLE_RULE_DILIMITER);
+            for (String rule : rules) {
+                rule = rule.trim();
+                LOG.info("jobNormalizationRule is loaded " + rule);
+                addRule(rule);
+            }
+        } catch (Exception ex) {
+            LOG.error("fail loading job name normalization rules", ex);
+            throw new RuntimeException(ex);
+        }
+    }
+
+    public static JobNameNormalization getInstance() {
+        return instance;
+    }
+
+    private void addRule(String rule) {
+        for (NormalizationOp op : NormalizationOp.values()) {
+            // split the rule to be source and target string
+            String[] elements = rule.split(op.toString());
+            if (elements == null || elements.length != 2) {
+                return;
+            }
+            JobNameNormalizationRule r = new JobNameNormalizationRule();
+            r.pattern = Pattern.compile(elements[0].trim());
+            r.op = op;
+            r.target = elements[1].trim();
+            _rules.add(r);
+            break;  //once one Op is matched, exit
+        }
+
+    }
+
+    public String normalize(String jobName) {
+        String normalizedJobName = jobName;
+        // go through each rules and do actions
+        for (JobNameNormalizationRule rule : _rules) {
+            Pattern p = rule.pattern;
+            Matcher m = p.matcher(jobName);
+            if (m.find()) {
+                normalizedJobName = rule.target;
+                int c = m.groupCount();
+                for (int i = 1; i < c + 1; i++) {
+                    normalizedJobName = normalizedJobName.replaceAll(PARAMETERIZED_PREFIX + String.valueOf(i), m.group(i));
+                }
+            }
+        }
+        return normalizedJobName;
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/0b852cbc/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/MRJobTagName.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/MRJobTagName.java b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/MRJobTagName.java
index ea8e4f4..9811772 100644
--- a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/MRJobTagName.java
+++ b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/MRJobTagName.java
@@ -36,11 +36,12 @@ public enum MRJobTagName {
     RULE_TYPE("ruleType"),
     JOB_TYPE("jobType");
 
-    private String tagName; 
+    private String tagName;
+
     private MRJobTagName(String tagName) {
         this.tagName = tagName;
     }
-    
+
     public String toString() {
 
         return this.tagName;

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/0b852cbc/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/SparkEntityConstant.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/SparkEntityConstant.java b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/SparkEntityConstant.java
index 35014b1..2098747 100644
--- a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/SparkEntityConstant.java
+++ b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/SparkEntityConstant.java
@@ -19,11 +19,11 @@ package org.apache.eagle.jpm.util;
 
 public class SparkEntityConstant {
 
-    public enum SPARK_STAGE_STATUS{
+    public enum SparkStageStatus {
         COMPLETE, FAILED
     }
 
-    public enum SPARK_JOB_STATUS{
+    public enum SparkJobStatus {
         SUCCEEDED, FAILED
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/0b852cbc/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/SparkJobTagName.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/SparkJobTagName.java b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/SparkJobTagName.java
index 1d38eea..f403332 100644
--- a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/SparkJobTagName.java
+++ b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/SparkJobTagName.java
@@ -34,11 +34,12 @@ public enum SparkJobTagName {
 
 
     private String tagName;
-    private SparkJobTagName(String tagName){
+
+    private SparkJobTagName(String tagName) {
         this.tagName = tagName;
     }
 
-    public String toString(){
+    public String toString() {
         return this.tagName;
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/0b852cbc/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/Utils.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/Utils.java b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/Utils.java
index 12eb1b5..2696269 100644
--- a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/Utils.java
+++ b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/Utils.java
@@ -54,7 +54,7 @@ public class Utils {
             SimpleDateFormat dateFormat = new SimpleDateFormat("yyyy-MM-dd'T'hh:mm:ss.SSSzzz");
             Date parsedDate = dateFormat.parse(date);
             timestamp = parsedDate.getTime();
-        } catch(ParseException e) {
+        } catch (ParseException e) {
             e.printStackTrace();
         }
 
@@ -68,21 +68,21 @@ public class Utils {
     public static long parseMemory(String memory) {
         if (memory.endsWith("g") || memory.endsWith("G")) {
             int executorGB = Integer.parseInt(memory.substring(0, memory.length() - 1));
-            return 1024l * 1024 * 1024 * executorGB;
+            return 1024L * 1024 * 1024 * executorGB;
         } else if (memory.endsWith("m") || memory.endsWith("M")) {
             int executorMB = Integer.parseInt(memory.substring(0, memory.length() - 1));
-            return 1024l * 1024 * executorMB;
+            return 1024L * 1024 * executorMB;
         } else if (memory.endsWith("k") || memory.endsWith("K")) {
             int executorKB = Integer.parseInt(memory.substring(0, memory.length() - 1));
-            return 1024l * executorKB;
+            return 1024L * executorKB;
         } else if (memory.endsWith("t") || memory.endsWith("T")) {
             int executorTB = Integer.parseInt(memory.substring(0, memory.length() - 1));
-            return 1024l * 1024 * 1024 * 1024 * executorTB;
+            return 1024L * 1024 * 1024 * 1024 * executorTB;
         } else if (memory.endsWith("p") || memory.endsWith("P")) {
             int executorPB = Integer.parseInt(memory.substring(0, memory.length() - 1));
-            return 1024l * 1024 * 1024 * 1024 * 1024 * executorPB;
+            return 1024L * 1024 * 1024 * 1024 * 1024 * executorPB;
         }
-        LOG.info("Cannot parse memory info " +  memory);
-        return 0l;
+        LOG.info("Cannot parse memory info " + memory);
+        return 0L;
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/0b852cbc/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/jobcounter/CounterGroupDictionary.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/jobcounter/CounterGroupDictionary.java b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/jobcounter/CounterGroupDictionary.java
index c8572e9..5a29be2 100644
--- a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/jobcounter/CounterGroupDictionary.java
+++ b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/jobcounter/CounterGroupDictionary.java
@@ -86,7 +86,7 @@ public final class CounterGroupDictionary {
             final Properties prop = new Properties();
             try {
                 prop.load(is);
-            } catch(Exception ex) {
+            } catch (Exception ex) {
                 final String errMsg = "Failed to load JobCounter.conf, reason: " + ex.getMessage();
                 LOG.error(errMsg, ex);
                 throw new JobCounterException(errMsg, ex);
@@ -100,6 +100,7 @@ public final class CounterGroupDictionary {
                 try {
                     is.close();
                 } catch (IOException e) {
+                    LOG.warn("{}", e);
                 }
             }
         }
@@ -162,18 +163,22 @@ public final class CounterGroupDictionary {
             this.description = description;
             this.groupKey = groupKey;
         }
+
         @Override
         public int getIndex() {
             return index;
         }
+
         @Override
         public List<String> getNames() {
             return counterNames;
         }
+
         @Override
         public String getDescription() {
             return description;
         }
+
         @Override
         public CounterGroupKey getGroupKey() {
             return groupKey;
@@ -200,22 +205,27 @@ public final class CounterGroupDictionary {
         public int getIndex() {
             return index;
         }
+
         @Override
         public String getName() {
             return name;
         }
+
         @Override
         public String getDescription() {
             return description;
         }
+
         @Override
         public int getCounterNumber() {
             return counterKeys.length;
         }
+
         @Override
         public List<CounterKey> listCounterKeys() {
             return Arrays.asList(counterKeys);
         }
+
         @Override
         public CounterKey getCounterKeyByName(String name) {
             for (CounterKey counterKey : counterKeys) {
@@ -227,6 +237,7 @@ public final class CounterGroupDictionary {
             }
             return null;
         }
+
         @Override
         public CounterKey getCounterKeyByID(int index) {
             if (index < 0 || index >= counterKeys.length) {

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/0b852cbc/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/jobcounter/CounterGroupKey.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/jobcounter/CounterGroupKey.java b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/jobcounter/CounterGroupKey.java
index 482623a..54cc82f 100644
--- a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/jobcounter/CounterGroupKey.java
+++ b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/jobcounter/CounterGroupKey.java
@@ -21,12 +21,17 @@ package org.apache.eagle.jpm.util.jobcounter;
 import java.util.List;
 
 public interface CounterGroupKey {
-
     String getName();
+
     String getDescription();
+
     int getIndex();
+
     int getCounterNumber();
+
     List<CounterKey> listCounterKeys();
+
     CounterKey getCounterKeyByName(String name);
+
     CounterKey getCounterKeyByID(int index);
 }

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/0b852cbc/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/jobcounter/CounterKey.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/jobcounter/CounterKey.java b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/jobcounter/CounterKey.java
index 8e4e519..7e8be35 100644
--- a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/jobcounter/CounterKey.java
+++ b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/jobcounter/CounterKey.java
@@ -21,10 +21,11 @@ package org.apache.eagle.jpm.util.jobcounter;
 import java.util.List;
 
 public interface CounterKey {
-
     List<String> getNames();
+
     String getDescription();
+
     int getIndex();
+
     CounterGroupKey getGroupKey();
-    
 }

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/0b852cbc/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/jobcounter/JobCounterException.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/jobcounter/JobCounterException.java b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/jobcounter/JobCounterException.java
index e60e1d4..5cab7c8 100644
--- a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/jobcounter/JobCounterException.java
+++ b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/jobcounter/JobCounterException.java
@@ -19,21 +19,17 @@
 package org.apache.eagle.jpm.util.jobcounter;
 
 public class JobCounterException extends Exception {
-
-    /**
-     * 
-     */
     private static final long serialVersionUID = -4525162176188266862L;
 
     /**
-     * Default constructor of JobCounterException
+     * Default constructor of JobCounterException.
      */
     public JobCounterException() {
         super();
     }
 
     /**
-     * Constructor of JobCounterException
+     * Constructor of JobCounterException.
      * 
      * @param message error message
      */
@@ -42,7 +38,7 @@ public class JobCounterException extends Exception {
     }
 
     /**
-     * Constructor of JobCounterException
+     * Constructor of JobCounterException.
      * 
      * @param message error message
      * @param cause the cause of the exception
@@ -53,7 +49,7 @@ public class JobCounterException extends Exception {
     }
 
     /**
-     * Constructor of JobCounterException
+     * Constructor of JobCounterException.
      * 
      * @param cause the cause of the exception
      */

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/0b852cbc/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/jobcounter/JobCounters.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/jobcounter/JobCounters.java b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/jobcounter/JobCounters.java
index 5ecda8c..b4bf600 100644
--- a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/jobcounter/JobCounters.java
+++ b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/jobcounter/JobCounters.java
@@ -35,7 +35,7 @@ public final class JobCounters implements Serializable {
         this.counters = counters;
     }
     
-    public String toString(){
+    public String toString() {
         return counters.toString();
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/0b852cbc/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/jobcounter/JobCountersSerDeser.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/jobcounter/JobCountersSerDeser.java b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/jobcounter/JobCountersSerDeser.java
index 89b9edd..5c485ff 100755
--- a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/jobcounter/JobCountersSerDeser.java
+++ b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/jobcounter/JobCountersSerDeser.java
@@ -37,7 +37,7 @@ public class JobCountersSerDeser implements EntitySerDeser<JobCounters> {
             return counters;
         }
 
-        final Map<String, Map<String, Long> > groupMap = counters.getCounters();
+        final Map<String, Map<String, Long>> groupMap = counters.getCounters();
         int pos = 0;
         final int totalGroups = Bytes.toInt(bytes, pos);
         pos += 4;

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/0b852cbc/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/jobrecover/RunningJobManager.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/jobrecover/RunningJobManager.java b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/jobrecover/RunningJobManager.java
index 81e2886..2d1af2c 100644
--- a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/jobrecover/RunningJobManager.java
+++ b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/jobrecover/RunningJobManager.java
@@ -18,11 +18,11 @@
 
 package org.apache.eagle.jpm.util.jobrecover;
 
+import org.apache.eagle.jpm.util.resourcefetch.model.AppInfo;
 import org.apache.commons.lang3.tuple.Pair;
 import org.apache.curator.framework.CuratorFramework;
 import org.apache.curator.framework.CuratorFrameworkFactory;
 import org.apache.curator.retry.RetryNTimes;
-import org.apache.eagle.jpm.util.resourceFetch.model.AppInfo;
 import org.apache.zookeeper.CreateMode;
 import org.codehaus.jettison.json.JSONException;
 import org.codehaus.jettison.json.JSONObject;
@@ -36,9 +36,9 @@ public class RunningJobManager implements Serializable {
     public static final Logger LOG = LoggerFactory.getLogger(RunningJobManager.class);
     private String zkRoot;
     private CuratorFramework curator;
-    private final static String ENTITY_TAGS_KEY = "entityTags";
-    private final static String APP_INFO_KEY = "appInfo";
-    private final static String ZNODE_LAST_FINISH_TIME = "lastFinishTime";
+    private static final String ENTITY_TAGS_KEY = "entityTags";
+    private static final String APP_INFO_KEY = "appInfo";
+    private static final String ZNODE_LAST_FINISH_TIME = "lastFinishTime";
 
     private CuratorFramework newCurator(String zkQuorum, int zkSessionTimeoutMs, int zkRetryTimes, int zkRetryInterval) throws Exception {
         return CuratorFrameworkFactory.newClient(
@@ -62,6 +62,7 @@ public class RunningJobManager implements Serializable {
                         .forPath(this.zkRoot);
             }
         } catch (Exception e) {
+            LOG.warn("{}", e);
         }
     }
 
@@ -85,7 +86,6 @@ public class RunningJobManager implements Serializable {
             }
             JSONObject object = new JSONObject(fields);
             Map<String, Map<String, String>> parseResult = parse(object);
-            Map<String, String> tags = parseResult.get(ENTITY_TAGS_KEY);
 
             Map<String, String> appInfoMap = parseResult.get(APP_INFO_KEY);
             AppInfo appInfo = new AppInfo();
@@ -110,6 +110,7 @@ public class RunningJobManager implements Serializable {
             appInfo.setAllocatedVCores(Integer.parseInt(appInfoMap.get("allocatedVCores")));
             appInfo.setRunningContainers(Integer.parseInt(appInfoMap.get("runningContainers")));
 
+            Map<String, String> tags = parseResult.get(ENTITY_TAGS_KEY);
             result.put(jobId, Pair.of(tags, appInfo));
         }
         return result;
@@ -142,7 +143,6 @@ public class RunningJobManager implements Serializable {
     public boolean update(String yarnAppId, String jobId, Map<String, String> tags, AppInfo app) {
         String path = this.zkRoot + "/" + yarnAppId + "/" + jobId;
         //InterProcessMutex lock = new InterProcessMutex(curator, path);
-        Map<String, String> fields = new HashMap<>();
         Map<String, String> appInfo = new HashMap<>();
         appInfo.put("id", app.getId());
         appInfo.put("user", app.getUser());
@@ -165,6 +165,7 @@ public class RunningJobManager implements Serializable {
         appInfo.put("allocatedVCores", app.getAllocatedVCores() + "");
         appInfo.put("runningContainers", app.getRunningContainers() + "");
 
+        Map<String, String> fields = new HashMap<>();
         fields.put(ENTITY_TAGS_KEY, (new JSONObject(tags)).toString());
         fields.put(APP_INFO_KEY, (new JSONObject(appInfo)).toString());
         try {
@@ -263,7 +264,7 @@ public class RunningJobManager implements Serializable {
             LOG.error("failed to recover last finish time {}", e);
         }
 
-        return 0l;
+        return 0L;
     }
 
     public void updateLastFinishTime(int partitionId, Long lastFinishTime) {

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/0b852cbc/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/RMResourceFetcher.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/RMResourceFetcher.java b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/RMResourceFetcher.java
index 2b49f9f..eb13c3c 100644
--- a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/RMResourceFetcher.java
+++ b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/RMResourceFetcher.java
@@ -17,21 +17,20 @@
 /**
  *
  */
-package org.apache.eagle.jpm.util.resourceFetch;
+package org.apache.eagle.jpm.util.resourcefetch;
 
 import org.apache.eagle.jpm.util.Constants;
-import org.apache.eagle.jpm.util.resourceFetch.connection.InputStreamUtils;
-import org.apache.eagle.jpm.util.resourceFetch.ha.HAURLSelector;
-import org.apache.eagle.jpm.util.resourceFetch.ha.HAURLSelectorImpl;
-import org.apache.eagle.jpm.util.resourceFetch.model.AppInfo;
-import org.apache.eagle.jpm.util.resourceFetch.model.AppsWrapper;
-import org.apache.eagle.jpm.util.resourceFetch.model.ClusterInfo;
-import org.apache.eagle.jpm.util.resourceFetch.model.ClusterInfoWrapper;
-import org.apache.eagle.jpm.util.resourceFetch.url.JobListServiceURLBuilderImpl;
-import org.apache.eagle.jpm.util.resourceFetch.url.ServiceURLBuilder;
-import org.apache.eagle.jpm.util.resourceFetch.url.SparkCompleteJobServiceURLBuilderImpl;
-import org.apache.eagle.jpm.util.resourceFetch.url.URLUtil;
-import org.apache.hadoop.mapreduce.Cluster;
+import org.apache.eagle.jpm.util.resourcefetch.connection.InputStreamUtils;
+import org.apache.eagle.jpm.util.resourcefetch.ha.HAURLSelector;
+import org.apache.eagle.jpm.util.resourcefetch.ha.HAURLSelectorImpl;
+import org.apache.eagle.jpm.util.resourcefetch.model.AppInfo;
+import org.apache.eagle.jpm.util.resourcefetch.model.AppsWrapper;
+import org.apache.eagle.jpm.util.resourcefetch.model.ClusterInfo;
+import org.apache.eagle.jpm.util.resourcefetch.model.ClusterInfoWrapper;
+import org.apache.eagle.jpm.util.resourcefetch.url.JobListServiceURLBuilderImpl;
+import org.apache.eagle.jpm.util.resourcefetch.url.ServiceURLBuilder;
+import org.apache.eagle.jpm.util.resourcefetch.url.SparkCompleteJobServiceURLBuilderImpl;
+import org.apache.eagle.jpm.util.resourcefetch.url.URLUtil;
 import org.codehaus.jackson.JsonParser;
 import org.codehaus.jackson.map.ObjectMapper;
 import org.slf4j.Logger;
@@ -43,135 +42,153 @@ import java.util.List;
 
 public class RMResourceFetcher implements ResourceFetcher<AppInfo> {
 
-	private static final Logger LOG = LoggerFactory.getLogger(RMResourceFetcher.class);
-	private final HAURLSelector selector;
-	private final ServiceURLBuilder jobListServiceURLBuilder;
-	private final ServiceURLBuilder sparkCompleteJobServiceURLBuilder;
-	private static final ObjectMapper OBJ_MAPPER = new ObjectMapper();
-
-	static {
-		OBJ_MAPPER.configure(JsonParser.Feature.ALLOW_NON_NUMERIC_NUMBERS, true);
-	}
-
-	public RMResourceFetcher(String[] RMBasePaths) {
-		this.jobListServiceURLBuilder = new JobListServiceURLBuilderImpl();
-		this.sparkCompleteJobServiceURLBuilder = new SparkCompleteJobServiceURLBuilderImpl();
-
-		this.selector = new HAURLSelectorImpl(RMBasePaths, jobListServiceURLBuilder, Constants.CompressionType.GZIP);
-	}
-
-	private void checkUrl() throws IOException {
-		if (!selector.checkUrl(jobListServiceURLBuilder.build(selector.getSelectedUrl(), Constants.JobState.RUNNING.name()))) {
-			selector.reSelectUrl();
-		}
-	}
-
-	private List<AppInfo> doFetchFinishApplicationsList(String urlString, Constants.CompressionType compressionType) throws Exception {
-		List<AppInfo> result;
-		InputStream is = null;
-		try {
-			checkUrl();
-			LOG.info("Going to call yarn api to fetch finished application list: " + urlString);
-			is = InputStreamUtils.getInputStream(urlString, null, compressionType);
-			final AppsWrapper appWrapper = OBJ_MAPPER.readValue(is, AppsWrapper.class);
-			if (appWrapper != null && appWrapper.getApps() != null
-					&& appWrapper.getApps().getApp() != null) {
-				result = appWrapper.getApps().getApp();
-				return result;
-			}
-			return null;
-		} finally {
-			if (is != null) { try { is.close();} catch (Exception e) { } }
-		}
-	}
-
-	private String getSparkRunningJobURL() {
-		StringBuilder sb = new StringBuilder();
-		sb.append(selector.getSelectedUrl()).append("/").append(Constants.V2_APPS_URL);
-		sb.append("?applicationTypes=SPARK&state=RUNNING&");
-		sb.append(Constants.ANONYMOUS_PARAMETER);
-		return sb.toString();
-	}
-
-	private String getMRRunningJobURL() {
-		return String.format("%s/%s?applicationTypes=MAPREDUCE&state=RUNNING&%s",
-				selector.getSelectedUrl(),
-				Constants.V2_APPS_URL,
-				Constants.ANONYMOUS_PARAMETER);
-	}
-
-	public String getMRFinishedJobURL(String lastFinishedTime) {
-		String url = URLUtil.removeTrailingSlash(selector.getSelectedUrl());
-		StringBuilder sb = new StringBuilder();
-		sb.append(url).append("/").append(Constants.V2_APPS_URL);
-		sb.append("?applicationTypes=MAPREDUCE&state=FINISHED&finishedTimeBegin=");
-		sb.append(lastFinishedTime).append("&").append(Constants.ANONYMOUS_PARAMETER);
-
-		return sb.toString();
-	}
-
-	private List<AppInfo> doFetchRunningApplicationsList(String urlString, Constants.CompressionType compressionType) throws Exception {
-		List<AppInfo> result;
-		InputStream is = null;
-		try {
-			checkUrl();
-			LOG.info("Going to call yarn api to fetch running application list: " + urlString);
-			is = InputStreamUtils.getInputStream(urlString, null, compressionType);
-			final AppsWrapper appWrapper = OBJ_MAPPER.readValue(is, AppsWrapper.class);
-			if (appWrapper != null && appWrapper.getApps() != null && appWrapper.getApps().getApp() != null) {
-				result = appWrapper.getApps().getApp();
-				return result;
-			}
-			return null;
-		} finally {
-			if (is != null)  { try { is.close();} catch (Exception e) { } }
-		}
-	}
-
-	private List<AppInfo> getResource(Constants.ResourceType resoureType, Constants.CompressionType compressionType, Object... parameter) throws Exception {
-		switch (resoureType) {
-			case COMPLETE_SPARK_JOB:
-				final String urlString = sparkCompleteJobServiceURLBuilder.build((String) parameter[0]);
-				return doFetchFinishApplicationsList(urlString, compressionType);
-			case RUNNING_SPARK_JOB:
-				return doFetchRunningApplicationsList(getSparkRunningJobURL(), compressionType);
-			case RUNNING_MR_JOB:
-				return doFetchRunningApplicationsList(getMRRunningJobURL(), compressionType);
-			case COMPLETE_MR_JOB:
-				return doFetchFinishApplicationsList(getMRFinishedJobURL((String) parameter[0]), compressionType);
-			default:
-				throw new Exception("Not support resourceType :" + resoureType);
-		}
-	}
-
-	public List<AppInfo> getResource(Constants.ResourceType resoureType, Object... parameter) throws Exception {
-		try {
-			return getResource(resoureType, Constants.CompressionType.GZIP, parameter);
-		} catch (java.util.zip.ZipException ex) {
-			return getResource(resoureType, Constants.CompressionType.NONE, parameter);
-		}
-	}
-
-	private String getClusterInfoURL() {
-		StringBuilder sb = new StringBuilder();
-		sb.append(selector.getSelectedUrl()).append("/").append(Constants.YARN_API_CLUSTER_INFO).append("?" + Constants.ANONYMOUS_PARAMETER);
-		return sb.toString();
-	}
-
-	public ClusterInfo getClusterInfo() throws Exception {
-		InputStream is = null;
-		try {
-			checkUrl();
-			final String urlString = getClusterInfoURL();
-			LOG.info("Calling yarn api to fetch cluster info: " + urlString);
-			is = InputStreamUtils.getInputStream(urlString, null, Constants.CompressionType.GZIP);
-			final ClusterInfoWrapper clusterInfoWrapper = OBJ_MAPPER.readValue(is, ClusterInfoWrapper.class);
-			if (clusterInfoWrapper != null && clusterInfoWrapper.getClusterInfo() != null) {
-				return clusterInfoWrapper.getClusterInfo();
-			}
-			return null;
-		} finally {
-			if (is != null)  { try { is.close();} catch (Exception e) { } }
-		}
-	}
+    private static final Logger LOG = LoggerFactory.getLogger(RMResourceFetcher.class);
+    private final HAURLSelector selector;
+    private final ServiceURLBuilder jobListServiceURLBuilder;
+    private final ServiceURLBuilder sparkCompleteJobServiceURLBuilder;
+    private static final ObjectMapper OBJ_MAPPER = new ObjectMapper();
+
+    static {
+        OBJ_MAPPER.configure(JsonParser.Feature.ALLOW_NON_NUMERIC_NUMBERS, true);
+    }
+
+    public RMResourceFetcher(String[] rmBasePaths) {
+        this.jobListServiceURLBuilder = new JobListServiceURLBuilderImpl();
+        this.sparkCompleteJobServiceURLBuilder = new SparkCompleteJobServiceURLBuilderImpl();
+
+        this.selector = new HAURLSelectorImpl(rmBasePaths, jobListServiceURLBuilder, Constants.CompressionType.GZIP);
+    }
+
+    private void checkUrl() throws IOException {
+        if (!selector.checkUrl(jobListServiceURLBuilder.build(selector.getSelectedUrl(), Constants.JobState.RUNNING.name()))) {
+            selector.reSelectUrl();
+        }
+    }
+
+    private List<AppInfo> doFetchFinishApplicationsList(String urlString, Constants.CompressionType compressionType) throws Exception {
+        List<AppInfo> result;
+        InputStream is = null;
+        try {
+            checkUrl();
+            LOG.info("Going to call yarn api to fetch finished application list: " + urlString);
+            is = InputStreamUtils.getInputStream(urlString, null, compressionType);
+            final AppsWrapper appWrapper = OBJ_MAPPER.readValue(is, AppsWrapper.class);
+            if (appWrapper != null && appWrapper.getApps() != null
+                && appWrapper.getApps().getApp() != null) {
+                result = appWrapper.getApps().getApp();
+                return result;
+            }
+            return null;
+        } finally {
+            if (is != null) {
+                try {
+                    is.close();
+                } catch (Exception e) {
+                    LOG.warn("{}", e);
+                }
+            }
+        }
+    }
+
+    private String getSparkRunningJobURL() {
+        StringBuilder sb = new StringBuilder();
+        sb.append(selector.getSelectedUrl()).append("/").append(Constants.V2_APPS_URL);
+        sb.append("?applicationTypes=SPARK&state=RUNNING&");
+        sb.append(Constants.ANONYMOUS_PARAMETER);
+        return sb.toString();
+    }
+
+    private String getMRRunningJobURL() {
+        return String.format("%s/%s?applicationTypes=MAPREDUCE&state=RUNNING&%s",
+            selector.getSelectedUrl(),
+            Constants.V2_APPS_URL,
+            Constants.ANONYMOUS_PARAMETER);
+    }
+
+    public String getMRFinishedJobURL(String lastFinishedTime) {
+        String url = URLUtil.removeTrailingSlash(selector.getSelectedUrl());
+        StringBuilder sb = new StringBuilder();
+        sb.append(url).append("/").append(Constants.V2_APPS_URL);
+        sb.append("?applicationTypes=MAPREDUCE&state=FINISHED&finishedTimeBegin=");
+        sb.append(lastFinishedTime).append("&").append(Constants.ANONYMOUS_PARAMETER);
+
+        return sb.toString();
+    }
+
+    private List<AppInfo> doFetchRunningApplicationsList(String urlString, Constants.CompressionType compressionType) throws Exception {
+        List<AppInfo> result;
+        InputStream is = null;
+        try {
+            checkUrl();
+            LOG.info("Going to call yarn api to fetch running application list: " + urlString);
+            is = InputStreamUtils.getInputStream(urlString, null, compressionType);
+            final AppsWrapper appWrapper = OBJ_MAPPER.readValue(is, AppsWrapper.class);
+            if (appWrapper != null && appWrapper.getApps() != null && appWrapper.getApps().getApp() != null) {
+                result = appWrapper.getApps().getApp();
+                return result;
+            }
+            return null;
+        } finally {
+            if (is != null) {
+                try {
+                    is.close();
+                } catch (Exception e) {
+                    LOG.warn("{}", e);
+                }
+            }
+        }
+    }
+
+    private List<AppInfo> getResource(Constants.ResourceType resoureType, Constants.CompressionType compressionType, Object... parameter) throws Exception {
+        switch (resoureType) {
+            case COMPLETE_SPARK_JOB:
+                final String urlString = sparkCompleteJobServiceURLBuilder.build((String) parameter[0]);
+                return doFetchFinishApplicationsList(urlString, compressionType);
+            case RUNNING_SPARK_JOB:
+                return doFetchRunningApplicationsList(getSparkRunningJobURL(), compressionType);
+            case RUNNING_MR_JOB:
+                return doFetchRunningApplicationsList(getMRRunningJobURL(), compressionType);
+            case COMPLETE_MR_JOB:
+                return doFetchFinishApplicationsList(getMRFinishedJobURL((String) parameter[0]), compressionType);
+            default:
+                throw new Exception("Not support resourceType :" + resoureType);
+        }
+    }
+
+    public List<AppInfo> getResource(Constants.ResourceType resoureType, Object... parameter) throws Exception {
+        try {
+            return getResource(resoureType, Constants.CompressionType.GZIP, parameter);
+        } catch (java.util.zip.ZipException ex) {
+            return getResource(resoureType, Constants.CompressionType.NONE, parameter);
+        }
+    }
+
+    private String getClusterInfoURL() {
+        StringBuilder sb = new StringBuilder();
+        sb.append(selector.getSelectedUrl()).append("/").append(Constants.YARN_API_CLUSTER_INFO).append("?" + Constants.ANONYMOUS_PARAMETER);
+        return sb.toString();
+    }
+
+    public ClusterInfo getClusterInfo() throws Exception {
+        InputStream is = null;
+        try {
+            checkUrl();
+            final String urlString = getClusterInfoURL();
+            LOG.info("Calling yarn api to fetch cluster info: " + urlString);
+            is = InputStreamUtils.getInputStream(urlString, null, Constants.CompressionType.GZIP);
+            final ClusterInfoWrapper clusterInfoWrapper = OBJ_MAPPER.readValue(is, ClusterInfoWrapper.class);
+            if (clusterInfoWrapper != null && clusterInfoWrapper.getClusterInfo() != null) {
+                return clusterInfoWrapper.getClusterInfo();
+            }
+            return null;
+        } finally {
+            if (is != null) {
+                try {
+                    is.close();
+                } catch (Exception e) {
+                    LOG.warn("{}", e);
+                }
+            }
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/0b852cbc/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/ResourceFetcher.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/ResourceFetcher.java b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/ResourceFetcher.java
index cd475e7..f920ddb 100644
--- a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/ResourceFetcher.java
+++ b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/ResourceFetcher.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.eagle.jpm.util.resourceFetch;
+package org.apache.eagle.jpm.util.resourcefetch;
 
 import org.apache.eagle.jpm.util.Constants;
 
@@ -22,6 +22,6 @@ import java.util.List;
 
 public interface ResourceFetcher<T> {
     //continue to refactor later
-	List<T> getResource(Constants.ResourceType resoureType, Object... parameter) throws Exception;
+    List<T> getResource(Constants.ResourceType resoureType, Object... parameter) throws Exception;
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/0b852cbc/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/SparkHistoryServerResourceFetcher.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/SparkHistoryServerResourceFetcher.java b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/SparkHistoryServerResourceFetcher.java
index 689d2f3..ce2d9b8 100644
--- a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/SparkHistoryServerResourceFetcher.java
+++ b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/SparkHistoryServerResourceFetcher.java
@@ -17,13 +17,13 @@
  *
  */
 
-package org.apache.eagle.jpm.util.resourceFetch;
+package org.apache.eagle.jpm.util.resourcefetch;
 
 import org.apache.eagle.jpm.util.Constants;
-import org.apache.eagle.jpm.util.resourceFetch.connection.InputStreamUtils;
-import org.apache.eagle.jpm.util.resourceFetch.model.SparkApplication;
-import org.apache.eagle.jpm.util.resourceFetch.url.ServiceURLBuilder;
-import org.apache.eagle.jpm.util.resourceFetch.url.SparkJobServiceURLBuilderImpl;
+import org.apache.eagle.jpm.util.resourcefetch.connection.InputStreamUtils;
+import org.apache.eagle.jpm.util.resourcefetch.model.SparkApplication;
+import org.apache.eagle.jpm.util.resourcefetch.url.ServiceURLBuilder;
+import org.apache.eagle.jpm.util.resourcefetch.url.SparkJobServiceURLBuilderImpl;
 import org.apache.commons.codec.binary.Base64;
 import org.codehaus.jackson.JsonParser;
 import org.codehaus.jackson.map.ObjectMapper;
@@ -49,10 +49,11 @@ public class SparkHistoryServerResourceFetcher implements ResourceFetcher<SparkA
         OBJ_MAPPER.configure(JsonParser.Feature.ALLOW_NON_NUMERIC_NUMBERS, true);
     }
 
-    public SparkHistoryServerResourceFetcher(String historyServerURL, String userName, String pwd){
+    public SparkHistoryServerResourceFetcher(String historyServerURL, String userName, String pwd) {
         this.historyServerURL = historyServerURL;
         this.sparkDetailJobServiceURLBuilder = new SparkJobServiceURLBuilderImpl();
-        this.auth = "Basic " + new String(new Base64().encode(String.format("%s:%s", userName, pwd).getBytes()));;
+        this.auth = "Basic " + new String(new Base64().encode(String.format("%s:%s", userName, pwd).getBytes()));
+        ;
     }
 
     private List<SparkApplication> doFetchSparkApplicationDetail(String appId) throws Exception {
@@ -66,14 +67,20 @@ public class SparkHistoryServerResourceFetcher implements ResourceFetcher<SparkA
         } catch (FileNotFoundException e) {
             return null;
         } finally {
-            if (is != null) { try {is.close();} catch (Exception e) { } }
+            if (is != null) {
+                try {
+                    is.close();
+                } catch (Exception e) {
+                    LOG.warn("{}", e);
+                }
+            }
         }
     }
 
-    public List<SparkApplication> getResource(Constants.ResourceType resoureType, Object... parameter) throws Exception{
-        switch(resoureType) {
+    public List<SparkApplication> getResource(Constants.ResourceType resoureType, Object... parameter) throws Exception {
+        switch (resoureType) {
             case SPARK_JOB_DETAIL:
-                return doFetchSparkApplicationDetail((String)parameter[0]);
+                return doFetchSparkApplicationDetail((String) parameter[0]);
             default:
                 throw new Exception("Not support resourceType :" + resoureType);
         }

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/0b852cbc/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/connection/InputStreamUtils.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/connection/InputStreamUtils.java b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/connection/InputStreamUtils.java
index 6d3fa45..e99e2aa 100644
--- a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/connection/InputStreamUtils.java
+++ b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/connection/InputStreamUtils.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.eagle.jpm.util.resourceFetch.connection;
+package org.apache.eagle.jpm.util.resourcefetch.connection;
 
 import org.apache.eagle.jpm.util.Constants;
 
@@ -26,44 +26,43 @@ import java.util.zip.GZIPInputStream;
 
 public class InputStreamUtils {
 
-	private static final int CONNECTION_TIMEOUT = 10 * 1000;
-	private static final int READ_TIMEOUT = 5 * 60 * 1000;
-	private static final String GZIP_HTTP_HEADER = "Accept-Encoding";
-	private static final String GZIP_COMPRESSION = "gzip";
-	
-	private static InputStream openGZIPInputStream(URL url, String auth, int timeout) throws IOException {
-		final URLConnection connection = url.openConnection();
-		connection.setConnectTimeout(CONNECTION_TIMEOUT);
-		connection.setReadTimeout(timeout);
-		connection.addRequestProperty(GZIP_HTTP_HEADER, GZIP_COMPRESSION);
-		if (null != auth){
-			connection.setRequestProperty ("Authorization", auth);
-		}
-		return new GZIPInputStream(connection.getInputStream());
-	}
-	
-	private static InputStream openInputStream(URL url, String auth, int timeout) throws IOException {
-		URLConnection connection = url.openConnection();
-		connection.setConnectTimeout(timeout);
-		if (null != auth){
-			connection.setRequestProperty ("Authorization", auth);
-		}
+    private static final int CONNECTION_TIMEOUT = 10 * 1000;
+    private static final int READ_TIMEOUT = 5 * 60 * 1000;
+    private static final String GZIP_HTTP_HEADER = "Accept-Encoding";
+    private static final String GZIP_COMPRESSION = "gzip";
 
-		return connection.getInputStream();
-	}
-	
-	public static InputStream getInputStream(String urlString, String auth, Constants.CompressionType compressionType, int timeout) throws Exception {
-		final URL url = URLConnectionUtils.getUrl(urlString);
-		if (compressionType.equals(Constants.CompressionType.GZIP)) {
-			return openGZIPInputStream(url, auth, timeout);
-		}
-		else { // CompressionType.NONE
-			return openInputStream(url, auth, timeout);
-		}
-	}
-	
-	public static InputStream getInputStream(String urlString, String auth, Constants.CompressionType compressionType) throws Exception {
-		return getInputStream(urlString, auth, compressionType, READ_TIMEOUT);
-	}
+    private static InputStream openGZIPInputStream(URL url, String auth, int timeout) throws IOException {
+        final URLConnection connection = url.openConnection();
+        connection.setConnectTimeout(CONNECTION_TIMEOUT);
+        connection.setReadTimeout(timeout);
+        connection.addRequestProperty(GZIP_HTTP_HEADER, GZIP_COMPRESSION);
+        if (null != auth) {
+            connection.setRequestProperty("Authorization", auth);
+        }
+        return new GZIPInputStream(connection.getInputStream());
+    }
+
+    private static InputStream openInputStream(URL url, String auth, int timeout) throws IOException {
+        URLConnection connection = url.openConnection();
+        connection.setConnectTimeout(timeout);
+        if (null != auth) {
+            connection.setRequestProperty("Authorization", auth);
+        }
+
+        return connection.getInputStream();
+    }
+
+    public static InputStream getInputStream(String urlString, String auth, Constants.CompressionType compressionType, int timeout) throws Exception {
+        final URL url = URLConnectionUtils.getUrl(urlString);
+        if (compressionType.equals(Constants.CompressionType.GZIP)) {
+            return openGZIPInputStream(url, auth, timeout);
+        } else { // CompressionType.NONE
+            return openInputStream(url, auth, timeout);
+        }
+    }
+
+    public static InputStream getInputStream(String urlString, String auth, Constants.CompressionType compressionType) throws Exception {
+        return getInputStream(urlString, auth, compressionType, READ_TIMEOUT);
+    }
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/0b852cbc/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/connection/JobUtils.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/connection/JobUtils.java b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/connection/JobUtils.java
index 2e7b248..44d27e8 100644
--- a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/connection/JobUtils.java
+++ b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/connection/JobUtils.java
@@ -14,30 +14,30 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.eagle.jpm.util.resourceFetch.connection;
+package org.apache.eagle.jpm.util.resourcefetch.connection;
 
 import org.apache.eagle.jpm.util.Constants;
 
 public class JobUtils {
-	
-	public static String checkAndAddLastSlash(String urlBase) {
-		if (!urlBase.endsWith("/")) {
-			return urlBase + "/";
-		}
-		return urlBase;
-	}
-	
-	public static String getJobIDByAppID(String appID) {
-		if (appID.startsWith(Constants.APPLICATION_PREFIX)) {
-			return appID.replace(Constants.APPLICATION_PREFIX, Constants.JOB_PREFIX);
-		}
-		return null;
-	}
 
-	public static String getAppIDByJobID(String jobID) {
-		if (jobID.startsWith(Constants.JOB_PREFIX)) {
-			return jobID.replace(Constants.JOB_PREFIX, Constants.APPLICATION_PREFIX);
-		}
-		return null;
-	}
+    public static String checkAndAddLastSlash(String urlBase) {
+        if (!urlBase.endsWith("/")) {
+            return urlBase + "/";
+        }
+        return urlBase;
+    }
+
+    public static String getJobIDByAppID(String appID) {
+        if (appID.startsWith(Constants.APPLICATION_PREFIX)) {
+            return appID.replace(Constants.APPLICATION_PREFIX, Constants.JOB_PREFIX);
+        }
+        return null;
+    }
+
+    public static String getAppIDByJobID(String jobID) {
+        if (jobID.startsWith(Constants.JOB_PREFIX)) {
+            return jobID.replace(Constants.JOB_PREFIX, Constants.APPLICATION_PREFIX);
+        }
+        return null;
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/0b852cbc/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/connection/URLConnectionUtils.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/connection/URLConnectionUtils.java b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/connection/URLConnectionUtils.java
index d340d7b..2cd60ba 100644
--- a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/connection/URLConnectionUtils.java
+++ b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/connection/URLConnectionUtils.java
@@ -14,12 +14,10 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.eagle.jpm.util.resourceFetch.connection;
+package org.apache.eagle.jpm.util.resourcefetch.connection;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-
-import javax.net.ssl.*;
 import java.io.IOException;
 import java.net.MalformedURLException;
 import java.net.URL;
@@ -27,76 +25,77 @@ import java.net.URLConnection;
 import java.security.KeyManagementException;
 import java.security.NoSuchAlgorithmException;
 import java.security.cert.CertificateException;
+import javax.net.ssl.*;
 
 public final class URLConnectionUtils {
-	//TODO: change some public method to private
+    //TODO: change some public method to private
     private static final Logger LOG = LoggerFactory.getLogger(URLConnectionUtils.class);
-	
-	public static URLConnection getConnection(String url) throws Exception {
-		if (url.startsWith("https://")) {
-			return getHTTPSConnection(url);
-		} else if (url.startsWith("http://")) {
-			return getHTTPConnection(url);
-		}
-		throw new Exception("Invalid input argument url: " + url);
-	}
 
-	public static URLConnection getHTTPConnection(String urlString) throws Exception {
-		final URL url = new URL(urlString);
-		return url.openConnection();
-	}
+    public static URLConnection getConnection(String url) throws Exception {
+        if (url.startsWith("https://")) {
+            return getHTTPSConnection(url);
+        } else if (url.startsWith("http://")) {
+            return getHTTPConnection(url);
+        }
+        throw new Exception("Invalid input argument url: " + url);
+    }
+
+    public static URLConnection getHTTPConnection(String urlString) throws Exception {
+        final URL url = new URL(urlString);
+        return url.openConnection();
+    }
 
-	public static URL getUrl(String urlString) throws Exception  {
-		if(urlString.toLowerCase().contains("https")){
-			return getHTTPSUrl(urlString);
-		}else if (urlString.toLowerCase().contains("http")) {
-			return getURL(urlString);
-		}
-		throw new Exception("Invalid input argument url: " + urlString);
-	}
-	
-	public static URL getURL(String urlString) throws MalformedURLException {
-		return new URL(urlString);
-	}
-	
-	public static URL getHTTPSUrl(String urlString) throws MalformedURLException, NoSuchAlgorithmException, KeyManagementException  {
-    	// Create a trust manager that does not validate certificate chains   
+    public static URL getUrl(String urlString) throws Exception {
+        if (urlString.toLowerCase().contains("https")) {
+            return getHTTPSUrl(urlString);
+        } else if (urlString.toLowerCase().contains("http")) {
+            return getURL(urlString);
+        }
+        throw new Exception("Invalid input argument url: " + urlString);
+    }
+
+    public static URL getURL(String urlString) throws MalformedURLException {
+        return new URL(urlString);
+    }
+
+    public static URL getHTTPSUrl(String urlString) throws MalformedURLException, NoSuchAlgorithmException, KeyManagementException {
+        // Create a trust manager that does not validate certificate chains
         final TrustManager[] trustAllCerts = new TrustManager[] {new TrustAllX509TrustManager()};
         // Install the all-trusting trust manager   
-        final SSLContext sc = SSLContext.getInstance("SSL");   
-        sc.init(null, trustAllCerts, new java.security.SecureRandom());   
-        HttpsURLConnection.setDefaultSSLSocketFactory(sc.getSocketFactory());   
+        final SSLContext sc = SSLContext.getInstance("SSL");
+        sc.init(null, trustAllCerts, new java.security.SecureRandom());
+        HttpsURLConnection.setDefaultSSLSocketFactory(sc.getSocketFactory());
         // Create all-trusting host name verifier   
-        final HostnameVerifier allHostsValid = new HostnameVerifier() {   
-            public boolean verify(String hostname, SSLSession session) {   
-                return true;   
-            }   
+        final HostnameVerifier allHostsValid = new HostnameVerifier() {
+            public boolean verify(String hostname, SSLSession session) {
+                return true;
+            }
         };
         HttpsURLConnection.setDefaultHostnameVerifier(allHostsValid);
         return new URL(urlString);
-	}
+    }
+
+    public static URLConnection getHTTPSConnection(String urlString) throws IOException, KeyManagementException, NoSuchAlgorithmException {
+        final URL url = getHTTPSUrl(urlString);
+        return url.openConnection();
+    }
 
-	public static URLConnection getHTTPSConnection(String urlString) throws IOException, KeyManagementException, NoSuchAlgorithmException  {
-       	final URL url = getHTTPSUrl(urlString);
-       	return url.openConnection();
-	}
-	
-	public static class TrustAllX509TrustManager implements X509TrustManager {
-		@Override
-		public void checkClientTrusted(
-				java.security.cert.X509Certificate[] chain, String authType)
-				throws CertificateException {
-		}
+    public static class TrustAllX509TrustManager implements X509TrustManager {
+        @Override
+        public void checkClientTrusted(
+            java.security.cert.X509Certificate[] chain, String authType)
+            throws CertificateException {
+        }
 
-		@Override
-		public void checkServerTrusted(
-				java.security.cert.X509Certificate[] chain, String authType)
-				throws CertificateException {
-		}
+        @Override
+        public void checkServerTrusted(
+            java.security.cert.X509Certificate[] chain, String authType)
+            throws CertificateException {
+        }
 
-		@Override
-		public java.security.cert.X509Certificate[] getAcceptedIssuers() {
-			return null;
-		}
-	}
+        @Override
+        public java.security.cert.X509Certificate[] getAcceptedIssuers() {
+            return null;
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/0b852cbc/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/ha/AbstractURLSelector.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/ha/AbstractURLSelector.java b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/ha/AbstractURLSelector.java
index 57c2902..d25d05b 100644
--- a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/ha/AbstractURLSelector.java
+++ b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/ha/AbstractURLSelector.java
@@ -16,10 +16,10 @@
  *
  */
 
-package org.apache.eagle.jpm.util.resourceFetch.ha;
+package org.apache.eagle.jpm.util.resourcefetch.ha;
 
 import org.apache.eagle.jpm.util.Constants;
-import org.apache.eagle.jpm.util.resourceFetch.connection.InputStreamUtils;
+import org.apache.eagle.jpm.util.resourcefetch.connection.InputStreamUtils;
 import org.apache.hadoop.util.StringUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -51,7 +51,13 @@ public abstract class AbstractURLSelector implements HAURLSelector {
             LOG.info("get input stream from url: " + urlString + " failed. ");
             return false;
         } finally {
-            if (is != null) { try {	is.close(); } catch (IOException e) {/*Do nothing*/} }
+            if (is != null) {
+                try {
+                    is.close();
+                } catch (IOException e) {
+                    LOG.warn("{}", e);
+                }
+            }
         }
         return true;
     }
@@ -66,9 +72,13 @@ public abstract class AbstractURLSelector implements HAURLSelector {
 
     @Override
     public void reSelectUrl() throws IOException {
-        if (reselectInProgress) return;
-        synchronized(this) {
-            if (reselectInProgress) return;
+        if (reselectInProgress) {
+            return;
+        }
+        synchronized (this) {
+            if (reselectInProgress) {
+                return;
+            }
             reselectInProgress = true;
             try {
                 LOG.info("Going to reselect url");
@@ -81,16 +91,16 @@ public abstract class AbstractURLSelector implements HAURLSelector {
                             LOG.info("Successfully switch to new url : " + selectedUrl);
                             return;
                         }
-                        LOG.info("try url " + urlToCheck + "fail for " + (time+1) + " times, sleep 5 seconds before try again. ");
+                        LOG.info("try url " + urlToCheck + "fail for " + (time + 1) + " times, sleep 5 seconds before try again. ");
                         try {
                             Thread.sleep(5 * 1000);
+                        } catch (InterruptedException ex) {
+                            LOG.warn("{}", ex);
                         }
-                        catch (InterruptedException ex) { /* Do Nothing */}
                     }
                 }
-                throw new IOException("No alive url found: "+ StringUtils.join(";", Arrays.asList(this.urls)));
-            }
-            finally {
+                throw new IOException("No alive url found: " + StringUtils.join(";", Arrays.asList(this.urls)));
+            } finally {
                 reselectInProgress = false;
             }
         }

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/0b852cbc/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/ha/HAURLSelector.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/ha/HAURLSelector.java b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/ha/HAURLSelector.java
index 6eea7e3..fa9b52b 100644
--- a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/ha/HAURLSelector.java
+++ b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/ha/HAURLSelector.java
@@ -14,15 +14,15 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.eagle.jpm.util.resourceFetch.ha;
+package org.apache.eagle.jpm.util.resourcefetch.ha;
 
 import java.io.IOException;
 
 public interface HAURLSelector {
-	
-	boolean checkUrl(String url);
-		
-	void reSelectUrl() throws IOException;
-	
-	String getSelectedUrl();
+
+    boolean checkUrl(String url);
+
+    void reSelectUrl() throws IOException;
+
+    String getSelectedUrl();
 }

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/0b852cbc/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/ha/HAURLSelectorImpl.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/ha/HAURLSelectorImpl.java b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/ha/HAURLSelectorImpl.java
index 4052ed0..7c188c6 100644
--- a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/ha/HAURLSelectorImpl.java
+++ b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/ha/HAURLSelectorImpl.java
@@ -14,11 +14,11 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.eagle.jpm.util.resourceFetch.ha;
+package org.apache.eagle.jpm.util.resourcefetch.ha;
 
 import org.apache.eagle.jpm.util.Constants;
-import org.apache.eagle.jpm.util.resourceFetch.connection.InputStreamUtils;
-import org.apache.eagle.jpm.util.resourceFetch.url.ServiceURLBuilder;
+import org.apache.eagle.jpm.util.resourcefetch.connection.InputStreamUtils;
+import org.apache.eagle.jpm.util.resourcefetch.url.ServiceURLBuilder;
 import org.apache.hadoop.util.StringUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -29,73 +29,81 @@ import java.util.Arrays;
 
 public class HAURLSelectorImpl implements HAURLSelector {
 
-	private final String[] urls;
-	private volatile String selectedUrl;
-	private final ServiceURLBuilder builder;
-	
-	private volatile boolean reselectInProgress;
-	private final Constants.CompressionType compressionType;
-	private static final long MAX_RETRY_TIME = 2;
-	private static final Logger LOG = LoggerFactory.getLogger(HAURLSelectorImpl.class);
-	
-	public HAURLSelectorImpl(String[] urls, ServiceURLBuilder builder, Constants.CompressionType compressionType) {
-		this.urls = urls;
-		this.compressionType = compressionType;
-		this.builder = builder;
-	}
-	
-	public boolean checkUrl(String urlString) {
-		InputStream is = null;
-		try {
-			is = InputStreamUtils.getInputStream(urlString, null, compressionType);
-		}
-		catch (Exception ex) {
-			LOG.info("get inputstream from url: " + urlString + " failed. ");
-			return false;
-		}
-		finally {
-			if (is != null) { try {	is.close(); } catch (IOException e) {/*Do nothing*/} }
-		}
-		return true;
-	}
+    private final String[] urls;
+    private volatile String selectedUrl;
+    private final ServiceURLBuilder builder;
 
-	@Override
-	public String getSelectedUrl() {
-		if (selectedUrl == null) {
-			selectedUrl = urls[0];
-		}
-		return selectedUrl;
-	}
-	
-	@Override
-	public void reSelectUrl() throws IOException {
-		if (reselectInProgress) return;
-		synchronized(this) {
-			if (reselectInProgress) return;
-			reselectInProgress = true;
-			try {
-				LOG.info("Going to reselect url");
-				for (int i = 0; i < urls.length; i++) {		
-					String urlToCheck = urls[i];
-					LOG.info("Going to try url :" + urlToCheck);
-					for (int time = 0; time < MAX_RETRY_TIME; time++) {
-						if (checkUrl(builder.build(urlToCheck, Constants.JobState.RUNNING.name()))) {
-							selectedUrl = urls[i];
-							LOG.info("Successfully switch to new url : " + selectedUrl);
-							return;
-						}
-						LOG.info("try url " + urlToCheck + "fail for " + (time+1) + " times, sleep 5 seconds before try again. ");
-						try {
-							Thread.sleep(1 * 1000);
-						}
-						catch (InterruptedException ex) { /* Do Nothing */}
-					}
-				}
-				throw new IOException("No alive url found: "+ StringUtils.join(";", Arrays.asList(this.urls)));
-			}
-			finally {
-				reselectInProgress = false;
-			}
-		}
-	}
+    private volatile boolean reselectInProgress;
+    private final Constants.CompressionType compressionType;
+    private static final long MAX_RETRY_TIME = 2;
+    private static final Logger LOG = LoggerFactory.getLogger(HAURLSelectorImpl.class);
+
+    public HAURLSelectorImpl(String[] urls, ServiceURLBuilder builder, Constants.CompressionType compressionType) {
+        this.urls = urls;
+        this.compressionType = compressionType;
+        this.builder = builder;
+    }
+
+    public boolean checkUrl(String urlString) {
+        InputStream is = null;
+        try {
+            is = InputStreamUtils.getInputStream(urlString, null, compressionType);
+        } catch (Exception ex) {
+            LOG.info("get inputstream from url: " + urlString + " failed. ");
+            return false;
+        } finally {
+            if (is != null) {
+                try {
+                    is.close();
+                } catch (IOException e) {
+                    LOG.warn("{}", e);
+                }
+            }
+        }
+        return true;
+    }
+
+    @Override
+    public String getSelectedUrl() {
+        if (selectedUrl == null) {
+            selectedUrl = urls[0];
+        }
+        return selectedUrl;
+    }
+
+    @Override
+    public void reSelectUrl() throws IOException {
+        if (reselectInProgress) {
+            return;
+        }
+        synchronized (this) {
+            if (reselectInProgress) {
+                return;
+            }
+            reselectInProgress = true;
+            try {
+                LOG.info("Going to reselect url");
+                for (int i = 0; i < urls.length; i++) {
+                    String urlToCheck = urls[i];
+                    LOG.info("Going to try url :" + urlToCheck);
+                    for (int time = 0; time < MAX_RETRY_TIME; time++) {
+                        if (checkUrl(builder.build(urlToCheck, Constants.JobState.RUNNING.name()))) {
+                            selectedUrl = urls[i];
+                            LOG.info("Successfully switch to new url : " + selectedUrl);
+                            return;
+                        }
+                        LOG.info("try url " + urlToCheck + "fail for " + (time + 1) + " times, sleep 5 seconds before try again. ");
+                        try {
+                            Thread.sleep(1 * 1000);
+                        } catch (InterruptedException ex) {
+                            LOG.warn("{}", ex);
+                        }
+                    }
+                }
+                throw new IOException("No alive url found: " + StringUtils.join(";", Arrays.asList(this.urls)));
+            } finally {
+                reselectInProgress = false;
+            }
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/0b852cbc/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/AppInfo.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/AppInfo.java b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/AppInfo.java
index 5f13616..f769217 100644
--- a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/AppInfo.java
+++ b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/AppInfo.java
@@ -14,20 +14,20 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.eagle.jpm.util.resourceFetch.model;
+package org.apache.eagle.jpm.util.resourcefetch.model;
 
 import org.codehaus.jackson.annotate.JsonIgnoreProperties;
 import org.codehaus.jackson.map.annotate.JsonSerialize;
 
 import java.io.Serializable;
 
-@JsonSerialize(include=JsonSerialize.Inclusion.NON_NULL)
+@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
 @JsonIgnoreProperties(ignoreUnknown = true)
 public class AppInfo implements Serializable {
-	private static final long serialVersionUID = 1L;
-	
-	private String id;
-	private String user;
+    private static final long serialVersionUID = 1L;
+
+    private String id;
+    private String user;
     private String name;
     private String queue;
     private String state;
@@ -46,109 +46,142 @@ public class AppInfo implements Serializable {
     private long allocatedMB;
     private int allocatedVCores;
     private int runningContainers;
-	
-	public String getId() {
-		return id;
-	}
-	public void setId(String id) {
-		this.id = id;
-	}
-	public String getUser() {
-		return user;
-	}
-	public void setUser(String user) {
-		this.user = user;
-	}
-	public String getName() {
-		return name;
-	}
-	public void setName(String name) {
-		this.name = name;
-	}
-	public String getQueue() {
-		return queue;
-	}
-	public void setQueue(String queue) {
-		this.queue = queue;
-	}
-	public String getState() {
-		return state;
-	}
-	public void setState(String state) {
-		this.state = state;
-	}
-	public String getFinalStatus() {
-		return finalStatus;
-	}
-	public void setFinalStatus(String finalStatus) {
-		this.finalStatus = finalStatus;
-	}
-	public double getProgress() {
-		return progress;
-	}
-	public void setProgress(double progress) {
-		this.progress = progress;
-	}
-	public String getTrackingUI() {
-		return trackingUI;
-	}
-	public void setTrackingUI(String trackingUI) {
-		this.trackingUI = trackingUI;
-	}
-	public String getTrackingUrl() {
-		return trackingUrl;
-	}
-	public void setTrackingUrl(String trackingUrl) {
-		this.trackingUrl = trackingUrl;
-	}
-	public String getDiagnostics() {
-		return diagnostics;
-	}
-	public void setDiagnostics(String diagnostics) {
-		this.diagnostics = diagnostics;
-	}
-	public String getClusterId() {
-		return clusterId;
-	}
-	public void setClusterId(String clusterId) {
-		this.clusterId = clusterId;
-	}
-	public String getApplicationType() {
-		return applicationType;
-	}
-	public void setApplicationType(String applicationType) {
-		this.applicationType = applicationType;
-	}
-	public long getStartedTime() {
-		return startedTime;
-	}
-	public void setStartedTime(long startedTime) {
-		this.startedTime = startedTime;
-	}
-	public long getFinishedTime() {
-		return finishedTime;
-	}
-	public void setFinishedTime(long finishedTime) {
-		this.finishedTime = finishedTime;
-	}
-	public long getElapsedTime() {
-		return elapsedTime;
-	}
-	public void setElapsedTime(long elapsedTime) {
-		this.elapsedTime = elapsedTime;
-	}
-	public String getAmContainerLogs() {
-		return amContainerLogs;
-	}
-	public void setAmContainerLogs(String amContainerLogs) {
-		this.amContainerLogs = amContainerLogs;
-	}
-	public String getAmHostHttpAddress() {
-		return amHostHttpAddress;
-	}
-	public void setAmHostHttpAddress(String amHostHttpAddress) {
-		this.amHostHttpAddress = amHostHttpAddress;
-	}
+
+    public String getId() {
+        return id;
+    }
+
+    public void setId(String id) {
+        this.id = id;
+    }
+
+    public String getUser() {
+        return user;
+    }
+
+    public void setUser(String user) {
+        this.user = user;
+    }
+
+    public String getName() {
+        return name;
+    }
+
+    public void setName(String name) {
+        this.name = name;
+    }
+
+    public String getQueue() {
+        return queue;
+    }
+
+    public void setQueue(String queue) {
+        this.queue = queue;
+    }
+
+    public String getState() {
+        return state;
+    }
+
+    public void setState(String state) {
+        this.state = state;
+    }
+
+    public String getFinalStatus() {
+        return finalStatus;
+    }
+
+    public void setFinalStatus(String finalStatus) {
+        this.finalStatus = finalStatus;
+    }
+
+    public double getProgress() {
+        return progress;
+    }
+
+    public void setProgress(double progress) {
+        this.progress = progress;
+    }
+
+    public String getTrackingUI() {
+        return trackingUI;
+    }
+
+    public void setTrackingUI(String trackingUI) {
+        this.trackingUI = trackingUI;
+    }
+
+    public String getTrackingUrl() {
+        return trackingUrl;
+    }
+
+    public void setTrackingUrl(String trackingUrl) {
+        this.trackingUrl = trackingUrl;
+    }
+
+    public String getDiagnostics() {
+        return diagnostics;
+    }
+
+    public void setDiagnostics(String diagnostics) {
+        this.diagnostics = diagnostics;
+    }
+
+    public String getClusterId() {
+        return clusterId;
+    }
+
+    public void setClusterId(String clusterId) {
+        this.clusterId = clusterId;
+    }
+
+    public String getApplicationType() {
+        return applicationType;
+    }
+
+    public void setApplicationType(String applicationType) {
+        this.applicationType = applicationType;
+    }
+
+    public long getStartedTime() {
+        return startedTime;
+    }
+
+    public void setStartedTime(long startedTime) {
+        this.startedTime = startedTime;
+    }
+
+    public long getFinishedTime() {
+        return finishedTime;
+    }
+
+    public void setFinishedTime(long finishedTime) {
+        this.finishedTime = finishedTime;
+    }
+
+    public long getElapsedTime() {
+        return elapsedTime;
+    }
+
+    public void setElapsedTime(long elapsedTime) {
+        this.elapsedTime = elapsedTime;
+    }
+
+    public String getAmContainerLogs() {
+        return amContainerLogs;
+    }
+
+    public void setAmContainerLogs(String amContainerLogs) {
+        this.amContainerLogs = amContainerLogs;
+    }
+
+    public String getAmHostHttpAddress() {
+        return amHostHttpAddress;
+    }
+
+    public void setAmHostHttpAddress(String amHostHttpAddress) {
+        this.amHostHttpAddress = amHostHttpAddress;
+    }
 
     public long getAllocatedMB() {
         return allocatedMB;

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/0b852cbc/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/Applications.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/Applications.java b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/Applications.java
index 741fa1d..22ee2b9 100644
--- a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/Applications.java
+++ b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/Applications.java
@@ -14,25 +14,25 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.eagle.jpm.util.resourceFetch.model;
+package org.apache.eagle.jpm.util.resourcefetch.model;
 
 import org.codehaus.jackson.annotate.JsonIgnoreProperties;
 import org.codehaus.jackson.map.annotate.JsonSerialize;
 
 import java.util.List;
 
-@JsonSerialize(include=JsonSerialize.Inclusion.NON_NULL)
+@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
 @JsonIgnoreProperties(ignoreUnknown = true)
 public class Applications {
 
-	private List<AppInfo> app;
+    private List<AppInfo> app;
 
-	public List<AppInfo> getApp() {
-		return app;
-	}
+    public List<AppInfo> getApp() {
+        return app;
+    }
+
+    public void setApp(List<AppInfo> app) {
+        this.app = app;
+    }
 
-	public void setApp(List<AppInfo> app) {
-		this.app = app;
-	}
-	
 }

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/0b852cbc/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/AppsWrapper.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/AppsWrapper.java b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/AppsWrapper.java
index 42ff4f8..63e37eb 100644
--- a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/AppsWrapper.java
+++ b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/AppsWrapper.java
@@ -14,24 +14,24 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.eagle.jpm.util.resourceFetch.model;
+package org.apache.eagle.jpm.util.resourcefetch.model;
 
 import org.codehaus.jackson.annotate.JsonIgnoreProperties;
 import org.codehaus.jackson.map.annotate.JsonSerialize;
 
-@JsonSerialize(include=JsonSerialize.Inclusion.NON_NULL)
+@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
 @JsonIgnoreProperties(ignoreUnknown = true)
 public class AppsWrapper {
-	private static final long serialVersionUID = 1L;
+    private static final long serialVersionUID = 1L;
 
-	private Applications apps;
+    private Applications apps;
 
-	public Applications getApps() {
-		return apps;
-	}
+    public Applications getApps() {
+        return apps;
+    }
+
+    public void setApps(Applications apps) {
+        this.apps = apps;
+    }
 
-	public void setApps(Applications apps) {
-		this.apps = apps;
-	}
-	
 }

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/0b852cbc/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/ClusterInfo.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/ClusterInfo.java b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/ClusterInfo.java
index 75a1e8a..6e16b7f 100644
--- a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/ClusterInfo.java
+++ b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/ClusterInfo.java
@@ -14,14 +14,14 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.eagle.jpm.util.resourceFetch.model;
+package org.apache.eagle.jpm.util.resourcefetch.model;
 
 import org.codehaus.jackson.annotate.JsonIgnoreProperties;
 import org.codehaus.jackson.map.annotate.JsonSerialize;
 
 import java.io.Serializable;
 
-@JsonSerialize(include=JsonSerialize.Inclusion.NON_NULL)
+@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
 @JsonIgnoreProperties(ignoreUnknown = true)
 public class ClusterInfo implements Serializable {
     private static final long serialVersionUID = 1L;

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/0b852cbc/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/ClusterInfoWrapper.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/ClusterInfoWrapper.java b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/ClusterInfoWrapper.java
index 301487e..4359e66 100644
--- a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/ClusterInfoWrapper.java
+++ b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/ClusterInfoWrapper.java
@@ -15,12 +15,12 @@
  * limitations under the License.
  */
 
-package org.apache.eagle.jpm.util.resourceFetch.model;
+package org.apache.eagle.jpm.util.resourcefetch.model;
 
 import org.codehaus.jackson.annotate.JsonIgnoreProperties;
 import org.codehaus.jackson.map.annotate.JsonSerialize;
 
-@JsonSerialize(include=JsonSerialize.Inclusion.NON_NULL)
+@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
 @JsonIgnoreProperties(ignoreUnknown = true)
 public class ClusterInfoWrapper {
     private ClusterInfo clusterInfo;

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/0b852cbc/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/JobCounterGroup.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/JobCounterGroup.java b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/JobCounterGroup.java
index 102843f..189767b 100644
--- a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/JobCounterGroup.java
+++ b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/JobCounterGroup.java
@@ -15,13 +15,14 @@
  * limitations under the License.
  */
 
-package org.apache.eagle.jpm.util.resourceFetch.model;
+package org.apache.eagle.jpm.util.resourcefetch.model;
 
 import org.codehaus.jackson.annotate.JsonIgnoreProperties;
 import org.codehaus.jackson.map.annotate.JsonSerialize;
 
 import java.util.List;
-@JsonSerialize(include=JsonSerialize.Inclusion.NON_NULL)
+
+@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
 @JsonIgnoreProperties(ignoreUnknown = true)
 public class JobCounterGroup {
     private String counterGroupName;
@@ -30,12 +31,15 @@ public class JobCounterGroup {
     public String getCounterGroupName() {
         return counterGroupName;
     }
+
     public void setCounterGroupName(String counterGroupName) {
         this.counterGroupName = counterGroupName;
     }
+
     public List<JobCounterItem> getCounter() {
         return counter;
     }
+
     public void setCounter(List<JobCounterItem> counter) {
         this.counter = counter;
     }



[32/52] [abbrv] incubator-eagle git commit: [EAGLE-511] Fix NullPointerException for spark history job

Posted by yo...@apache.org.
[EAGLE-511] Fix NullPointerException for spark history job

Author: pkuwm <ih...@gmail.com>

Closes #404 from pkuwm/EAGLE-511.


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

Branch: refs/heads/master
Commit: 9488afc15e3d721f55aaaf10da46673d3ebb69a7
Parents: 3f7004f
Author: pkuwm <ih...@gmail.com>
Authored: Fri Sep 2 10:58:45 2016 +0800
Committer: Qingwen Zhao <qi...@gmail.com>
Committed: Fri Sep 2 10:58:45 2016 +0800

----------------------------------------------------------------------
 .../environment/impl/StormExecutionRuntime.java |  4 +--
 .../jpm/spark/crawl/JHFSparkEventReader.java    | 28 +++++++++++---------
 .../eagle/jpm/spark/crawl/JHFSparkParser.java   |  2 +-
 .../eagle/jpm/spark/entity/SparkTask.java       |  6 ++---
 .../spark/history/SparkHistoryJobAppConfig.java |  2 --
 .../src/main/resources/application.conf         |  3 +--
 .../org/apache/eagle/jpm/util/JSONUtils.java    | 15 ++++++-----
 7 files changed, 30 insertions(+), 30 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/9488afc1/eagle-core/eagle-app/eagle-app-base/src/main/java/org/apache/eagle/app/environment/impl/StormExecutionRuntime.java
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-app/eagle-app-base/src/main/java/org/apache/eagle/app/environment/impl/StormExecutionRuntime.java b/eagle-core/eagle-app/eagle-app-base/src/main/java/org/apache/eagle/app/environment/impl/StormExecutionRuntime.java
index 1b989ac..04cc19b 100644
--- a/eagle-core/eagle-app/eagle-app-base/src/main/java/org/apache/eagle/app/environment/impl/StormExecutionRuntime.java
+++ b/eagle-core/eagle-app/eagle-app-base/src/main/java/org/apache/eagle/app/environment/impl/StormExecutionRuntime.java
@@ -98,9 +98,9 @@ public class StormExecutionRuntime implements ExecutionRuntime<StormEnvironment,
         String topologyName = config.getString("appId");
         Preconditions.checkNotNull(topologyName,"[appId] is required by null for "+executor.getClass().getCanonicalName());
         StormTopology topology = executor.execute(config, environment);
-        LOG.info("Starting {} ({})",topologyName,executor.getClass().getCanonicalName());
+        LOG.info("Starting {} ({}), mode: {}",topologyName,executor.getClass().getCanonicalName(), config.getString("mode"));
         Config conf = getStormConfig();
-        if(config.getString("mode").equals(ApplicationEntity.Mode.CLUSTER.name())){
+        if(ApplicationEntity.Mode.CLUSTER.name().equalsIgnoreCase(config.getString("mode"))){
             String jarFile = config.hasPath("jarPath") ? config.getString("jarPath") : null;
             if(jarFile == null){
                 jarFile = DynamicJarPathFinder.findPath(executor.getClass());

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/9488afc1/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/spark/crawl/JHFSparkEventReader.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/spark/crawl/JHFSparkEventReader.java b/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/spark/crawl/JHFSparkEventReader.java
index 6c68b48..22b715a 100644
--- a/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/spark/crawl/JHFSparkEventReader.java
+++ b/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/spark/crawl/JHFSparkEventReader.java
@@ -23,6 +23,7 @@ import org.apache.commons.lang.ArrayUtils;
 import org.apache.eagle.jpm.spark.entity.*;
 import org.apache.eagle.jpm.util.*;
 import org.apache.eagle.log.base.taggedlog.TaggedLogAPIEntity;
+import org.apache.eagle.service.client.EagleServiceClientException;
 import org.apache.eagle.service.client.impl.EagleServiceBaseClient;
 import org.apache.eagle.service.client.impl.EagleServiceClientImpl;
 import org.json.simple.JSONArray;
@@ -30,6 +31,7 @@ import org.json.simple.JSONObject;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import java.io.IOException;
 import java.util.*;
 
 public class JHFSparkEventReader {
@@ -44,7 +46,7 @@ public class JHFSparkEventReader {
     private Map<Integer, SparkJob> jobs;
     private Map<String, SparkStage> stages;
     private Map<Integer, Set<String>> jobStageMap;
-    private Map<Integer, SparkTask> tasks;
+    private Map<Long, SparkTask> tasks;
     private EagleServiceClientImpl client;
     private Map<String, Map<Integer, Boolean>> stageTaskStatusMap;
 
@@ -61,7 +63,7 @@ public class JHFSparkEventReader {
         jobs = new HashMap<Integer, SparkJob>();
         stages = new HashMap<String, SparkStage>();
         jobStageMap = new HashMap<Integer, Set<String>>();
-        tasks = new HashMap<Integer, SparkTask>();
+        tasks = new HashMap<Long, SparkTask>();
         executors = new HashMap<String, SparkExecutor>();
         stageTaskStatusMap = new HashMap<>();
         conf = ConfigFactory.load();
@@ -72,7 +74,7 @@ public class JHFSparkEventReader {
         return this.app;
     }
 
-    public void read(JSONObject eventObj) throws Exception {
+    public void read(JSONObject eventObj) {
         String eventType = (String) eventObj.get("Event");
         if (eventType.equalsIgnoreCase(EventType.SparkListenerApplicationStart.toString())) {
             handleAppStarted(eventObj);
@@ -168,7 +170,7 @@ public class JHFSparkEventReader {
         this.lastEventTime = appStartTime;
     }
 
-    private void handleExecutorAdd(JSONObject event) throws Exception {
+    private void handleExecutorAdd(JSONObject event) {
         String executorID = (String) event.get("Executor ID");
         long executorAddTime = JSONUtils.getLong(event, "Timestamp", lastEventTime);
         this.lastEventTime = executorAddTime;
@@ -178,7 +180,7 @@ public class JHFSparkEventReader {
 
     }
 
-    private void handleBlockManagerAdd(JSONObject event) throws Exception {
+    private void handleBlockManagerAdd(JSONObject event) {
         long maxMemory = JSONUtils.getLong(event, "Maximum Memory");
         long timestamp = JSONUtils.getLong(event, "Timestamp", lastEventTime);
         this.lastEventTime = timestamp;
@@ -197,7 +199,7 @@ public class JHFSparkEventReader {
 
     private void handleTaskEnd(JSONObject event) {
         JSONObject taskInfo = JSONUtils.getJSONObject(event, "Task Info");
-        int taskId = JSONUtils.getInt(taskInfo, "Task ID");
+        long taskId = JSONUtils.getLong(taskInfo, "Task ID");
         SparkTask task = tasks.get(taskId);
         if (task == null) {
             return;
@@ -261,10 +263,10 @@ public class JHFSparkEventReader {
         task.getTags().put(SparkJobTagName.SPARK_STAGE_ATTEMPT_ID.toString(), Long.toString(JSONUtils.getLong(event, "Stage Attempt ID")));
 
         JSONObject taskInfo = JSONUtils.getJSONObject(event, "Task Info");
-        int taskId = JSONUtils.getInt(taskInfo, "Task ID");
+        long taskId = JSONUtils.getLong(taskInfo, "Task ID");
         task.setTaskId(taskId);
 
-        task.getTags().put(SparkJobTagName.SPARK_TASK_INDEX.toString(), Integer.toString(JSONUtils.getInt(taskInfo, "Index")));
+        task.getTags().put(SparkJobTagName.SPARK_TASK_INDEX.toString(), Long.toString(JSONUtils.getLong(taskInfo, "Index")));
         task.getTags().put(SparkJobTagName.SPARK_TASK_ATTEMPT_ID.toString(), Integer.toString(JSONUtils.getInt(taskInfo, "Attempt")));
         long launchTime = JSONUtils.getLong(taskInfo, "Launch Time", lastEventTime);
         this.lastEventTime = launchTime;
@@ -323,7 +325,7 @@ public class JHFSparkEventReader {
         String key = this.generateStageKey(Integer.toString(stageId), Integer.toString(stageAttemptId));
         stageTaskStatusMap.put(key, new HashMap<Integer, Boolean>());
 
-        if (!stages.containsKey(this.generateStageKey(Integer.toString(stageId), Integer.toString(stageAttemptId)))) {
+        if (!stages.containsKey(key)) {
             //may be further attempt for one stage
             String baseAttempt = this.generateStageKey(Integer.toString(stageId), "0");
             if (stages.containsKey(baseAttempt)) {
@@ -651,7 +653,7 @@ public class JHFSparkEventReader {
     }
 
 
-    private SparkExecutor initiateExecutor(String executorID, long startTime) throws Exception {
+    private SparkExecutor initiateExecutor(String executorID, long startTime) {
         if (!executors.containsKey(executorID)) {
             SparkExecutor executor = new SparkExecutor();
             executor.setTags(new HashMap<>(this.app.getTags()));
@@ -703,9 +705,9 @@ public class JHFSparkEventReader {
         return client;
     }
 
-    private void doFlush(List entities) throws Exception {
-        LOG.info("start flushing entities of total number " + entities.size());
+    private void doFlush(List entities) throws IOException, EagleServiceClientException {
         client.create(entities);
-        LOG.info("finish flushing entities of total number " + entities.size());
+        int size = (entities == null ? 0 : entities.size());
+        LOG.info("finish flushing entities of total number " + size);
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/9488afc1/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/spark/crawl/JHFSparkParser.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/spark/crawl/JHFSparkParser.java b/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/spark/crawl/JHFSparkParser.java
index 2ba3c73..02fc5cf 100644
--- a/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/spark/crawl/JHFSparkParser.java
+++ b/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/spark/crawl/JHFSparkParser.java
@@ -50,7 +50,7 @@ public class JHFSparkParser implements JHFParserBase {
                     try {
                         this.eventReader.read(eventObj);
                     } catch (Exception e) {
-                        logger.error("Fail to read eventObj. Exception: " + e);
+                        e.printStackTrace();
                     }
                 }
             }

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/9488afc1/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/spark/entity/SparkTask.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/spark/entity/SparkTask.java b/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/spark/entity/SparkTask.java
index fb2fce5..5d8f1d3 100644
--- a/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/spark/entity/SparkTask.java
+++ b/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/spark/entity/SparkTask.java
@@ -34,7 +34,7 @@ import org.codehaus.jackson.annotate.JsonIgnoreProperties;
 public class SparkTask extends TaggedLogAPIEntity {
 
     @Column("a")
-    private int taskId;
+    private long taskId;
     @Column("b")
     private long launchTime;
     @Column("c")
@@ -80,7 +80,7 @@ public class SparkTask extends TaggedLogAPIEntity {
     @Column("v")
     private boolean failed;
 
-    public int getTaskId() {
+    public long getTaskId() {
         return taskId;
     }
 
@@ -177,7 +177,7 @@ public class SparkTask extends TaggedLogAPIEntity {
         valueChanged("failed");
     }
 
-    public void setTaskId(int taskId) {
+    public void setTaskId(long taskId) {
         this.taskId = taskId;
         valueChanged("taskId");
     }

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/9488afc1/eagle-jpm/eagle-jpm-spark-history/src/main/java/org/apache/eagle/jpm/spark/history/SparkHistoryJobAppConfig.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-spark-history/src/main/java/org/apache/eagle/jpm/spark/history/SparkHistoryJobAppConfig.java b/eagle-jpm/eagle-jpm-spark-history/src/main/java/org/apache/eagle/jpm/spark/history/SparkHistoryJobAppConfig.java
index ed499db..0fc74d7 100644
--- a/eagle-jpm/eagle-jpm-spark-history/src/main/java/org/apache/eagle/jpm/spark/history/SparkHistoryJobAppConfig.java
+++ b/eagle-jpm/eagle-jpm-spark-history/src/main/java/org/apache/eagle/jpm/spark/history/SparkHistoryJobAppConfig.java
@@ -82,7 +82,6 @@ public class SparkHistoryJobAppConfig implements Serializable {
         this.eagleInfo.host = config.getString("eagleProps.eagle.service.host");
         this.eagleInfo.port = config.getInt("eagleProps.eagle.service.port");
 
-        this.stormConfig.mode = config.getString("storm.mode");
         this.stormConfig.topologyName = config.getString("storm.name");
         this.stormConfig.workerNo = config.getInt("storm.worker.num");
         this.stormConfig.timeoutSec = config.getInt("storm.messageTimeoutSec");
@@ -118,7 +117,6 @@ public class SparkHistoryJobAppConfig implements Serializable {
     }
 
     public static class StormConfig implements Serializable {
-        public String mode;
         public int workerNo;
         public int timeoutSec;
         public String topologyName;

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/9488afc1/eagle-jpm/eagle-jpm-spark-history/src/main/resources/application.conf
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-spark-history/src/main/resources/application.conf b/eagle-jpm/eagle-jpm-spark-history/src/main/resources/application.conf
index 483e2e9..58dd552 100644
--- a/eagle-jpm/eagle-jpm-spark-history/src/main/resources/application.conf
+++ b/eagle-jpm/eagle-jpm-spark-history/src/main/resources/application.conf
@@ -46,7 +46,6 @@
   },
   "storm":{
     worker.num: 2,
-    "mode": "local",
     "name":"sparkHistoryJob",
     "messageTimeoutSec": 3000,
     "pendingSpout": 1000,
@@ -75,5 +74,5 @@
     }
   },
   "appId": "sparkHistoryJob",
-  "mode": "LOCAL"
+  "mode": "CLUSTER"
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/9488afc1/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/JSONUtils.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/JSONUtils.java b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/JSONUtils.java
index 38500b0..8a12cc5 100644
--- a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/JSONUtils.java
+++ b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/JSONUtils.java
@@ -30,23 +30,24 @@ public class JSONUtils {
         }
 
         try {
-            return obj.get(field).toString();
+            return (String) obj.get(field);
         } catch (JSONException e) {
             e.printStackTrace();
             return null;
         }
     }
 
-    public static int getInt(JSONObject obj, String field) {
+    public static int getInt(JSONObject obj, String field) throws JSONException {
         if (obj == null || StringUtils.isEmpty(field)) {
             return 0;
         }
-
+        Object object = obj.get(field);
         try {
-            return (int) obj.get(field);
-        } catch (JSONException e) {
-            e.printStackTrace();
-            return 0;
+            return object instanceof Number ? ((Number) object).intValue()
+                    : Integer.parseInt((String) object);
+        } catch (Exception e) {
+            throw new JSONException("JSONObject[" + field
+                    + "] is not an int.");
         }
     }
 


[50/52] [abbrv] incubator-eagle git commit: [EAGLE-520] Fix and decouple co-processor from eagle aggreation query service

Posted by yo...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/21187b55/eagle-core/eagle-query/eagle-storage-hbase/src/main/java/org/apache/eagle/storage/hbase/query/aggregate/GenericAggregateQuery.java
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-query/eagle-storage-hbase/src/main/java/org/apache/eagle/storage/hbase/query/aggregate/GenericAggregateQuery.java b/eagle-core/eagle-query/eagle-storage-hbase/src/main/java/org/apache/eagle/storage/hbase/query/aggregate/GenericAggregateQuery.java
deleted file mode 100755
index 8f20b61..0000000
--- a/eagle-core/eagle-query/eagle-storage-hbase/src/main/java/org/apache/eagle/storage/hbase/query/aggregate/GenericAggregateQuery.java
+++ /dev/null
@@ -1,423 +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.eagle.storage.hbase.query.aggregate;
-
-import org.apache.eagle.log.entity.GenericMetricEntity;
-import org.apache.eagle.log.entity.HBaseInternalLogHelper;
-import org.apache.eagle.log.entity.SearchCondition;
-import org.apache.eagle.log.entity.meta.EntityConstants;
-import org.apache.eagle.log.entity.meta.EntityDefinition;
-import org.apache.eagle.log.entity.meta.EntityDefinitionManager;
-import org.apache.eagle.query.GenericQuery;
-import org.apache.eagle.query.QueryConstants;
-import org.apache.eagle.query.aggregate.AggregateCondition;
-import org.apache.eagle.query.aggregate.AggregateFunctionType;
-import org.apache.eagle.query.aggregate.raw.GroupbyKey;
-import org.apache.eagle.query.aggregate.raw.GroupbyKeyValue;
-import org.apache.eagle.query.aggregate.raw.GroupbyValue;
-import org.apache.eagle.query.aggregate.timeseries.PostFlatAggregateSort;
-import org.apache.eagle.query.aggregate.timeseries.SortOption;
-import org.apache.eagle.query.aggregate.timeseries.TimeSeriesAggregator;
-import org.apache.eagle.query.aggregate.timeseries.TimeSeriesPostFlatAggregateSort;
-import org.apache.eagle.storage.hbase.query.coprocessor.AggregateResult;
-import org.apache.eagle.storage.hbase.query.coprocessor.impl.AggregateResultCallbackImpl;
-import org.apache.eagle.common.DateTimeUtil;
-import org.apache.hadoop.io.BytesWritable;
-import org.apache.hadoop.io.DoubleWritable;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-import java.io.UnsupportedEncodingException;
-import java.util.*;
-
-/**
- * AggregateQuery
- *
- * <ol>
- *   <li>Open HBase connection</li>
- *   <li>Aggregate through Coprocessor</li>
- *   <li>Build GroupAggregateQuery.GroupAggregateQueryReader to process result and order as sort options</li>
- *   <li>Return result list</li>
- * </ol>
- *
- * @since : 11/7/14,2014
- */
-public class GenericAggregateQuery implements GenericQuery {
-	private static final Logger LOG = LoggerFactory.getLogger(GenericAggregateQuery.class);
-	private final List<AggregateFunctionType> sortFuncs;
-	private final List<String> sortFields;
-
-	private EntityDefinition entityDef;
-	private SearchCondition searchCondition;
-	private AggregateCondition aggregateCondition;
-	private String prefix;
-	private long lastTimestamp = 0;
-	private long firstTimestamp = 0;
-	private List<SortOption> sortOptions;
-	private int top;
-
-	private int aggFuncNum;
-	private int sortAggFuncNum;
-	private int sortFuncNum;
-
-	/**
-	 *
-	 * @param serviceName
-	 * @param condition
-	 * @param aggregateCondition
-	 * @param metricName
-	 * @throws InstantiationException
-	 * @throws IllegalAccessException
-	 */
-	public GenericAggregateQuery(String serviceName, SearchCondition condition, AggregateCondition aggregateCondition, String metricName)
-			throws InstantiationException, IllegalAccessException{
-		this(serviceName, condition, aggregateCondition, metricName,null,null,null,0);
-	}
-
-	/**
-	 *
-	 * @param serviceName
-	 * @param condition
-	 * @param aggregateCondition
-	 * @param metricName
-	 * @param sortOptions
-	 * @param sortFunctionTypes
-	 * @param sortFields
-	 * @param top
-	 * @throws InstantiationException
-	 * @throws IllegalAccessException
-	 */
-	public GenericAggregateQuery(String serviceName, SearchCondition condition,
-	                           AggregateCondition aggregateCondition, String metricName,
-	                           List<SortOption> sortOptions,List<AggregateFunctionType> sortFunctionTypes,List<String> sortFields,int top)
-			throws InstantiationException, IllegalAccessException{
-		checkNotNull(serviceName, "serviceName");
-		this.searchCondition = condition;
-		this.entityDef = EntityDefinitionManager.getEntityByServiceName(serviceName);
-		checkNotNull(entityDef, "EntityDefinition");
-		checkNotNull(entityDef, "GroupAggregateCondition");
-		this.aggregateCondition = aggregateCondition;
-		this.aggFuncNum = this.aggregateCondition.getAggregateFunctionTypes().size();
-		this.sortOptions = sortOptions;
-		this.sortFuncs  = sortFunctionTypes;
-		this.sortFuncNum = this.sortOptions == null ? 0: this.sortOptions.size();
-		this.sortFields = sortFields;
-		this.top = top;
-
-		if(serviceName.equals(GenericMetricEntity.GENERIC_METRIC_SERVICE)){
-			if(LOG.isDebugEnabled()) LOG.debug("list metric aggregate query");
-			if(metricName == null || metricName.isEmpty()){
-				throw new IllegalArgumentException("metricName should not be empty for metric list query");
-			}
-			if(!condition.getOutputFields().contains(GenericMetricEntity.VALUE_FIELD)){
-				condition.getOutputFields().add(GenericMetricEntity.VALUE_FIELD);
-			}
-			this.prefix = metricName;
-		}else{
-			if(LOG.isDebugEnabled()) LOG.debug("list entity aggregate query");
-			this.prefix = entityDef.getPrefix();
-		}
-
-		// Add sort oriented aggregation functions into aggregateCondtion
-		if(this.sortOptions!=null){
-			// if sort for time series aggregation
-			if(this.aggregateCondition.isTimeSeries()) {
-				this.sortAggFuncNum = 0;
-				int index = 0;
-				for (SortOption sortOption : this.sortOptions) {
-					if (!sortOption.isInGroupby()) {
-						if (LOG.isDebugEnabled())
-							LOG.debug("Add additional aggregation functions for sort options " + sortOption.toString() + " in index: " + (this.aggFuncNum + this.sortAggFuncNum));
-						AggregateFunctionType _sortFunc = this.sortFuncs.get(index);
-						if (AggregateFunctionType.avg.equals(_sortFunc)) {
-							this.aggregateCondition.getAggregateFunctionTypes().add(AggregateFunctionType.sum);
-						} else {
-							this.aggregateCondition.getAggregateFunctionTypes().add(_sortFunc);
-						}
-						this.aggregateCondition.getAggregateFields().add(this.sortFields.get(index));
-
-						sortOption.setIndex(this.sortAggFuncNum);
-						sortAggFuncNum++;
-					}
-					index++;
-				}
-			}
-		}
-	}
-
-
-	private void checkNotNull(Object o, String message){
-		if(o == null){
-			throw new IllegalArgumentException(message + " should not be null");
-		}
-	}
-
-	/**
-	 * TODO: Return List<GroupAggregateAPIEntity>
-	 *
-	 * @see GenericAggregateQuery.TimeSeriesGroupAggregateQueryReader#result()
-	 * @see GenericAggregateQuery.FlatGroupAggregateQueryReader#result()
-	 *
- 	 */
-	@Override
-	@SuppressWarnings("raw")
-	public List result() throws Exception {
-		Date start = null;
-		Date end = null;
-		// shortcut to avoid read when pageSize=0
-		if(searchCondition.getPageSize() <= 0){
-			return null;
-		}
-		// Process the time range if needed
-		if(entityDef.isTimeSeries()){
-			start = DateTimeUtil.humanDateToDate(searchCondition.getStartTime());
-			end = DateTimeUtil.humanDateToDate(searchCondition.getEndTime());
-		}else{
-			start = DateTimeUtil.humanDateToDate(EntityConstants.FIXED_READ_START_HUMANTIME);
-			end = DateTimeUtil.humanDateToDate(EntityConstants.FIXED_READ_END_HUMANTIME);
-		}
-		// Generate the output qualifiers
-		final byte[][] outputQualifiers = HBaseInternalLogHelper.getOutputQualifiers(entityDef, searchCondition.getOutputFields());
-		GenericAggregateReader reader = new GenericAggregateReader(entityDef,
-				searchCondition.getPartitionValues(),
-				start, end, searchCondition.getFilter(), searchCondition.getStartRowkey(), outputQualifiers, this.prefix,this.aggregateCondition);
-		try{
-			if(LOG.isDebugEnabled()) LOG.debug("open and read group aggregate reader");
-			reader.open();
-			List result = buildGroupAggregateQueryReader(reader,this.aggregateCondition.isTimeSeries()).result();
-			if(result == null) throw new IOException("result is null");
-			this.firstTimestamp = reader.getFirstTimestamp();
-			this.lastTimestamp = reader.getLastTimestamp();
-			if(LOG.isDebugEnabled()) LOG.debug("finish read aggregated " + result.size() + " rows");
-			return result;
-		}catch (IOException ex){
-			LOG.error("Fail reading aggregated results", ex);
-			throw ex;
-		}finally{
-			if(reader != null) {
-				if(LOG.isDebugEnabled()) LOG.debug("Release HBase connection");
-				reader.close();
-			}
-		}
-	}
-
-	///////////////////////////////////////////////////////////
-	// GroupAggregateQueryReader(GroupAggregateLogReader)
-	// 	|_ FlatGroupAggregateQueryReader
-	// 	|_ TimeSeriesGroupAggregateQueryReader
-	///////////////////////////////////////////////////////////
-
-	/**
-	 * Factory method for {@link GroupAggregateQueryReader}
-	 * <pre>
-	 * {@link GroupAggregateQueryReader}
-	 * |_ {@link FlatGroupAggregateQueryReader}
-	 * |_ {@link TimeSeriesGroupAggregateQueryReader}
-	 * </pre>
-	 * @param reader
-	 * @param isTimeSeries
-	 * @return
-	 * @throws IOException
-	 */
-	private  GroupAggregateQueryReader  buildGroupAggregateQueryReader(GenericAggregateReader reader,boolean isTimeSeries) throws IOException{
-		if(isTimeSeries){
-			return new TimeSeriesGroupAggregateQueryReader(reader,this);
-		}else{
-			return new FlatGroupAggregateQueryReader(reader,this);
-		}
-	}
-
-	private abstract class GroupAggregateQueryReader {
-		protected final GenericAggregateReader reader;
-		protected final GenericAggregateQuery query;
-
-		public GroupAggregateQueryReader(GenericAggregateReader reader, GenericAggregateQuery query){
-			this.reader = reader;
-			this.query = query;
-		}
-		public abstract <T> List<T> result() throws Exception;
-
-		protected Map<List<String>, List<Double>> keyValuesToMap(List<GroupbyKeyValue> entities) throws Exception {
-			Map<List<String>, List<Double>> aggResultMap = new HashMap<List<String>, List<Double>>();
-			try {
-				for(GroupbyKeyValue keyValue:entities){
-					List<String> key = new ArrayList<String>();
-					for(BytesWritable bw:keyValue.getKey().getValue()){
-						key.add(new String(bw.copyBytes(), QueryConstants.CHARSET));
-					}
-					List<Double> value = new ArrayList<Double>();
-					for(DoubleWritable wa:keyValue.getValue().getValue()){
-						value.add(wa.get());
-					}
-					aggResultMap.put(key, value);
-				}
-			} catch (UnsupportedEncodingException e) {
-				LOG.error(QueryConstants.CHARSET +" not support: "+e.getMessage(),e);
-			}
-			return aggResultMap;
-		}
-	}
-
-	private class FlatGroupAggregateQueryReader extends GroupAggregateQueryReader{
-		public FlatGroupAggregateQueryReader(GenericAggregateReader reader, GenericAggregateQuery query) {
-			super(reader,query);
-		}
-		@Override
-		public List<Map.Entry<List<String>, List<Double>>> result() throws Exception {
-			Map<List<String>, List<Double>> aggResultMap = this.keyValuesToMap(this.reader.read());
-			if(this.query.sortOptions == null)
-				return new ArrayList<Map.Entry<List<String>, List<Double>>>(aggResultMap.entrySet());
-			if(LOG.isDebugEnabled()) LOG.debug("Flat sorting");
-			return PostFlatAggregateSort.sort(aggResultMap, this.query.sortOptions, this.query.top);
-		}
-	}
-
-	private class TimeSeriesGroupAggregateQueryReader extends GroupAggregateQueryReader{
-		private final Date start;
-		private final Date end;
-		private final int pointsNum;
-		private final int aggFuncNum;
-		private final List<SortOption> sortOptions;
-		private final List<AggregateFunctionType> sortFuncs;
-		private final int sortAggFuncNum;
-
-		public TimeSeriesGroupAggregateQueryReader(GenericAggregateReader reader, GenericAggregateQuery query) throws IOException {
-			super(reader,query);
-			try {
-				if(entityDef.isTimeSeries()){
-						this.start = DateTimeUtil.humanDateToDate(searchCondition.getStartTime());
-					this.end = DateTimeUtil.humanDateToDate(searchCondition.getEndTime());
-				}else{
-					start = DateTimeUtil.humanDateToDate(EntityConstants.FIXED_READ_START_HUMANTIME);
-					end = DateTimeUtil.humanDateToDate(EntityConstants.FIXED_READ_END_HUMANTIME);
-				}
-				this.pointsNum = (int)((end.getTime()-1-start.getTime())/this.query.aggregateCondition.getIntervalMS() + 1);
-				this.aggFuncNum = this.query.aggFuncNum;
-				this.sortOptions = this.query.sortOptions;
-				this.sortFuncs = this.query.sortFuncs;
-				this.sortAggFuncNum = this.query.sortAggFuncNum;
-			} catch (Exception e) {
-				throw new IOException(e);
-			}
-		}
-
-		/**
-		 * <h2>TimeSeriesReader result</h2>
-		 * <ol>
-		 *  <li>generateTimeSeriesDataPoints()</li>
-		 *  <li>if not sort options, return generate time series data points</li>
-		 *  <li>if requiring sort, sort time series data points by order of flat aggregation</li>
-		 * </ol>
-		 *
-		 * 	<h2>Time Series Sort Algorithms</h2>
-		 * 	<ol>
-		 *	<li>Flat aggregate on grouped fields without time series bucket index</li>
-		 *	<li>Flat aggregated result according given sortOptions</li>
-		 *	<li>Sort Time Series Result according the same order of flat aggregated keys</li>
-		 * </ol>
-		 *
-		 * @see #convertToTimeSeriesDataPoints(java.util.List)
-		 *
-		 * @return
-		 * @throws Exception
-		 */
-		@Override
-		public List<Map.Entry<List<String>, List<double[]>>> result() throws Exception {
-			List<GroupbyKeyValue> result = this.reader.read();
-
-			// aggregated data points only
-			Map<List<String>,List<double[]>> timeseriesDataPoints = convertToTimeSeriesDataPoints(result);
-
-			if(this.query.sortOptions == null)
-				// return time-series data points without sort
-				return new ArrayList<Map.Entry<List<String>, List<double[]>>>(timeseriesDataPoints.entrySet());
-
-			LOG.info("Time series sorting");
-
-			// Time Series Sort Steps
-			// ======================
-			// 1. Flat aggregate on grouped fields without time series bucket index
-			// 2. Flat aggregated result according given sortOptions
-			// 3. Sort Time Series Result according flat aggregated keys' order
-
-			// 1. Flat aggregate on grouped fields without time series bucket index
-			AggregateResultCallbackImpl callback = new AggregateResultCallbackImpl(this.sortFuncs);
-			for(GroupbyKeyValue kv:result){
-				ArrayList<BytesWritable> copykey = new ArrayList<BytesWritable>(kv.getKey().getValue());
-				// remove time series bucket index
-				copykey.remove(copykey.size()-1);
-				GroupbyKey key = new GroupbyKey();
-
-				// [this.aggFuncNum,this.aggFuncNum + this.sortFuncNum)
-				GroupbyValue value = new GroupbyValue();
-				for(int i = this.aggFuncNum;i<this.aggFuncNum+this.sortAggFuncNum;i++){
-					value.add(kv.getValue().get(i));
-					value.addMeta(kv.getValue().getMeta(i));
-				}
-				key.addAll(copykey);
-				GroupbyKeyValue keyValue = new GroupbyKeyValue(key,value);
-				callback.update(keyValue);
-			}
-			AggregateResult callbackResult = callback.result();
-			Map<List<String>, List<Double>> mapForSort = this.keyValuesToMap(callbackResult.getKeyValues());
-
-			// 2. Flat aggregated result according given sortOptions
-//			List<Map.Entry<List<String>, List<Double>>> flatSort = PostFlatAggregateSort.sort(mapForSort , this.sortOptions, Integer.MAX_VALUE);
-//			mapForSort = new HashMap<List<String>, List<Double>>();
-//			for(Map.Entry<List<String>, List<Double>> entry:flatSort){
-//				mapForSort.put(entry.getKey(),entry.getValue());
-//			}
-
-			// 3. Sort Time Series Result according flat aggregated keys' order
-			return TimeSeriesPostFlatAggregateSort.sort(mapForSort,timeseriesDataPoints,this.sortOptions,this.query.top);
-		}
-
-		/**
-		 * Convert raw GroupbyKeyValue list into time-series data points hash map
-		 *
-		 * @param result <code>List&lt;GroupbyKeyValue&gt;</code>
-		 * @return Map&lt;List&lt;String&gt;,List&lt;double[]&gt;&gt;
-		 * @throws Exception
-		 */
-		private Map<List<String>,List<double[]>> convertToTimeSeriesDataPoints(List<GroupbyKeyValue> result) throws Exception {
-			Map<List<String>, List<Double>> aggResultMap = this.keyValuesToMap(result);
-			Map<List<String>,List<double[]>> timeseriesDataPoints = TimeSeriesAggregator.toMetric(aggResultMap,this.pointsNum,this.aggFuncNum);
-			return timeseriesDataPoints;
-		}
-	}
-
-	/**
-	 * Get last / max timestamp
-	 *
-	 * @return lastTimestamp
-	 */
-	@Override
-	public long getLastTimestamp() {
-		return this.lastTimestamp;
-	}
-
-	/**
-	 * Get first / min timestamp
-	 *
-	 * @return firstTimestamp
-	 */
-	@Override
-	public long getFirstTimeStamp() {
-		return this.firstTimestamp;
-	}
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/21187b55/eagle-core/eagle-query/eagle-storage-hbase/src/main/java/org/apache/eagle/storage/hbase/query/aggregate/GenericAggregateReader.java
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-query/eagle-storage-hbase/src/main/java/org/apache/eagle/storage/hbase/query/aggregate/GenericAggregateReader.java b/eagle-core/eagle-query/eagle-storage-hbase/src/main/java/org/apache/eagle/storage/hbase/query/aggregate/GenericAggregateReader.java
index 3d0fa94..67c957b 100755
--- a/eagle-core/eagle-query/eagle-storage-hbase/src/main/java/org/apache/eagle/storage/hbase/query/aggregate/GenericAggregateReader.java
+++ b/eagle-core/eagle-query/eagle-storage-hbase/src/main/java/org/apache/eagle/storage/hbase/query/aggregate/GenericAggregateReader.java
@@ -19,10 +19,11 @@ package org.apache.eagle.storage.hbase.query.aggregate;
 import org.apache.eagle.log.entity.AbstractHBaseLogReader;
 import org.apache.eagle.log.entity.meta.EntityDefinition;
 import org.apache.eagle.query.aggregate.AggregateCondition;
-import org.apache.eagle.storage.hbase.query.coprocessor.impl.AggregateClientImpl;
 import org.apache.eagle.query.aggregate.raw.GroupbyKeyValue;
 import org.apache.eagle.storage.hbase.query.coprocessor.AggregateClient;
 import org.apache.eagle.storage.hbase.query.coprocessor.AggregateResult;
+import org.apache.eagle.storage.hbase.query.coprocessor.impl.AggregateClientImpl;
+
 import org.apache.hadoop.hbase.client.HTableInterface;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.filter.Filter;
@@ -31,101 +32,93 @@ import java.io.IOException;
 import java.util.Date;
 import java.util.List;
 
-/**
- * @since : 11/7/14,2014
- */
 public class GenericAggregateReader extends AbstractHBaseLogReader<List<GroupbyKeyValue>> {
-	private final long startTime;
-	private final long endTime;
-	private AggregateClient aggregateClient = new AggregateClientImpl();
-	private EntityDefinition ed;
-	private final AggregateCondition aggregateCondition;
-	private AggregateResult result;
+    private final long startTime;
+    private final long endTime;
+    private AggregateClient aggregateClient = new AggregateClientImpl();
+    private EntityDefinition ed;
+    private final AggregateCondition aggregateCondition;
+    private AggregateResult result;
 
-	/**
-	 *
-	 * @param ed                Entity Definition
-	 * @param partitions        Partition values
-	 * @param startTime         Start time
-	 * @param endTime           End time
-	 * @param filter            HBase filter for scanning
-	 * @param lastScanKey       Last HBase scan row key in String
-	 * @param outputQualifiers  HBase output qualifiers in bytes
-	 * @param condition         GroupAggregateCondition Object
-	 *
-	 * @see org.apache.eagle.query.aggregate.AggregateCondition
-	 */
-	@SuppressWarnings("unused")
-	private GenericAggregateReader(EntityDefinition ed,
-	                                List<String> partitions,
-	                                Date startTime,
-	                                Date endTime,
-	                                Filter filter,
-	                                String lastScanKey,
-	                                byte[][] outputQualifiers,
-	                                AggregateCondition condition) {
-		super(ed, partitions, startTime, endTime, filter, lastScanKey, outputQualifiers);
-		this.ed = ed;
-		this.startTime = startTime.getTime();
-		this.endTime = endTime.getTime();
-		this.aggregateCondition = condition;
-	}
+    /**
+     * @param ed               Entity Definition
+     * @param partitions       Partition values
+     * @param startTime        Start time
+     * @param endTime          End time
+     * @param filter           HBase filter for scanning
+     * @param lastScanKey      Last HBase scan row key in String
+     * @param outputQualifiers HBase output qualifiers in bytes
+     * @param condition        GroupAggregateCondition Object
+     * @see org.apache.eagle.query.aggregate.AggregateCondition
+     */
+    private GenericAggregateReader(EntityDefinition ed,
+                                   List<String> partitions,
+                                   Date startTime,
+                                   Date endTime,
+                                   Filter filter,
+                                   String lastScanKey,
+                                   byte[][] outputQualifiers,
+                                   AggregateCondition condition) {
+        super(ed, partitions, startTime, endTime, filter, lastScanKey, outputQualifiers);
+        this.ed = ed;
+        this.startTime = startTime.getTime();
+        this.endTime = endTime.getTime();
+        this.aggregateCondition = condition;
+    }
 
-	/**
-	 *
-	 * @param ed                Entity Definition
-	 * @param partitions        Partition values
-	 * @param startTime         Start time
-	 * @param endTime           End time
-	 * @param filter            HBase filter for scanning
-	 * @param lastScanKey       Last HBase scan row key in String
-	 * @param outputQualifiers  HBase output qualifiers in bytes
-	 * @param prefix            HBase prefix, not necessary except for GenericMetric query
-	 * @param condition         GroupAggregateCondition Object
-	 *
-	 * @see org.apache.eagle.query.aggregate.AggregateCondition
-	 */
-	public GenericAggregateReader(EntityDefinition ed,
-	                               List<String> partitions,
-	                               Date startTime,
-	                               Date endTime,
-	                               Filter filter,
-	                               String lastScanKey,
-	                               byte[][] outputQualifiers,
-	                               String prefix,
-	                               AggregateCondition condition) {
-		super(ed, partitions, startTime, endTime, filter, lastScanKey, outputQualifiers, prefix);
-		this.ed = ed;
-		this.startTime = startTime.getTime();
-		this.endTime = endTime.getTime();
-		this.aggregateCondition = condition;
-	}
+    /**
+     * @param ed               Entity Definition
+     * @param partitions       Partition values
+     * @param startTime        Start time
+     * @param endTime          End time
+     * @param filter           HBase filter for scanning
+     * @param lastScanKey      Last HBase scan row key in String
+     * @param outputQualifiers HBase output qualifiers in bytes
+     * @param prefix           HBase prefix, not necessary except for GenericMetric query
+     * @param condition        GroupAggregateCondition Object
+     * @see org.apache.eagle.query.aggregate.AggregateCondition
+     */
+    public GenericAggregateReader(EntityDefinition ed,
+                                  List<String> partitions,
+                                  Date startTime,
+                                  Date endTime,
+                                  Filter filter,
+                                  String lastScanKey,
+                                  byte[][] outputQualifiers,
+                                  String prefix,
+                                  AggregateCondition condition) {
+        super(ed, partitions, startTime, endTime, filter, lastScanKey, outputQualifiers, prefix);
+        this.ed = ed;
+        this.startTime = startTime.getTime();
+        this.endTime = endTime.getTime();
+        this.aggregateCondition = condition;
+    }
 
-	@Override
-	protected void onOpen(HTableInterface tbl, Scan scan) throws IOException {
-		this.result = this.aggregateClient.aggregate(
-				tbl,
-				this.ed,
-				scan,
-				this.aggregateCondition.getGroupbyFields(),
-				this.aggregateCondition.getAggregateFunctionTypes(),
-				this.aggregateCondition.getAggregateFields(),
-				this.aggregateCondition.isTimeSeries(),
-				this.startTime,
-				this.endTime,
-				this.aggregateCondition.getIntervalMS());
-	}
+    @Override
+    protected void onOpen(HTableInterface tbl, Scan scan) throws IOException {
+        this.result = this.aggregateClient.aggregate(
+            tbl,
+            this.ed,
+            scan,
+            this.aggregateCondition.getGroupbyFields(),
+            this.aggregateCondition.getAggregateFunctionTypes(),
+            this.aggregateCondition.getAggregateFields(),
+            this.aggregateCondition.isTimeSeries(),
+            this.startTime,
+            this.endTime,
+            this.aggregateCondition.getIntervalMS());
+    }
 
-	@Override
-	public List<GroupbyKeyValue> read() throws IOException {
-		return this.result.getKeyValues();
-	}
+    @Override
+    public List<GroupbyKeyValue> read() throws IOException {
+        return this.result.getKeyValues();
+    }
 
-	public long getFirstTimestamp() {
-		return this.result.getStartTimestamp();
-	}
+    public long getFirstTimestamp() {
+        return this.result.getStartTimestamp();
+    }
 
-	public long getLastTimestamp() {
-		return this.result.getStopTimestamp();
-	}
+    public long getLastTimestamp() {
+        return this.result.getStopTimestamp();
+    }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/21187b55/eagle-core/eagle-query/eagle-storage-hbase/src/main/java/org/apache/eagle/storage/hbase/query/aggregate/GenericCoprocessorAggregateQuery.java
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-query/eagle-storage-hbase/src/main/java/org/apache/eagle/storage/hbase/query/aggregate/GenericCoprocessorAggregateQuery.java b/eagle-core/eagle-query/eagle-storage-hbase/src/main/java/org/apache/eagle/storage/hbase/query/aggregate/GenericCoprocessorAggregateQuery.java
new file mode 100644
index 0000000..85044ec
--- /dev/null
+++ b/eagle-core/eagle-query/eagle-storage-hbase/src/main/java/org/apache/eagle/storage/hbase/query/aggregate/GenericCoprocessorAggregateQuery.java
@@ -0,0 +1,394 @@
+/*
+ * 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.eagle.storage.hbase.query.aggregate;
+
+import org.apache.eagle.common.DateTimeUtil;
+import org.apache.eagle.log.entity.GenericMetricEntity;
+import org.apache.eagle.log.entity.HBaseInternalLogHelper;
+import org.apache.eagle.log.entity.SearchCondition;
+import org.apache.eagle.log.entity.meta.EntityConstants;
+import org.apache.eagle.log.entity.meta.EntityDefinition;
+import org.apache.eagle.log.entity.meta.EntityDefinitionManager;
+import org.apache.eagle.query.GenericQuery;
+import org.apache.eagle.query.QueryConstants;
+import org.apache.eagle.query.aggregate.AggregateCondition;
+import org.apache.eagle.query.aggregate.AggregateFunctionType;
+import org.apache.eagle.query.aggregate.raw.GroupbyKey;
+import org.apache.eagle.query.aggregate.raw.GroupbyKeyValue;
+import org.apache.eagle.query.aggregate.raw.GroupbyValue;
+import org.apache.eagle.query.aggregate.timeseries.PostFlatAggregateSort;
+import org.apache.eagle.query.aggregate.timeseries.SortOption;
+import org.apache.eagle.query.aggregate.timeseries.TimeSeriesAggregator;
+import org.apache.eagle.query.aggregate.timeseries.TimeSeriesPostFlatAggregateSort;
+import org.apache.eagle.storage.hbase.query.coprocessor.AggregateResult;
+import org.apache.eagle.storage.hbase.query.coprocessor.impl.AggregateResultCallbackImpl;
+
+import org.apache.hadoop.io.BytesWritable;
+import org.apache.hadoop.io.DoubleWritable;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.io.UnsupportedEncodingException;
+import java.util.*;
+
+/**
+ * Coprocessor based AggregateQuery
+ * <ol>
+ * <li>Open HBase connection</li>
+ * <li>Aggregate through Coprocessor</li>
+ * <li>Build GroupAggregateQuery.GroupAggregateQueryReader to process result and order as sort options</li>
+ * <li>Return result list</li>
+ * </ol>
+ */
+public class GenericCoprocessorAggregateQuery implements GenericQuery {
+    private static final Logger LOG = LoggerFactory.getLogger(GenericCoprocessorAggregateQuery.class);
+    private final List<AggregateFunctionType> sortFuncs;
+    private final List<String> sortFields;
+
+    private EntityDefinition entityDef;
+    private SearchCondition searchCondition;
+    private AggregateCondition aggregateCondition;
+    private String prefix;
+    private long lastTimestamp = 0;
+    private long firstTimestamp = 0;
+    private List<SortOption> sortOptions;
+    private int top;
+
+    private int aggFuncNum;
+    private int sortAggFuncNum;
+    private int sortFuncNum;
+
+    public GenericCoprocessorAggregateQuery(String serviceName, SearchCondition condition, AggregateCondition aggregateCondition, String metricName)
+        throws InstantiationException, IllegalAccessException {
+        this(serviceName, condition, aggregateCondition, metricName, null, null, null, 0);
+    }
+
+    public GenericCoprocessorAggregateQuery(String serviceName, SearchCondition condition,
+                                            AggregateCondition aggregateCondition, String metricName,
+                                            List<SortOption> sortOptions, List<AggregateFunctionType> sortFunctionTypes, List<String> sortFields, int top)
+        throws InstantiationException, IllegalAccessException {
+        checkNotNull(serviceName, "serviceName");
+        this.searchCondition = condition;
+        this.entityDef = EntityDefinitionManager.getEntityByServiceName(serviceName);
+        checkNotNull(entityDef, "EntityDefinition");
+        checkNotNull(entityDef, "GroupAggregateCondition");
+        this.aggregateCondition = aggregateCondition;
+        this.aggFuncNum = this.aggregateCondition.getAggregateFunctionTypes().size();
+        this.sortOptions = sortOptions;
+        this.sortFuncs = sortFunctionTypes;
+        this.sortFuncNum = this.sortOptions == null ? 0 : this.sortOptions.size();
+        this.sortFields = sortFields;
+        this.top = top;
+
+        if (serviceName.equals(GenericMetricEntity.GENERIC_METRIC_SERVICE)) {
+            if (LOG.isDebugEnabled()) {
+                LOG.debug("list metric aggregate query");
+            }
+            if (metricName == null || metricName.isEmpty()) {
+                throw new IllegalArgumentException("metricName should not be empty for metric list query");
+            }
+            if (!condition.getOutputFields().contains(GenericMetricEntity.VALUE_FIELD)) {
+                condition.getOutputFields().add(GenericMetricEntity.VALUE_FIELD);
+            }
+            this.prefix = metricName;
+        } else {
+            if (LOG.isDebugEnabled()) {
+                LOG.debug("list entity aggregate query");
+            }
+            this.prefix = entityDef.getPrefix();
+        }
+
+        // Add sort oriented aggregation functions into aggregateCondtion
+        if (this.sortOptions != null) {
+            // if sort for time series aggregation
+            if (this.aggregateCondition.isTimeSeries()) {
+                this.sortAggFuncNum = 0;
+                int index = 0;
+                for (SortOption sortOption : this.sortOptions) {
+                    if (!sortOption.isInGroupby()) {
+                        if (LOG.isDebugEnabled()) {
+                            LOG.debug("Add additional aggregation functions for sort options " + sortOption.toString() + " in index: " + (this.aggFuncNum + this.sortAggFuncNum));
+                        }
+                        AggregateFunctionType _sortFunc = this.sortFuncs.get(index);
+                        if (AggregateFunctionType.avg.equals(_sortFunc)) {
+                            this.aggregateCondition.getAggregateFunctionTypes().add(AggregateFunctionType.sum);
+                        } else {
+                            this.aggregateCondition.getAggregateFunctionTypes().add(_sortFunc);
+                        }
+                        this.aggregateCondition.getAggregateFields().add(this.sortFields.get(index));
+
+                        sortOption.setIndex(this.sortAggFuncNum);
+                        sortAggFuncNum++;
+                    }
+                    index++;
+                }
+            }
+        }
+    }
+
+
+    private void checkNotNull(Object o, String message) {
+        if (o == null) {
+            throw new IllegalArgumentException(message + " should not be null");
+        }
+    }
+
+    /**
+     * @see GenericCoprocessorAggregateQuery.TimeSeriesGroupAggregateQueryReader#result()
+     * @see GenericCoprocessorAggregateQuery.FlatGroupAggregateQueryReader#result()
+     */
+    @Override
+    public List result() throws Exception {
+        Date start = null;
+        Date end = null;
+        // shortcut to avoid read when pageSize=0
+        if (searchCondition.getPageSize() <= 0) {
+            return null;
+        }
+        // Process the time range if needed
+        if (entityDef.isTimeSeries()) {
+            start = DateTimeUtil.humanDateToDate(searchCondition.getStartTime());
+            end = DateTimeUtil.humanDateToDate(searchCondition.getEndTime());
+        } else {
+            start = DateTimeUtil.humanDateToDate(EntityConstants.FIXED_READ_START_HUMANTIME);
+            end = DateTimeUtil.humanDateToDate(EntityConstants.FIXED_READ_END_HUMANTIME);
+        }
+        // Generate the output qualifiers
+        final byte[][] outputQualifiers = HBaseInternalLogHelper.getOutputQualifiers(entityDef, searchCondition.getOutputFields());
+        GenericAggregateReader reader = new GenericAggregateReader(entityDef,
+            searchCondition.getPartitionValues(),
+            start, end, searchCondition.getFilter(), searchCondition.getStartRowkey(), outputQualifiers, this.prefix, this.aggregateCondition);
+        try {
+            if (LOG.isDebugEnabled()) {
+                LOG.debug("open and read group aggregate reader");
+            }
+            reader.open();
+            List result = buildGroupAggregateQueryReader(reader, this.aggregateCondition.isTimeSeries()).result();
+            if (result == null) {
+                throw new IOException("result is null");
+            }
+            this.firstTimestamp = reader.getFirstTimestamp();
+            this.lastTimestamp = reader.getLastTimestamp();
+            if (LOG.isDebugEnabled()) {
+                LOG.debug("finish read aggregated " + result.size() + " rows");
+            }
+            return result;
+        } catch (IOException ex) {
+            LOG.error("Fail reading aggregated results", ex);
+            throw ex;
+        } finally {
+            if (LOG.isDebugEnabled()) {
+                LOG.debug("Release HBase connection");
+            }
+            reader.close();
+        }
+    }
+    
+    /**
+     * Factory method for {@link GroupAggregateQueryReader}.
+     * <pre>
+     * {@link GroupAggregateQueryReader}
+     * |_ {@link FlatGroupAggregateQueryReader}
+     * |_ {@link TimeSeriesGroupAggregateQueryReader}
+     * </pre>
+     */
+    private GroupAggregateQueryReader buildGroupAggregateQueryReader(GenericAggregateReader reader, boolean isTimeSeries) throws IOException {
+        if (isTimeSeries) {
+            return new TimeSeriesGroupAggregateQueryReader(reader, this);
+        } else {
+            return new FlatGroupAggregateQueryReader(reader, this);
+        }
+    }
+
+    private abstract class GroupAggregateQueryReader {
+        protected final GenericAggregateReader reader;
+        protected final GenericCoprocessorAggregateQuery query;
+
+        public GroupAggregateQueryReader(GenericAggregateReader reader, GenericCoprocessorAggregateQuery query) {
+            this.reader = reader;
+            this.query = query;
+        }
+
+        public abstract <T> List<T> result() throws Exception;
+
+        protected Map<List<String>, List<Double>> keyValuesToMap(List<GroupbyKeyValue> entities) throws Exception {
+            Map<List<String>, List<Double>> aggResultMap = new HashMap<List<String>, List<Double>>();
+            try {
+                for (GroupbyKeyValue keyValue : entities) {
+                    List<String> key = new ArrayList<String>();
+                    for (BytesWritable bw : keyValue.getKey().getValue()) {
+                        key.add(new String(bw.copyBytes(), QueryConstants.CHARSET));
+                    }
+                    List<Double> value = new ArrayList<Double>();
+                    for (DoubleWritable wa : keyValue.getValue().getValue()) {
+                        value.add(wa.get());
+                    }
+                    aggResultMap.put(key, value);
+                }
+            } catch (UnsupportedEncodingException e) {
+                LOG.error(QueryConstants.CHARSET + " not support: " + e.getMessage(), e);
+            }
+            return aggResultMap;
+        }
+    }
+
+    private class FlatGroupAggregateQueryReader extends GroupAggregateQueryReader {
+        public FlatGroupAggregateQueryReader(GenericAggregateReader reader, GenericCoprocessorAggregateQuery query) {
+            super(reader, query);
+        }
+
+        @Override
+        public List<Map.Entry<List<String>, List<Double>>> result() throws Exception {
+            Map<List<String>, List<Double>> aggResultMap = this.keyValuesToMap(this.reader.read());
+            if (this.query.sortOptions == null) {
+                return new ArrayList<Map.Entry<List<String>, List<Double>>>(aggResultMap.entrySet());
+            }
+            if (LOG.isDebugEnabled()) {
+                LOG.debug("Flat sorting");
+            }
+            return PostFlatAggregateSort.sort(aggResultMap, this.query.sortOptions, this.query.top);
+        }
+    }
+
+    private class TimeSeriesGroupAggregateQueryReader extends GroupAggregateQueryReader {
+        private final Date start;
+        private final Date end;
+        private final int pointsNum;
+        private final int aggFuncNum;
+        private final List<SortOption> sortOptions;
+        private final List<AggregateFunctionType> sortFuncs;
+        private final int sortAggFuncNum;
+
+        public TimeSeriesGroupAggregateQueryReader(GenericAggregateReader reader, GenericCoprocessorAggregateQuery query) throws IOException {
+            super(reader, query);
+            try {
+                if (entityDef.isTimeSeries()) {
+                    this.start = DateTimeUtil.humanDateToDate(searchCondition.getStartTime());
+                    this.end = DateTimeUtil.humanDateToDate(searchCondition.getEndTime());
+                } else {
+                    start = DateTimeUtil.humanDateToDate(EntityConstants.FIXED_READ_START_HUMANTIME);
+                    end = DateTimeUtil.humanDateToDate(EntityConstants.FIXED_READ_END_HUMANTIME);
+                }
+                this.pointsNum = (int) ((end.getTime() - 1 - start.getTime()) / this.query.aggregateCondition.getIntervalMS() + 1);
+                this.aggFuncNum = this.query.aggFuncNum;
+                this.sortOptions = this.query.sortOptions;
+                this.sortFuncs = this.query.sortFuncs;
+                this.sortAggFuncNum = this.query.sortAggFuncNum;
+            } catch (Exception e) {
+                throw new IOException(e);
+            }
+        }
+
+        /**
+         * <h2>TimeSeriesReader result</h2>
+         * <ol>
+         * <li>generateTimeSeriesDataPoints()</li>
+         * <li>if not sort options, return generate time series data points</li>
+         * <li>if requiring sort, sort time series data points by order of flat aggregation</li>
+         * </ol>
+         * <h2>Time Series Sort Algorithms</h2>
+         * <ol>
+         * <li>Flat aggregate on grouped fields without time series bucket index</li>
+         * <li>Flat aggregated result according given sortOptions</li>
+         * <li>Sort Time Series Result according the same order of flat aggregated keys</li>
+         * </ol>.
+         */
+        @Override
+        public List<Map.Entry<List<String>, List<double[]>>> result() throws Exception {
+            List<GroupbyKeyValue> result = this.reader.read();
+
+            // aggregated data points only
+            Map<List<String>, List<double[]>> timeseriesDataPoints = convertToTimeSeriesDataPoints(result);
+            
+            // return time-series data points without sort
+            if (this.query.sortOptions == null) {
+                return new ArrayList<>(timeseriesDataPoints.entrySet());
+            }
+
+            LOG.info("Time series sorting");
+
+            // Time Series Sort Steps
+            // ======================
+            // 1. Flat aggregate on grouped fields without time series bucket index
+            // 2. Flat aggregated result according given sortOptions
+            // 3. Sort Time Series Result according flat aggregated keys' order
+
+            // 1. Flat aggregate on grouped fields without time series bucket index
+            AggregateResultCallbackImpl callback = new AggregateResultCallbackImpl(this.sortFuncs);
+            for (GroupbyKeyValue kv : result) {
+                ArrayList<BytesWritable> copykey = new ArrayList<BytesWritable>(kv.getKey().getValue());
+                // remove time series bucket index
+                copykey.remove(copykey.size() - 1);
+                GroupbyKey key = new GroupbyKey();
+
+                // [this.aggFuncNum,this.aggFuncNum + this.sortFuncNum)
+                GroupbyValue value = new GroupbyValue();
+                for (int i = this.aggFuncNum; i < this.aggFuncNum + this.sortAggFuncNum; i++) {
+                    value.add(kv.getValue().get(i));
+                    value.addMeta(kv.getValue().getMeta(i));
+                }
+                key.addAll(copykey);
+                GroupbyKeyValue keyValue = new GroupbyKeyValue(key, value);
+                callback.update(keyValue);
+            }
+            AggregateResult callbackResult = callback.result();
+            Map<List<String>, List<Double>> mapForSort = this.keyValuesToMap(callbackResult.getKeyValues());
+
+            // 2. Flat aggregated result according given sortOptions
+            // List<Map.Entry<List<String>, List<Double>>> flatSort = PostFlatAggregateSort.sort(mapForSort , this.sortOptions, Integer.MAX_VALUE);
+            // mapForSort = new HashMap<List<String>, List<Double>>();
+            // for(Map.Entry<List<String>, List<Double>> entry:flatSort){ mapForSort.put(entry.getKey(),entry.getValue()); }
+
+            // 3. Sort Time Series Result according flat aggregated keys' order
+            return TimeSeriesPostFlatAggregateSort.sort(mapForSort, timeseriesDataPoints, this.sortOptions, this.query.top);
+        }
+
+        /**
+         * Convert raw GroupbyKeyValue list into time-series data points hash map.
+         *
+         * @param result <code>List&lt;GroupbyKeyValue&gt;</code>
+         * @return Map&lt;List&lt;String&gt;,List&lt;double[]&gt;&gt;
+         * @throws Exception
+         */
+        private Map<List<String>, List<double[]>> convertToTimeSeriesDataPoints(List<GroupbyKeyValue> result) throws Exception {
+            Map<List<String>, List<Double>> aggResultMap = this.keyValuesToMap(result);
+            Map<List<String>, List<double[]>> timeseriesDataPoints = TimeSeriesAggregator.toMetric(aggResultMap, this.pointsNum, this.aggFuncNum);
+            return timeseriesDataPoints;
+        }
+    }
+
+    /**
+     * Get last / max timestamp.
+     *
+     * @return lastTimestamp
+     */
+    @Override
+    public long getLastTimestamp() {
+        return this.lastTimestamp;
+    }
+
+    /**
+     * Get first / min timestamp.
+     *
+     * @return firstTimestamp
+     */
+    @Override
+    public long getFirstTimeStamp() {
+        return this.firstTimestamp;
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/21187b55/eagle-core/eagle-query/eagle-storage-hbase/src/main/java/org/apache/eagle/storage/hbase/query/aggregate/GenericInMemoryAggregateQuery.java
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-query/eagle-storage-hbase/src/main/java/org/apache/eagle/storage/hbase/query/aggregate/GenericInMemoryAggregateQuery.java b/eagle-core/eagle-query/eagle-storage-hbase/src/main/java/org/apache/eagle/storage/hbase/query/aggregate/GenericInMemoryAggregateQuery.java
new file mode 100644
index 0000000..667390c
--- /dev/null
+++ b/eagle-core/eagle-query/eagle-storage-hbase/src/main/java/org/apache/eagle/storage/hbase/query/aggregate/GenericInMemoryAggregateQuery.java
@@ -0,0 +1,211 @@
+/*
+ * 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.eagle.storage.hbase.query.aggregate;
+
+import org.apache.eagle.common.DateTimeUtil;
+import org.apache.eagle.log.entity.*;
+import org.apache.eagle.log.entity.meta.EntityDefinition;
+import org.apache.eagle.log.entity.meta.EntityDefinitionManager;
+import org.apache.eagle.query.GenericQuery;
+import org.apache.eagle.query.aggregate.AggregateCondition;
+import org.apache.eagle.query.aggregate.AggregateFunctionType;
+import org.apache.eagle.query.aggregate.timeseries.*;
+
+import org.apache.commons.lang.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+public class GenericInMemoryAggregateQuery implements GenericQuery {
+    private static final Logger LOG = LoggerFactory.getLogger(GenericCoprocessorAggregateQuery.class);
+    private final List<AggregateFunctionType> sortFuncs;
+    private final List<String> sortFields;
+    private final String serviceName;
+
+    private EntityDefinition entityDef;
+    private SearchCondition searchCondition;
+    private AggregateCondition aggregateCondition;
+    private String prefix;
+    private long lastTimestamp = 0;
+    private long firstTimestamp = 0;
+    private List<SortOption> sortOptions;
+    private int top;
+
+    private int aggFuncNum;
+    private int sortAggFuncNum;
+    private int sortFuncNum;
+
+    public GenericInMemoryAggregateQuery(String serviceName, SearchCondition condition,
+                                         AggregateCondition aggregateCondition, String metricName,
+                                         List<SortOption> sortOptions, List<AggregateFunctionType> sortFunctionTypes, List<String> sortFields, int top)
+            throws InstantiationException, IllegalAccessException {
+        checkNotNull(serviceName, "serviceName");
+        this.searchCondition = condition;
+        this.entityDef = EntityDefinitionManager.getEntityByServiceName(serviceName);
+        checkNotNull(entityDef, "EntityDefinition");
+        checkNotNull(entityDef, "GroupAggregateCondition");
+        this.aggregateCondition = aggregateCondition;
+        this.aggFuncNum = this.aggregateCondition.getAggregateFunctionTypes().size();
+        this.sortOptions = sortOptions;
+        this.sortFuncs = sortFunctionTypes;
+        this.sortFuncNum = this.sortOptions == null ? 0 : this.sortOptions.size();
+        this.sortFields = sortFields;
+        this.top = top;
+        this.serviceName = serviceName;
+
+        if (serviceName.equals(GenericMetricEntity.GENERIC_METRIC_SERVICE)) {
+            if (LOG.isDebugEnabled()) {
+                LOG.debug("list metric aggregate query");
+            }
+            if (metricName == null || metricName.isEmpty()) {
+                throw new IllegalArgumentException("metricName should not be empty for metric list query");
+            }
+            if (!condition.getOutputFields().contains(GenericMetricEntity.VALUE_FIELD)) {
+                condition.getOutputFields().add(GenericMetricEntity.VALUE_FIELD);
+            }
+            this.prefix = metricName;
+        } else {
+            if (LOG.isDebugEnabled()) {
+                LOG.debug("list entity aggregate query");
+            }
+            this.prefix = entityDef.getPrefix();
+        }
+
+        // Add sort oriented aggregation functions into aggregateCondtion
+        if (this.sortOptions != null) {
+            // if sort for time series aggregation
+            if (this.aggregateCondition.isTimeSeries()) {
+                this.sortAggFuncNum = 0;
+                int index = 0;
+                for (SortOption sortOption : this.sortOptions) {
+                    if (!sortOption.isInGroupby()) {
+                        if (LOG.isDebugEnabled()) {
+                            LOG.debug("Add additional aggregation functions for sort options " + sortOption.toString() + " in index: " + (this.aggFuncNum + this.sortAggFuncNum));
+                        }
+                        AggregateFunctionType _sortFunc = this.sortFuncs.get(index);
+                        if (AggregateFunctionType.avg.equals(_sortFunc)) {
+                            this.aggregateCondition.getAggregateFunctionTypes().add(AggregateFunctionType.sum);
+                        } else {
+                            this.aggregateCondition.getAggregateFunctionTypes().add(_sortFunc);
+                        }
+                        this.aggregateCondition.getAggregateFields().add(this.sortFields.get(index));
+
+                        sortOption.setIndex(this.sortAggFuncNum);
+                        sortAggFuncNum++;
+                    }
+                    index++;
+                }
+            }
+        }
+    }
+
+    private void checkNotNull(Object o, String message) {
+        if (o == null) {
+            throw new IllegalArgumentException(message + " should not be null");
+        }
+    }
+
+    @Override
+    public <T> List<T> result() throws Exception {
+        // non time-series based aggregate query, not hierarchical
+        final List<String> groupbyFields = aggregateCondition.getGroupbyFields();
+        final List<String> aggregateFields = aggregateCondition.getAggregateFields();
+        final List<String> filterFields = searchCondition.getOutputFields();
+        final List<String> outputFields = new ArrayList<>();
+        if (groupbyFields != null) {
+            outputFields.addAll(groupbyFields);
+        }
+        if (filterFields != null) {
+            outputFields.addAll(filterFields);
+        }
+        if (sortFields != null) {
+            outputFields.addAll(sortFields);
+        }
+        outputFields.addAll(aggregateFields);
+        searchCondition.setOutputFields(outputFields);
+
+        if (searchCondition.isOutputAll()) {
+            LOG.info("Output: ALL");
+        } else {
+            LOG.info("Output: " + StringUtils.join(searchCondition.getOutputFields(), ", "));
+        }
+
+        if (!this.aggregateCondition.isTimeSeries()) {
+            FlatAggregator agg = new FlatAggregator(groupbyFields, aggregateCondition.getAggregateFunctionTypes(), aggregateCondition.getAggregateFields());
+            StreamReader reader = null;
+            if (this.entityDef.getMetricDefinition() == null) {
+                reader = new GenericEntityStreamReader(serviceName, searchCondition);
+            } else { // metric aggregation need metric reader
+                reader = new GenericMetricEntityDecompactionStreamReader(this.prefix, searchCondition);
+            }
+            reader.register(agg);
+            reader.readAsStream();
+            ArrayList<Map.Entry<List<String>, List<Double>>> obj = new ArrayList<>();
+            obj.addAll(agg.result().entrySet());
+            this.firstTimestamp = reader.getFirstTimestamp();
+            this.lastTimestamp = reader.getLastTimestamp();
+            if (this.sortOptions == null) {
+                return (List<T>) obj;
+            } else { // has sort options
+                return (List<T>) PostFlatAggregateSort.sort(agg.result(), this.sortOptions, top);
+            }
+        } else {
+            StreamReader reader;
+            if (entityDef.getMetricDefinition() == null) {
+                reader = new GenericEntityStreamReader(serviceName, searchCondition);
+            } else {
+                reader = new GenericMetricEntityDecompactionStreamReader(this.prefix, searchCondition);
+            }
+            TimeSeriesAggregator tsAgg = new TimeSeriesAggregator(groupbyFields,
+                aggregateCondition.getAggregateFunctionTypes(), aggregateFields,
+                    DateTimeUtil.humanDateToDate(searchCondition.getStartTime()).getTime(),
+                DateTimeUtil.humanDateToDate(searchCondition.getEndTime()).getTime(), aggregateCondition.getIntervalMS() * 60 * 1000);
+            reader.register(tsAgg);
+
+            // for sorting
+            FlatAggregator sortAgg = null;
+            if (sortOptions != null) {
+                sortAgg = new FlatAggregator(groupbyFields, sortFuncs, sortFields);
+                reader.register(sortAgg);
+            }
+            reader.readAsStream();
+            ArrayList<Map.Entry<List<String>, List<double[]>>> obj = new ArrayList<Map.Entry<List<String>, List<double[]>>>();
+            obj.addAll(tsAgg.getMetric().entrySet());
+
+            this.firstTimestamp = reader.getFirstTimestamp();
+            this.lastTimestamp = reader.getLastTimestamp();
+            if (sortOptions == null) {
+                return (List<T>) obj;
+            } else { // has sort options
+                return (List<T>) TimeSeriesPostFlatAggregateSort.sort(sortAgg.result(), tsAgg.getMetric(), this.sortOptions, top);
+            }
+        }
+    }
+
+    @Override
+    public long getLastTimestamp() {
+        return this.firstTimestamp;
+    }
+
+    @Override
+    public long getFirstTimeStamp() {
+        return this.lastTimestamp;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/21187b55/eagle-core/eagle-query/eagle-storage-hbase/src/main/java/org/apache/eagle/storage/hbase/query/coprocessor/AggregateClient.java
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-query/eagle-storage-hbase/src/main/java/org/apache/eagle/storage/hbase/query/coprocessor/AggregateClient.java b/eagle-core/eagle-query/eagle-storage-hbase/src/main/java/org/apache/eagle/storage/hbase/query/coprocessor/AggregateClient.java
index cd6a5b9..828da44 100755
--- a/eagle-core/eagle-query/eagle-storage-hbase/src/main/java/org/apache/eagle/storage/hbase/query/coprocessor/AggregateClient.java
+++ b/eagle-core/eagle-query/eagle-storage-hbase/src/main/java/org/apache/eagle/storage/hbase/query/coprocessor/AggregateClient.java
@@ -25,35 +25,27 @@ import java.io.IOException;
 import java.util.List;
 
 /**
- * Coprocessor-based Aggregation Universal Client Interface
+ * Coprocessor-based Aggregation Universal Client Interface.
  *
  * <h2>Flat or RAW Aggregation:</h2>
- * <pre>
- * AggregateResult aggregate( HTableInterface table, String serviceName, Scan scan, List<String> groupbyFields, List<AggregateFunctionType> aggregateFuncTypes, List<String> aggregatedFields) throws IOException
- * </pre>
- *
+ * @see #aggregate(HTableInterface, EntityDefinition, Scan, List, List, List)
  * <h2>Time Series Aggregation:</h2>
- * <pre>
- * AggregateResult aggregate(HTableInterface table, String serviceName, Scan scan, List<String> groupbyFields, List<AggregateFunctionType> aggregateFuncTypes, List<String> aggregatedFields, boolean timeSeries, long startTime, long endTime, long intervalMin) throws IOException
- * </pre>
- * @since : 11/3/14,2014
+ * @see #aggregate(HTableInterface, EntityDefinition, Scan, List, List, List, boolean, long, long, long)
+ *
  */
-public interface AggregateClient
-{
+public interface AggregateClient {
 
-	/**
-	 * Flat Aggregation
-	 *
-	 *
-	 * @param table                   HTable connections
-	 * @param scan                    HBase Scan
-	 * @param groupbyFields           Grouped by fields name
-	 * @param aggregateFuncTypes      Aggregate function types
-	 * @param aggregatedFields        Aggregate field names
-	 * @return                        Return AggregateResult
-	 * @throws Exception
-	 */
-	AggregateResult aggregate(final HTableInterface table,                            // HTable connections
+    /**
+     * Flat Aggregation.
+     *
+     * @param table              HTable connections
+     * @param scan               HBase Scan
+     * @param groupbyFields      Grouped by fields name
+     * @param aggregateFuncTypes Aggregate function types
+     * @param aggregatedFields   Aggregate field names
+     * @return Return AggregateResult
+     */
+    AggregateResult aggregate(final HTableInterface table,                            // HTable connections
                               final EntityDefinition entityDefinition,                               // Eagle service name
                               final Scan scan,                                        // HBase Scan
                               final List<String> groupbyFields,                       // Grouped by fields name
@@ -61,21 +53,20 @@ public interface AggregateClient
                               final List<String> aggregatedFields                     // Aggregate field names
     ) throws IOException;
 
-	/**
-	 * Time Series Aggregation
-	 *
-	 * @param table                   HTable connections
-	 * @param entityDefinition        Eagle EntityDefinition
-	 * @param scan                    HBase Scan
-	 * @param groupbyFields           Grouped by fields name
-	 * @param aggregateFuncTypes      Aggregate function types
-	 * @param aggregatedFields        Aggregate field names
-	 * @param timeSeries              Is time series aggregations?
-	 * @param intervalMin             The interval in minutes if it's time series aggregation
-	 * @return                        Return AggregateResult
-	 * @throws Exception
-	 */
-	AggregateResult aggregate(final HTableInterface table,                                // HTable connections
+    /**
+     * Time Series Aggregation.
+     *
+     * @param table              HTable connections
+     * @param entityDefinition   Eagle EntityDefinition
+     * @param scan               HBase Scan
+     * @param groupbyFields      Grouped by fields name
+     * @param aggregateFuncTypes Aggregate function types
+     * @param aggregatedFields   Aggregate field names
+     * @param timeSeries         Is time series aggregations?
+     * @param intervalMin        The interval in minutes if it's time series aggregation
+     * @return Return AggregateResult
+     */
+    AggregateResult aggregate(final HTableInterface table,                                // HTable connections
                               final EntityDefinition entityDefinition,                               // Eagle service name
                               final Scan scan,                                        // HBase Scan
                               final List<String> groupbyFields,                       // Grouped by fields name

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/21187b55/eagle-core/eagle-query/eagle-storage-hbase/src/main/java/org/apache/eagle/storage/hbase/query/coprocessor/AggregateProtocol.java
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-query/eagle-storage-hbase/src/main/java/org/apache/eagle/storage/hbase/query/coprocessor/AggregateProtocol.java b/eagle-core/eagle-query/eagle-storage-hbase/src/main/java/org/apache/eagle/storage/hbase/query/coprocessor/AggregateProtocol.java
index fe2c414..eb85046 100755
--- a/eagle-core/eagle-query/eagle-storage-hbase/src/main/java/org/apache/eagle/storage/hbase/query/coprocessor/AggregateProtocol.java
+++ b/eagle-core/eagle-query/eagle-storage-hbase/src/main/java/org/apache/eagle/storage/hbase/query/coprocessor/AggregateProtocol.java
@@ -29,35 +29,32 @@ import java.util.List;
  */
 public interface AggregateProtocol {
 
-	/**
-	 *
-	 *
-	 * @param entityDefinition
-	 * @param scan
-	 * @param groupbyFields
-	 * @param aggregateFuncTypes
-	 * @param aggregatedFields
-	 * @return AggregateResult
-	 * @throws java.io.IOException
-	 */
-	AggregateResult aggregate(EntityDefinition entityDefinition,
+    /**
+     * @param entityDefinition
+     * @param scan
+     * @param groupbyFields
+     * @param aggregateFuncTypes
+     * @param aggregatedFields
+     * @return AggregateResult
+     * @throws java.io.IOException
+     */
+    AggregateResult aggregate(EntityDefinition entityDefinition,
                               Scan scan,
                               List<String> groupbyFields,
                               List<byte[]> aggregateFuncTypes,
                               List<String> aggregatedFields) throws IOException;
 
-	/**
-	 *
-	 * @param entityDefinition
-	 * @param scan
-	 * @param groupbyFields
-	 * @param aggregateFuncTypes
-	 * @param aggregatedFields
-	 * @param intervalMin
-	 * @return AggregateResult
-	 * @throws java.io.IOException
-	 */
-	AggregateResult aggregate(EntityDefinition entityDefinition,
+    /**
+     * @param entityDefinition
+     * @param scan
+     * @param groupbyFields
+     * @param aggregateFuncTypes
+     * @param aggregatedFields
+     * @param intervalMin
+     * @return AggregateResult
+     * @throws java.io.IOException
+     */
+    AggregateResult aggregate(EntityDefinition entityDefinition,
                               Scan scan,
                               List<String> groupbyFields,
                               List<byte[]> aggregateFuncTypes,


[10/52] [abbrv] incubator-eagle git commit: [EAGLE-460] Convert MR running app with new app framework

Posted by yo...@apache.org.
[EAGLE-460] Convert MR running app with new app framework

https://issues.apache.org/jira/browse/EAGLE-460

Author: Hao Chen <ha...@apache.org>

Closes #381 from haoch/EAGLE-460.


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

Branch: refs/heads/master
Commit: b54a63e0497f6dbca6203fc3024ca649bc767f25
Parents: 0b852cb
Author: Hao Chen <ha...@apache.org>
Authored: Wed Aug 24 21:25:57 2016 +0800
Committer: Hao Chen <ha...@apache.org>
Committed: Wed Aug 24 21:25:57 2016 +0800

----------------------------------------------------------------------
 .../queue/model/scheduler/SchedulerInfo.java    |  86 ++++-----
 .../queue/model/scheduler/SchedulerWrapper.java |  16 +-
 .../hadoop/queue/model/scheduler/User.java      |  74 ++++----
 .../storm/HadoopQueueMetricPersistBolt.java     |  31 ++--
 .../storm/HadoopQueueRunningExtractor.java      |  23 +--
 .../apache/eagle/app/jpm/JPMApplication.java    |  13 +-
 eagle-jpm/eagle-jpm-mr-running/pom.xml          |   5 +
 .../jpm/mr/running/MRRunningJobApplication.java |  77 ++++++++
 .../MRRunningJobApplicationProvider.java        |  26 +++
 .../jpm/mr/running/MRRunningJobConfig.java      | 156 +++++++++++++++++
 .../eagle/jpm/mr/running/MRRunningJobMain.java  |  75 +-------
 .../running/config/MRRunningConfigManager.java  | 151 ----------------
 .../parser/MRJobEntityCreationHandler.java      |   6 +-
 .../jpm/mr/running/parser/MRJobParser.java      |  34 ++--
 .../mr/running/recover/MRRunningJobManager.java |   8 +-
 .../running/storm/MRRunningJobFetchSpout.java   |  30 ++--
 .../mr/running/storm/MRRunningJobParseBolt.java |  34 ++--
 ....running.MRRunningJobApplicationProvider.xml | 175 +++++++++++++++++++
 ...org.apache.eagle.app.spi.ApplicationProvider |  16 ++
 .../src/main/resources/application.conf         |   8 +-
 .../MRRunningJobApplicationProviderTest.java    |  35 ++++
 .../mr/running/MRRunningJobApplicationTest.java |  27 +++
 .../service/jpm/MRJobExecutionResource.java     |  34 ++--
 .../storm/SparkRunningJobFetchSpout.java        |  14 +-
 .../running/storm/SparkRunningJobParseBolt.java |   5 +-
 25 files changed, 733 insertions(+), 426 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/b54a63e0/eagle-jpm/eagle-hadoop-queue/src/main/java/org/apache/eagle/hadoop/queue/model/scheduler/SchedulerInfo.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-hadoop-queue/src/main/java/org/apache/eagle/hadoop/queue/model/scheduler/SchedulerInfo.java b/eagle-jpm/eagle-hadoop-queue/src/main/java/org/apache/eagle/hadoop/queue/model/scheduler/SchedulerInfo.java
index 8ed7745..8c51600 100644
--- a/eagle-jpm/eagle-hadoop-queue/src/main/java/org/apache/eagle/hadoop/queue/model/scheduler/SchedulerInfo.java
+++ b/eagle-jpm/eagle-hadoop-queue/src/main/java/org/apache/eagle/hadoop/queue/model/scheduler/SchedulerInfo.java
@@ -21,63 +21,63 @@ package org.apache.eagle.hadoop.queue.model.scheduler;
 import org.codehaus.jackson.annotate.JsonIgnoreProperties;
 import org.codehaus.jackson.map.annotate.JsonSerialize;
 
-@JsonSerialize(include=JsonSerialize.Inclusion.NON_NULL)
+@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
 @JsonIgnoreProperties(ignoreUnknown = true)
 public class SchedulerInfo {
-	private String type;
-	private double capacity;
-	private double usedCapacity;
-	private double maxCapacity;
-	private String queueName;
-	private Queues queues;
+    private String type;
+    private double capacity;
+    private double usedCapacity;
+    private double maxCapacity;
+    private String queueName;
+    private Queues queues;
 
-	public Queues getQueues() {
-		return queues;
-	}
+    public Queues getQueues() {
+        return queues;
+    }
 
-	public void setQueues(Queues queues) {
-		this.queues = queues;
-	}
+    public void setQueues(Queues queues) {
+        this.queues = queues;
+    }
 
 
-	public double getUsedCapacity() {
-		return usedCapacity;
-	}
+    public double getUsedCapacity() {
+        return usedCapacity;
+    }
 
-	public void setUsedCapacity(double usedCapacity) {
-		this.usedCapacity = usedCapacity;
-	}
+    public void setUsedCapacity(double usedCapacity) {
+        this.usedCapacity = usedCapacity;
+    }
 
-	public String getType() {
-		return type;
-	}
+    public String getType() {
+        return type;
+    }
 
-	public void setType(String type) {
-		this.type = type;
-	}
+    public void setType(String type) {
+        this.type = type;
+    }
 
-	public double getCapacity() {
-		return capacity;
-	}
+    public double getCapacity() {
+        return capacity;
+    }
 
-	public void setCapacity(double capacity) {
-		this.capacity = capacity;
-	}
+    public void setCapacity(double capacity) {
+        this.capacity = capacity;
+    }
 
-	public double getMaxCapacity() {
-		return maxCapacity;
-	}
+    public double getMaxCapacity() {
+        return maxCapacity;
+    }
 
-	public void setMaxCapacity(double maxCapacity) {
-		this.maxCapacity = maxCapacity;
-	}
+    public void setMaxCapacity(double maxCapacity) {
+        this.maxCapacity = maxCapacity;
+    }
 
-	public String getQueueName() {
-		return queueName;
-	}
+    public String getQueueName() {
+        return queueName;
+    }
 
-	public void setQueueName(String queueName) {
-		this.queueName = queueName;
-	}
+    public void setQueueName(String queueName) {
+        this.queueName = queueName;
+    }
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/b54a63e0/eagle-jpm/eagle-hadoop-queue/src/main/java/org/apache/eagle/hadoop/queue/model/scheduler/SchedulerWrapper.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-hadoop-queue/src/main/java/org/apache/eagle/hadoop/queue/model/scheduler/SchedulerWrapper.java b/eagle-jpm/eagle-hadoop-queue/src/main/java/org/apache/eagle/hadoop/queue/model/scheduler/SchedulerWrapper.java
index f181f2f..61b3685 100644
--- a/eagle-jpm/eagle-hadoop-queue/src/main/java/org/apache/eagle/hadoop/queue/model/scheduler/SchedulerWrapper.java
+++ b/eagle-jpm/eagle-hadoop-queue/src/main/java/org/apache/eagle/hadoop/queue/model/scheduler/SchedulerWrapper.java
@@ -21,16 +21,16 @@ package org.apache.eagle.hadoop.queue.model.scheduler;
 import org.codehaus.jackson.annotate.JsonIgnoreProperties;
 import org.codehaus.jackson.map.annotate.JsonSerialize;
 
-@JsonSerialize(include=JsonSerialize.Inclusion.NON_NULL)
+@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
 @JsonIgnoreProperties(ignoreUnknown = true)
 public class SchedulerWrapper {
-	public Scheduler getScheduler() {
-		return scheduler;
-	}
+    public Scheduler getScheduler() {
+        return scheduler;
+    }
 
-	public void setScheduler(Scheduler scheduler) {
-		this.scheduler = scheduler;
-	}
+    public void setScheduler(Scheduler scheduler) {
+        this.scheduler = scheduler;
+    }
 
-	private Scheduler scheduler;
+    private Scheduler scheduler;
 }

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/b54a63e0/eagle-jpm/eagle-hadoop-queue/src/main/java/org/apache/eagle/hadoop/queue/model/scheduler/User.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-hadoop-queue/src/main/java/org/apache/eagle/hadoop/queue/model/scheduler/User.java b/eagle-jpm/eagle-hadoop-queue/src/main/java/org/apache/eagle/hadoop/queue/model/scheduler/User.java
index bac9eb2..7bddf71 100644
--- a/eagle-jpm/eagle-hadoop-queue/src/main/java/org/apache/eagle/hadoop/queue/model/scheduler/User.java
+++ b/eagle-jpm/eagle-hadoop-queue/src/main/java/org/apache/eagle/hadoop/queue/model/scheduler/User.java
@@ -21,43 +21,43 @@ package org.apache.eagle.hadoop.queue.model.scheduler;
 import org.codehaus.jackson.annotate.JsonIgnoreProperties;
 import org.codehaus.jackson.map.annotate.JsonSerialize;
 
-@JsonSerialize(include=JsonSerialize.Inclusion.NON_NULL)
+@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
 @JsonIgnoreProperties(ignoreUnknown = true)
 public class User {
-	private String username;
-	private ResourcesUsed resourcesUsed;
-	private int numPendingApplications;
-	private int numActiveApplications;
-
-	public String getUsername() {
-		return username;
-	}
-
-	public void setUsername(String username) {
-		this.username = username;
-	}
-
-	public ResourcesUsed getResourcesUsed() {
-		return resourcesUsed;
-	}
-
-	public void setResourcesUsed(ResourcesUsed resourcesUsed) {
-		this.resourcesUsed = resourcesUsed;
-	}
-
-	public int getNumPendingApplications() {
-		return numPendingApplications;
-	}
-
-	public void setNumPendingApplications(int numPendingApplications) {
-		this.numPendingApplications = numPendingApplications;
-	}
-
-	public int getNumActiveApplications() {
-		return numActiveApplications;
-	}
-
-	public void setNumActiveApplications(int numActiveApplications) {
-		this.numActiveApplications = numActiveApplications;
-	}
+    private String username;
+    private ResourcesUsed resourcesUsed;
+    private int numPendingApplications;
+    private int numActiveApplications;
+
+    public String getUsername() {
+        return username;
+    }
+
+    public void setUsername(String username) {
+        this.username = username;
+    }
+
+    public ResourcesUsed getResourcesUsed() {
+        return resourcesUsed;
+    }
+
+    public void setResourcesUsed(ResourcesUsed resourcesUsed) {
+        this.resourcesUsed = resourcesUsed;
+    }
+
+    public int getNumPendingApplications() {
+        return numPendingApplications;
+    }
+
+    public void setNumPendingApplications(int numPendingApplications) {
+        this.numPendingApplications = numPendingApplications;
+    }
+
+    public int getNumActiveApplications() {
+        return numActiveApplications;
+    }
+
+    public void setNumActiveApplications(int numActiveApplications) {
+        this.numActiveApplications = numActiveApplications;
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/b54a63e0/eagle-jpm/eagle-hadoop-queue/src/main/java/org/apache/eagle/hadoop/queue/storm/HadoopQueueMetricPersistBolt.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-hadoop-queue/src/main/java/org/apache/eagle/hadoop/queue/storm/HadoopQueueMetricPersistBolt.java b/eagle-jpm/eagle-hadoop-queue/src/main/java/org/apache/eagle/hadoop/queue/storm/HadoopQueueMetricPersistBolt.java
index c99ea41..db61841 100644
--- a/eagle-jpm/eagle-hadoop-queue/src/main/java/org/apache/eagle/hadoop/queue/storm/HadoopQueueMetricPersistBolt.java
+++ b/eagle-jpm/eagle-hadoop-queue/src/main/java/org/apache/eagle/hadoop/queue/storm/HadoopQueueMetricPersistBolt.java
@@ -18,19 +18,20 @@
 
 package org.apache.eagle.hadoop.queue.storm;
 
-import backtype.storm.task.OutputCollector;
-import backtype.storm.task.TopologyContext;
-import backtype.storm.topology.OutputFieldsDeclarer;
-import backtype.storm.topology.base.BaseRichBolt;
-import backtype.storm.tuple.Tuple;
-import com.typesafe.config.Config;
-import org.apache.eagle.hadoop.queue.model.scheduler.RunningQueueAPIEntity;
 import org.apache.eagle.hadoop.queue.common.HadoopClusterConstants;
+import org.apache.eagle.hadoop.queue.model.scheduler.RunningQueueAPIEntity;
 import org.apache.eagle.log.entity.GenericMetricEntity;
 import org.apache.eagle.log.entity.GenericServiceAPIResponseEntity;
 import org.apache.eagle.service.client.EagleServiceConnector;
 import org.apache.eagle.service.client.IEagleServiceClient;
 import org.apache.eagle.service.client.impl.EagleServiceClientImpl;
+
+import backtype.storm.task.OutputCollector;
+import backtype.storm.task.TopologyContext;
+import backtype.storm.topology.OutputFieldsDeclarer;
+import backtype.storm.topology.base.BaseRichBolt;
+import backtype.storm.tuple.Tuple;
+import com.typesafe.config.Config;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -39,7 +40,7 @@ import java.util.Map;
 
 public class HadoopQueueMetricPersistBolt extends BaseRichBolt {
 
-    private final static Logger LOG = LoggerFactory.getLogger(HadoopQueueMetricPersistBolt.class);
+    private static final Logger LOG = LoggerFactory.getLogger(HadoopQueueMetricPersistBolt.class);
 
     private Config config;
     private IEagleServiceClient client;
@@ -65,7 +66,7 @@ public class HadoopQueueMetricPersistBolt extends BaseRichBolt {
         if (dataType.equalsIgnoreCase(HadoopClusterConstants.DataType.METRIC.toString())) {
             List<GenericMetricEntity> metrics = (List<GenericMetricEntity>) data;
             writeMetrics(metrics);
-         } else if (dataType.equalsIgnoreCase(HadoopClusterConstants.DataType.ENTITY.toString())) {
+        } else if (dataType.equalsIgnoreCase(HadoopClusterConstants.DataType.ENTITY.toString())) {
             List<RunningQueueAPIEntity> entities = (List<RunningQueueAPIEntity>) data;
             writeEntities(entities);
         }
@@ -77,12 +78,12 @@ public class HadoopQueueMetricPersistBolt extends BaseRichBolt {
 
     }
 
-    private void writeEntities(List<RunningQueueAPIEntity> entities){
+    private void writeEntities(List<RunningQueueAPIEntity> entities) {
         try {
             GenericServiceAPIResponseEntity response = client.create(entities);
-            if(!response.isSuccess()){
+            if (!response.isSuccess()) {
                 LOG.error("Got exception from eagle service: " + response.getException());
-            }else{
+            } else {
                 LOG.info("Successfully wrote " + entities.size() + " RunningQueueAPIEntity entities");
             }
         } catch (Exception e) {
@@ -91,12 +92,12 @@ public class HadoopQueueMetricPersistBolt extends BaseRichBolt {
         entities.clear();
     }
 
-    private void writeMetrics(List<GenericMetricEntity> entities){
+    private void writeMetrics(List<GenericMetricEntity> entities) {
         try {
             GenericServiceAPIResponseEntity response = client.create(entities);
-            if(response.isSuccess()){
+            if (response.isSuccess()) {
                 LOG.info("Successfully wrote " + entities.size() + " GenericMetricEntity entities");
-            }else{
+            } else {
                 LOG.error(response.getException());
             }
         } catch (Exception e) {

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/b54a63e0/eagle-jpm/eagle-hadoop-queue/src/main/java/org/apache/eagle/hadoop/queue/storm/HadoopQueueRunningExtractor.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-hadoop-queue/src/main/java/org/apache/eagle/hadoop/queue/storm/HadoopQueueRunningExtractor.java b/eagle-jpm/eagle-hadoop-queue/src/main/java/org/apache/eagle/hadoop/queue/storm/HadoopQueueRunningExtractor.java
index 3c4391b..ef0c762 100644
--- a/eagle-jpm/eagle-hadoop-queue/src/main/java/org/apache/eagle/hadoop/queue/storm/HadoopQueueRunningExtractor.java
+++ b/eagle-jpm/eagle-hadoop-queue/src/main/java/org/apache/eagle/hadoop/queue/storm/HadoopQueueRunningExtractor.java
@@ -18,16 +18,17 @@
 
 package org.apache.eagle.hadoop.queue.storm;
 
-import backtype.storm.spout.SpoutOutputCollector;
-import com.typesafe.config.Config;
-import org.apache.eagle.hadoop.queue.crawler.ClusterMetricsCrawler;
-import org.apache.eagle.hadoop.queue.crawler.RunningAppsCrawler;
-import org.apache.eagle.hadoop.queue.crawler.SchedulerInfoCrawler;
 import org.apache.eagle.hadoop.queue.common.HadoopYarnResourceUtils;
 import org.apache.eagle.hadoop.queue.common.YarnClusterResourceURLBuilder;
 import org.apache.eagle.hadoop.queue.common.YarnURLSelectorImpl;
+import org.apache.eagle.hadoop.queue.crawler.ClusterMetricsCrawler;
+import org.apache.eagle.hadoop.queue.crawler.RunningAppsCrawler;
+import org.apache.eagle.hadoop.queue.crawler.SchedulerInfoCrawler;
 import org.apache.eagle.jpm.util.Constants;
 import org.apache.eagle.jpm.util.resourcefetch.ha.HAURLSelector;
+
+import backtype.storm.spout.SpoutOutputCollector;
+import com.typesafe.config.Config;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -38,10 +39,10 @@ import java.util.concurrent.*;
 
 public class HadoopQueueRunningExtractor {
 
-    private final static Logger LOGGER = LoggerFactory.getLogger(HadoopQueueRunningExtractor.class);
-    private final static int MAX_NUM_THREADS = 10;
-    private final static int MAX_WAIT_TIME = 10;
-    private final static String DEFAULT_SITE = "sandbox";
+    private static final Logger LOGGER = LoggerFactory.getLogger(HadoopQueueRunningExtractor.class);
+    private static final int MAX_NUM_THREADS = 10;
+    private static final int MAX_WAIT_TIME = 10;
+    private static final String DEFAULT_SITE = "sandbox";
 
     private String site;
     private String urlBases;
@@ -53,10 +54,10 @@ public class HadoopQueueRunningExtractor {
     public HadoopQueueRunningExtractor(Config eagleConf, SpoutOutputCollector collector) {
         site = HadoopYarnResourceUtils.getConfigValue(eagleConf, "eagleProps.site", DEFAULT_SITE);
         urlBases = HadoopYarnResourceUtils.getConfigValue(eagleConf, "dataSourceConfig.RMEndPoints", "");
-        if(urlBases == null){
+        if (urlBases == null) {
             throw new IllegalArgumentException(site + ".baseurl is null");
         }
-        String [] urls = urlBases.split(",");
+        String[] urls = urlBases.split(",");
         urlSelector = new YarnURLSelectorImpl(urls, Constants.CompressionType.GZIP);
         executorService = Executors.newFixedThreadPool(MAX_NUM_THREADS);
         this.collector = collector;

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/b54a63e0/eagle-jpm/eagle-jpm-app/src/main/java/org/apache/eagle/app/jpm/JPMApplication.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-app/src/main/java/org/apache/eagle/app/jpm/JPMApplication.java b/eagle-jpm/eagle-jpm-app/src/main/java/org/apache/eagle/app/jpm/JPMApplication.java
index d084008..68b7eff 100644
--- a/eagle-jpm/eagle-jpm-app/src/main/java/org/apache/eagle/app/jpm/JPMApplication.java
+++ b/eagle-jpm/eagle-jpm-app/src/main/java/org/apache/eagle/app/jpm/JPMApplication.java
@@ -16,6 +16,9 @@
  */
 package org.apache.eagle.app.jpm;
 
+import org.apache.eagle.app.StormApplication;
+import org.apache.eagle.app.environment.impl.StormEnvironment;
+
 import backtype.storm.generated.StormTopology;
 import backtype.storm.spout.SpoutOutputCollector;
 import backtype.storm.task.TopologyContext;
@@ -24,8 +27,6 @@ import backtype.storm.topology.TopologyBuilder;
 import backtype.storm.topology.base.BaseRichSpout;
 import backtype.storm.tuple.Fields;
 import com.typesafe.config.Config;
-import org.apache.eagle.app.StormApplication;
-import org.apache.eagle.app.environment.impl.StormEnvironment;
 
 import java.util.Arrays;
 import java.util.Map;
@@ -41,16 +42,16 @@ public class JPMApplication extends StormApplication {
     }
 
     private class RandomEventSpout extends BaseRichSpout {
-        private SpoutOutputCollector _collector;
+        private SpoutOutputCollector collector;
         @Override
         public void open(Map map, TopologyContext topologyContext, SpoutOutputCollector spoutOutputCollector) {
-            _collector = spoutOutputCollector;
+            collector = spoutOutputCollector;
         }
 
         @Override
         public void nextTuple() {
-            _collector.emit(Arrays.asList("disk.usage",System.currentTimeMillis(),"host_1",56.7));
-            _collector.emit(Arrays.asList("cpu.usage",System.currentTimeMillis(),"host_2",99.8));
+            collector.emit(Arrays.asList("disk.usage",System.currentTimeMillis(),"host_1",56.7));
+            collector.emit(Arrays.asList("cpu.usage",System.currentTimeMillis(),"host_2",99.8));
         }
 
         @Override

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/b54a63e0/eagle-jpm/eagle-jpm-mr-running/pom.xml
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-mr-running/pom.xml b/eagle-jpm/eagle-jpm-mr-running/pom.xml
index 414a221..d4ca4ce 100644
--- a/eagle-jpm/eagle-jpm-mr-running/pom.xml
+++ b/eagle-jpm/eagle-jpm-mr-running/pom.xml
@@ -80,6 +80,11 @@
             <artifactId>eagle-jpm-entity</artifactId>
             <version>${project.version}</version>
         </dependency>
+        <dependency>
+            <groupId>org.apache.eagle</groupId>
+            <artifactId>eagle-app-base</artifactId>
+            <version>${project.version}</version>
+        </dependency>
     </dependencies>
     <build>
         <resources>

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/b54a63e0/eagle-jpm/eagle-jpm-mr-running/src/main/java/org/apache/eagle/jpm/mr/running/MRRunningJobApplication.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-mr-running/src/main/java/org/apache/eagle/jpm/mr/running/MRRunningJobApplication.java b/eagle-jpm/eagle-jpm-mr-running/src/main/java/org/apache/eagle/jpm/mr/running/MRRunningJobApplication.java
new file mode 100644
index 0000000..21ee1d9
--- /dev/null
+++ b/eagle-jpm/eagle-jpm-mr-running/src/main/java/org/apache/eagle/jpm/mr/running/MRRunningJobApplication.java
@@ -0,0 +1,77 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.eagle.jpm.mr.running;
+
+import backtype.storm.generated.StormTopology;
+import backtype.storm.topology.TopologyBuilder;
+import backtype.storm.tuple.Fields;
+import com.typesafe.config.Config;
+import org.apache.eagle.app.StormApplication;
+import org.apache.eagle.app.environment.impl.StormEnvironment;
+import org.apache.eagle.jpm.mr.running.storm.MRRunningJobFetchSpout;
+import org.apache.eagle.jpm.mr.running.storm.MRRunningJobParseBolt;
+import org.apache.eagle.jpm.util.Constants;
+
+import java.util.List;
+
+public class MRRunningJobApplication extends StormApplication {
+    @Override
+    public StormTopology execute(Config config, StormEnvironment environment) {
+        //1. trigger init conf
+        MRRunningJobConfig mrRunningJobConfig = MRRunningJobConfig.getInstance(config);
+
+        List<String> confKeyKeys = mrRunningJobConfig.getConfig().getStringList("MRConfigureKeys.jobConfigKey");
+        confKeyKeys.add(Constants.JobConfiguration.CASCADING_JOB);
+        confKeyKeys.add(Constants.JobConfiguration.HIVE_JOB);
+        confKeyKeys.add(Constants.JobConfiguration.PIG_JOB);
+        confKeyKeys.add(Constants.JobConfiguration.SCOOBI_JOB);
+        confKeyKeys.add(0, mrRunningJobConfig.getConfig().getString("MRConfigureKeys.jobNameKey"));
+
+        //2. init topology
+        TopologyBuilder topologyBuilder = new TopologyBuilder();
+        String spoutName = "mrRunningJobFetchSpout";
+        String boltName = "mrRunningJobParseBolt";
+        int parallelism = mrRunningJobConfig.getConfig().getInt("envContextConfig.parallelismConfig." + spoutName);
+        int tasks = mrRunningJobConfig.getConfig().getInt("envContextConfig.tasks." + spoutName);
+        if (parallelism > tasks) {
+            parallelism = tasks;
+        }
+        topologyBuilder.setSpout(
+            spoutName,
+            new MRRunningJobFetchSpout(
+                mrRunningJobConfig.getJobExtractorConfig(),
+                mrRunningJobConfig.getEndpointConfig(),
+                mrRunningJobConfig.getZkStateConfig()),
+            parallelism
+        ).setNumTasks(tasks);
+
+        parallelism = mrRunningJobConfig.getConfig().getInt("envContextConfig.parallelismConfig." + boltName);
+        tasks = mrRunningJobConfig.getConfig().getInt("envContextConfig.tasks." + boltName);
+        if (parallelism > tasks) {
+            parallelism = tasks;
+        }
+        topologyBuilder.setBolt(boltName,
+            new MRRunningJobParseBolt(
+                mrRunningJobConfig.getEagleServiceConfig(),
+                mrRunningJobConfig.getEndpointConfig(),
+                mrRunningJobConfig.getJobExtractorConfig(),
+                mrRunningJobConfig.getZkStateConfig(),
+                confKeyKeys),
+            parallelism).setNumTasks(tasks).fieldsGrouping(spoutName, new Fields("appId"));
+        return topologyBuilder.createTopology();
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/b54a63e0/eagle-jpm/eagle-jpm-mr-running/src/main/java/org/apache/eagle/jpm/mr/running/MRRunningJobApplicationProvider.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-mr-running/src/main/java/org/apache/eagle/jpm/mr/running/MRRunningJobApplicationProvider.java b/eagle-jpm/eagle-jpm-mr-running/src/main/java/org/apache/eagle/jpm/mr/running/MRRunningJobApplicationProvider.java
new file mode 100644
index 0000000..45a841b
--- /dev/null
+++ b/eagle-jpm/eagle-jpm-mr-running/src/main/java/org/apache/eagle/jpm/mr/running/MRRunningJobApplicationProvider.java
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.eagle.jpm.mr.running;
+
+import org.apache.eagle.app.spi.AbstractApplicationProvider;
+
+public class MRRunningJobApplicationProvider extends AbstractApplicationProvider<MRRunningJobApplication> {
+    @Override
+    public MRRunningJobApplication getApplication() {
+        return new MRRunningJobApplication();
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/b54a63e0/eagle-jpm/eagle-jpm-mr-running/src/main/java/org/apache/eagle/jpm/mr/running/MRRunningJobConfig.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-mr-running/src/main/java/org/apache/eagle/jpm/mr/running/MRRunningJobConfig.java b/eagle-jpm/eagle-jpm-mr-running/src/main/java/org/apache/eagle/jpm/mr/running/MRRunningJobConfig.java
new file mode 100644
index 0000000..ec6740b
--- /dev/null
+++ b/eagle-jpm/eagle-jpm-mr-running/src/main/java/org/apache/eagle/jpm/mr/running/MRRunningJobConfig.java
@@ -0,0 +1,156 @@
+/*
+ * 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.eagle.jpm.mr.running;
+
+import org.apache.eagle.common.config.ConfigOptionParser;
+
+import com.typesafe.config.Config;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Serializable;
+
+public class MRRunningJobConfig implements Serializable {
+    private static final Logger LOG = LoggerFactory.getLogger(MRRunningJobConfig.class);
+
+    public String getEnv() {
+        return env;
+    }
+
+    private String env;
+
+    public ZKStateConfig getZkStateConfig() {
+        return zkStateConfig;
+    }
+
+    private ZKStateConfig zkStateConfig;
+
+    public EagleServiceConfig getEagleServiceConfig() {
+        return eagleServiceConfig;
+    }
+
+    private EagleServiceConfig eagleServiceConfig;
+
+    public JobExtractorConfig getJobExtractorConfig() {
+        return jobExtractorConfig;
+    }
+
+    private JobExtractorConfig jobExtractorConfig;
+
+    public EndpointConfig getEndpointConfig() {
+        return endpointConfig;
+    }
+
+    private EndpointConfig endpointConfig;
+
+    public static class ZKStateConfig implements Serializable {
+        public String zkQuorum;
+        public String zkRoot;
+        public int zkSessionTimeoutMs;
+        public int zkRetryTimes;
+        public int zkRetryInterval;
+        public String zkPort;
+    }
+
+    public static class EagleServiceConfig implements Serializable {
+        public String eagleServiceHost;
+        public int eagleServicePort;
+        public int readTimeoutSeconds;
+        public int maxFlushNum;
+        public String username;
+        public String password;
+    }
+
+    public static class JobExtractorConfig implements Serializable {
+        public String site;
+        public int fetchRunningJobInterval;
+        public int parseJobThreadPoolSize;
+        public int topAndBottomTaskByElapsedTime;
+    }
+
+    public static class EndpointConfig implements Serializable {
+        public String[] rmUrls;
+    }
+
+    public Config getConfig() {
+        return config;
+    }
+
+    private Config config;
+
+    private static MRRunningJobConfig manager = new MRRunningJobConfig();
+
+    private MRRunningJobConfig() {
+        this.eagleServiceConfig = new EagleServiceConfig();
+        this.jobExtractorConfig = new JobExtractorConfig();
+        this.endpointConfig = new EndpointConfig();
+        this.zkStateConfig = new ZKStateConfig();
+    }
+
+    public static MRRunningJobConfig getInstance(String[] args) {
+        try {
+            LOG.info("Loading from configuration file");
+            return getInstance(new ConfigOptionParser().load(args));
+        } catch (Exception e) {
+            LOG.error("failed to load config");
+            throw new IllegalArgumentException("Failed to load config", e);
+        }
+    }
+
+    public static MRRunningJobConfig getInstance(Config config) {
+        manager.init(config);
+        return manager;
+    }
+
+    private void init(Config config) {
+        this.config = config;
+        this.env = config.getString("envContextConfig.env");
+
+        //parse eagle zk
+        this.zkStateConfig.zkQuorum = config.getString("zookeeperConfig.zkQuorum");
+        this.zkStateConfig.zkPort = config.getString("zookeeperConfig.zkPort");
+        this.zkStateConfig.zkSessionTimeoutMs = config.getInt("zookeeperConfig.zkSessionTimeoutMs");
+        this.zkStateConfig.zkRetryTimes = config.getInt("zookeeperConfig.zkRetryTimes");
+        this.zkStateConfig.zkRetryInterval = config.getInt("zookeeperConfig.zkRetryInterval");
+        this.zkStateConfig.zkRoot = config.getString("zookeeperConfig.zkRoot");
+
+        // parse eagle service endpoint
+        this.eagleServiceConfig.eagleServiceHost = config.getString("eagleProps.eagleService.host");
+        String port = config.getString("eagleProps.eagleService.port");
+        this.eagleServiceConfig.eagleServicePort = Integer.parseInt(port);
+        this.eagleServiceConfig.username = config.getString("eagleProps.eagleService.username");
+        this.eagleServiceConfig.password = config.getString("eagleProps.eagleService.password");
+        this.eagleServiceConfig.readTimeoutSeconds = config.getInt("eagleProps.eagleService.readTimeOutSeconds");
+        this.eagleServiceConfig.maxFlushNum = config.getInt("eagleProps.eagleService.maxFlushNum");
+        //parse job extractor
+        this.jobExtractorConfig.site = config.getString("jobExtractorConfig.site");
+        this.jobExtractorConfig.fetchRunningJobInterval = config.getInt("jobExtractorConfig.fetchRunningJobInterval");
+        this.jobExtractorConfig.parseJobThreadPoolSize = config.getInt("jobExtractorConfig.parseJobThreadPoolSize");
+        this.jobExtractorConfig.topAndBottomTaskByElapsedTime = config.getInt("jobExtractorConfig.topAndBottomTaskByElapsedTime");
+
+        //parse data source config
+        this.endpointConfig.rmUrls = config.getString("dataSourceConfig.rmUrls").split(",");
+
+        LOG.info("Successfully initialized MRRunningJobConfig");
+        LOG.info("env: " + this.env);
+        LOG.info("site: " + this.jobExtractorConfig.site);
+        LOG.info("eagle.service.host: " + this.eagleServiceConfig.eagleServiceHost);
+        LOG.info("eagle.service.port: " + this.eagleServiceConfig.eagleServicePort);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/b54a63e0/eagle-jpm/eagle-jpm-mr-running/src/main/java/org/apache/eagle/jpm/mr/running/MRRunningJobMain.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-mr-running/src/main/java/org/apache/eagle/jpm/mr/running/MRRunningJobMain.java b/eagle-jpm/eagle-jpm-mr-running/src/main/java/org/apache/eagle/jpm/mr/running/MRRunningJobMain.java
index 87079fd..a3d6d74 100644
--- a/eagle-jpm/eagle-jpm-mr-running/src/main/java/org/apache/eagle/jpm/mr/running/MRRunningJobMain.java
+++ b/eagle-jpm/eagle-jpm-mr-running/src/main/java/org/apache/eagle/jpm/mr/running/MRRunningJobMain.java
@@ -18,79 +18,8 @@
 
 package org.apache.eagle.jpm.mr.running;
 
-
-import org.apache.eagle.jpm.mr.running.config.MRRunningConfigManager;
-import org.apache.eagle.jpm.mr.running.storm.MRRunningJobFetchSpout;
-import org.apache.eagle.jpm.mr.running.storm.MRRunningJobParseBolt;
-import org.apache.eagle.jpm.util.Constants;
-import backtype.storm.Config;
-import backtype.storm.LocalCluster;
-import backtype.storm.StormSubmitter;
-import backtype.storm.topology.TopologyBuilder;
-import backtype.storm.tuple.Fields;
-import java.util.List;
-
 public class MRRunningJobMain {
     public static void main(String[] args) {
-
-        try {
-            //1. trigger init conf
-            MRRunningConfigManager mrRunningConfigManager = MRRunningConfigManager.getInstance(args);
-
-            List<String> confKeyKeys = mrRunningConfigManager.getConfig().getStringList("MRConfigureKeys.jobConfigKey");
-            confKeyKeys.add(Constants.JobConfiguration.CASCADING_JOB);
-            confKeyKeys.add(Constants.JobConfiguration.HIVE_JOB);
-            confKeyKeys.add(Constants.JobConfiguration.PIG_JOB);
-            confKeyKeys.add(Constants.JobConfiguration.SCOOBI_JOB);
-            confKeyKeys.add(0, mrRunningConfigManager.getConfig().getString("MRConfigureKeys.jobNameKey"));
-
-            //2. init topology
-            TopologyBuilder topologyBuilder = new TopologyBuilder();
-            String topologyName = mrRunningConfigManager.getConfig().getString("envContextConfig.topologyName");
-            String spoutName = "mrRunningJobFetchSpout";
-            String boltName = "mrRunningJobParseBolt";
-            int parallelism = mrRunningConfigManager.getConfig().getInt("envContextConfig.parallelismConfig." + spoutName);
-            int tasks = mrRunningConfigManager.getConfig().getInt("envContextConfig.tasks." + spoutName);
-            if (parallelism > tasks) {
-                parallelism = tasks;
-            }
-            topologyBuilder.setSpout(
-                    spoutName,
-                    new MRRunningJobFetchSpout(
-                            mrRunningConfigManager.getJobExtractorConfig(),
-                            mrRunningConfigManager.getEndpointConfig(),
-                            mrRunningConfigManager.getZkStateConfig()),
-                    parallelism
-            ).setNumTasks(tasks);
-
-            parallelism = mrRunningConfigManager.getConfig().getInt("envContextConfig.parallelismConfig." + boltName);
-            tasks = mrRunningConfigManager.getConfig().getInt("envContextConfig.tasks." + boltName);
-            if (parallelism > tasks) {
-                parallelism = tasks;
-            }
-            topologyBuilder.setBolt(boltName,
-                    new MRRunningJobParseBolt(
-                            mrRunningConfigManager.getEagleServiceConfig(),
-                            mrRunningConfigManager.getEndpointConfig(),
-                            mrRunningConfigManager.getJobExtractorConfig(),
-                            mrRunningConfigManager.getZkStateConfig(),
-                            confKeyKeys),
-                    parallelism).setNumTasks(tasks).fieldsGrouping(spoutName, new Fields("appId"));
-
-            backtype.storm.Config config = new backtype.storm.Config();
-            config.setNumWorkers(mrRunningConfigManager.getConfig().getInt("envContextConfig.workers"));
-            config.put(Config.TOPOLOGY_DEBUG, true);
-            if (!mrRunningConfigManager.getEnv().equals("local")) {
-                //cluster mode
-                //parse conf here
-                StormSubmitter.submitTopology(topologyName, config, topologyBuilder.createTopology());
-            } else {
-                //local mode
-                LocalCluster cluster = new LocalCluster();
-                cluster.submitTopology(topologyName, config, topologyBuilder.createTopology());
-            }
-        } catch (Exception e) {
-            e.printStackTrace();
-        }
+        new MRRunningJobApplication().run(args);
     }
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/b54a63e0/eagle-jpm/eagle-jpm-mr-running/src/main/java/org/apache/eagle/jpm/mr/running/config/MRRunningConfigManager.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-mr-running/src/main/java/org/apache/eagle/jpm/mr/running/config/MRRunningConfigManager.java b/eagle-jpm/eagle-jpm-mr-running/src/main/java/org/apache/eagle/jpm/mr/running/config/MRRunningConfigManager.java
deleted file mode 100644
index 42426e4..0000000
--- a/eagle-jpm/eagle-jpm-mr-running/src/main/java/org/apache/eagle/jpm/mr/running/config/MRRunningConfigManager.java
+++ /dev/null
@@ -1,151 +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.eagle.jpm.mr.running.config;
-
-import org.apache.eagle.common.config.ConfigOptionParser;
-import com.typesafe.config.Config;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.Serializable;
-
-public class MRRunningConfigManager implements Serializable {
-    private static final Logger LOG = LoggerFactory.getLogger(MRRunningConfigManager.class);
-
-    public String getEnv() {
-        return env;
-    }
-
-    private String env;
-
-    public ZKStateConfig getZkStateConfig() {
-        return zkStateConfig;
-    }
-
-    private ZKStateConfig zkStateConfig;
-
-    public EagleServiceConfig getEagleServiceConfig() {
-        return eagleServiceConfig;
-    }
-
-    private EagleServiceConfig eagleServiceConfig;
-
-    public JobExtractorConfig getJobExtractorConfig() {
-        return jobExtractorConfig;
-    }
-
-    private JobExtractorConfig jobExtractorConfig;
-
-    public EndpointConfig getEndpointConfig() {
-        return endpointConfig;
-    }
-
-    private EndpointConfig endpointConfig;
-
-    public static class ZKStateConfig implements Serializable {
-        public String zkQuorum;
-        public String zkRoot;
-        public int zkSessionTimeoutMs;
-        public int zkRetryTimes;
-        public int zkRetryInterval;
-        public String zkPort;
-    }
-
-    public static class EagleServiceConfig implements Serializable {
-        public String eagleServiceHost;
-        public int eagleServicePort;
-        public int readTimeoutSeconds;
-        public int maxFlushNum;
-        public String username;
-        public String password;
-    }
-
-    public static class JobExtractorConfig implements Serializable {
-        public String site;
-        public int fetchRunningJobInterval;
-        public int parseJobThreadPoolSize;
-        public int topAndBottomTaskByElapsedTime;
-    }
-
-    public static class EndpointConfig implements Serializable {
-        public String[] rmUrls;
-    }
-
-    public Config getConfig() {
-        return config;
-    }
-
-    private Config config;
-
-    private static MRRunningConfigManager manager = new MRRunningConfigManager();
-
-    private MRRunningConfigManager() {
-        this.eagleServiceConfig = new EagleServiceConfig();
-        this.jobExtractorConfig = new JobExtractorConfig();
-        this.endpointConfig = new EndpointConfig();
-        this.zkStateConfig = new ZKStateConfig();
-    }
-
-    public static MRRunningConfigManager getInstance(String[] args) {
-        manager.init(args);
-        return manager;
-    }
-
-    private void init(String[] args) {
-        try {
-            LOG.info("Loading from configuration file");
-            this.config = new ConfigOptionParser().load(args);
-        } catch (Exception e) {
-            LOG.error("failed to load config");
-        }
-
-        this.env = config.getString("envContextConfig.env");
-
-        //parse eagle zk
-        this.zkStateConfig.zkQuorum = config.getString("zookeeperConfig.zkQuorum");
-        this.zkStateConfig.zkPort = config.getString("zookeeperConfig.zkPort");
-        this.zkStateConfig.zkSessionTimeoutMs = config.getInt("zookeeperConfig.zkSessionTimeoutMs");
-        this.zkStateConfig.zkRetryTimes = config.getInt("zookeeperConfig.zkRetryTimes");
-        this.zkStateConfig.zkRetryInterval = config.getInt("zookeeperConfig.zkRetryInterval");
-        this.zkStateConfig.zkRoot = config.getString("zookeeperConfig.zkRoot");
-
-        // parse eagle service endpoint
-        this.eagleServiceConfig.eagleServiceHost = config.getString("eagleProps.eagleService.host");
-        String port = config.getString("eagleProps.eagleService.port");
-        this.eagleServiceConfig.eagleServicePort = Integer.parseInt(port);
-        this.eagleServiceConfig.username = config.getString("eagleProps.eagleService.username");
-        this.eagleServiceConfig.password = config.getString("eagleProps.eagleService.password");
-        this.eagleServiceConfig.readTimeoutSeconds = config.getInt("eagleProps.eagleService.readTimeOutSeconds");
-        this.eagleServiceConfig.maxFlushNum = config.getInt("eagleProps.eagleService.maxFlushNum");
-        //parse job extractor
-        this.jobExtractorConfig.site = config.getString("jobExtractorConfig.site");
-        this.jobExtractorConfig.fetchRunningJobInterval = config.getInt("jobExtractorConfig.fetchRunningJobInterval");
-        this.jobExtractorConfig.parseJobThreadPoolSize = config.getInt("jobExtractorConfig.parseJobThreadPoolSize");
-        this.jobExtractorConfig.topAndBottomTaskByElapsedTime = config.getInt("jobExtractorConfig.topAndBottomTaskByElapsedTime");
-
-        //parse data source config
-        this.endpointConfig.rmUrls = config.getStringList("dataSourceConfig.rmUrls").toArray(new String[0]);
-
-        LOG.info("Successfully initialized MRRunningConfigManager");
-        LOG.info("env: " + this.env);
-        LOG.info("site: " + this.jobExtractorConfig.site);
-        LOG.info("eagle.service.host: " + this.eagleServiceConfig.eagleServiceHost);
-        LOG.info("eagle.service.port: " + this.eagleServiceConfig.eagleServicePort);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/b54a63e0/eagle-jpm/eagle-jpm-mr-running/src/main/java/org/apache/eagle/jpm/mr/running/parser/MRJobEntityCreationHandler.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-mr-running/src/main/java/org/apache/eagle/jpm/mr/running/parser/MRJobEntityCreationHandler.java b/eagle-jpm/eagle-jpm-mr-running/src/main/java/org/apache/eagle/jpm/mr/running/parser/MRJobEntityCreationHandler.java
index a43ed16..1a0fb61 100644
--- a/eagle-jpm/eagle-jpm-mr-running/src/main/java/org/apache/eagle/jpm/mr/running/parser/MRJobEntityCreationHandler.java
+++ b/eagle-jpm/eagle-jpm-mr-running/src/main/java/org/apache/eagle/jpm/mr/running/parser/MRJobEntityCreationHandler.java
@@ -18,7 +18,7 @@
 
 package org.apache.eagle.jpm.mr.running.parser;
 
-import org.apache.eagle.jpm.mr.running.config.MRRunningConfigManager;
+import org.apache.eagle.jpm.mr.running.MRRunningJobConfig;
 import org.apache.eagle.jpm.mr.running.parser.metrics.JobExecutionMetricsCreationListener;
 import org.apache.eagle.jpm.mr.running.parser.metrics.TaskExecutionMetricsCreationListener;
 import org.apache.eagle.jpm.mr.runningentity.JobExecutionAPIEntity;
@@ -37,11 +37,11 @@ public class MRJobEntityCreationHandler {
     private static final Logger LOG = LoggerFactory.getLogger(MRJobEntityCreationHandler.class);
 
     private List<TaggedLogAPIEntity> entities = new ArrayList<>();
-    private MRRunningConfigManager.EagleServiceConfig eagleServiceConfig;
+    private MRRunningJobConfig.EagleServiceConfig eagleServiceConfig;
     private JobExecutionMetricsCreationListener jobMetricsListener;
     private TaskExecutionMetricsCreationListener taskMetricsListener;
 
-    public MRJobEntityCreationHandler(MRRunningConfigManager.EagleServiceConfig eagleServiceConfig) {
+    public MRJobEntityCreationHandler(MRRunningJobConfig.EagleServiceConfig eagleServiceConfig) {
         this.eagleServiceConfig = eagleServiceConfig;
         jobMetricsListener = new JobExecutionMetricsCreationListener();
         taskMetricsListener = new TaskExecutionMetricsCreationListener();

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/b54a63e0/eagle-jpm/eagle-jpm-mr-running/src/main/java/org/apache/eagle/jpm/mr/running/parser/MRJobParser.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-mr-running/src/main/java/org/apache/eagle/jpm/mr/running/parser/MRJobParser.java b/eagle-jpm/eagle-jpm-mr-running/src/main/java/org/apache/eagle/jpm/mr/running/parser/MRJobParser.java
index 3b31d93..9148c0c 100644
--- a/eagle-jpm/eagle-jpm-mr-running/src/main/java/org/apache/eagle/jpm/mr/running/parser/MRJobParser.java
+++ b/eagle-jpm/eagle-jpm-mr-running/src/main/java/org/apache/eagle/jpm/mr/running/parser/MRJobParser.java
@@ -18,7 +18,7 @@
 
 package org.apache.eagle.jpm.mr.running.parser;
 
-import org.apache.eagle.jpm.mr.running.config.MRRunningConfigManager;
+import org.apache.eagle.jpm.mr.running.MRRunningJobConfig;
 import org.apache.eagle.jpm.mr.running.recover.MRRunningJobManager;
 import org.apache.eagle.jpm.mr.runningentity.JobConfig;
 import org.apache.eagle.jpm.mr.runningentity.JobExecutionAPIEntity;
@@ -77,14 +77,14 @@ public class MRJobParser implements Runnable {
     private boolean first;
     private Set<String> finishedTaskIds;
     private List<String> configKeys;
-    private MRRunningConfigManager.JobExtractorConfig jobExtractorConfig;
+    private MRRunningJobConfig.JobExtractorConfig jobExtractorConfig;
 
     static {
         OBJ_MAPPER.configure(JsonParser.Feature.ALLOW_NON_NUMERIC_NUMBERS, true);
     }
 
-    public MRJobParser(MRRunningConfigManager.JobExtractorConfig jobExtractorConfig,
-                       MRRunningConfigManager.EagleServiceConfig eagleServiceConfig,
+    public MRJobParser(MRRunningJobConfig.JobExtractorConfig jobExtractorConfig,
+                       MRRunningJobConfig.EagleServiceConfig eagleServiceConfig,
                        AppInfo app, Map<String, JobExecutionAPIEntity> mrJobMap,
                        MRRunningJobManager runningJobManager, ResourceFetcher rmResourceFetcher,
                        List<String> configKeys) {
@@ -393,8 +393,8 @@ public class MRJobParser implements Runnable {
         Comparator<MRTask> byElapsedTimeDecrease = (e1, e2) -> -1 * Long.compare(e1.getElapsedTime(), e2.getElapsedTime());
         //2, get finished bottom n
         Iterator<MRTask> taskIteratorIncrease = tasks.stream()
-                .filter(task -> task.getState().equals(Constants.TaskState.SUCCEEDED.toString()))
-                .sorted(byElapsedTimeIncrease).iterator();
+            .filter(task -> task.getState().equals(Constants.TaskState.SUCCEEDED.toString()))
+            .sorted(byElapsedTimeIncrease).iterator();
         int i = 0;
         while (taskIteratorIncrease.hasNext() && i < jobExtractorConfig.topAndBottomTaskByElapsedTime) {
             MRTask mrTask = taskIteratorIncrease.next();
@@ -405,8 +405,8 @@ public class MRJobParser implements Runnable {
         }
         //3, fetch finished top n
         Iterator<MRTask> taskIteratorDecrease = tasks.stream()
-                .filter(task -> task.getState().equals(Constants.TaskState.SUCCEEDED.toString()))
-                .sorted(byElapsedTimeDecrease).iterator();
+            .filter(task -> task.getState().equals(Constants.TaskState.SUCCEEDED.toString()))
+            .sorted(byElapsedTimeDecrease).iterator();
         i = 0;
         while (taskIteratorDecrease.hasNext() && i < jobExtractorConfig.topAndBottomTaskByElapsedTime) {
             MRTask mrTask = taskIteratorDecrease.next();
@@ -417,8 +417,8 @@ public class MRJobParser implements Runnable {
         }
         //4, fetch running top n
         taskIteratorDecrease = tasks.stream()
-                .filter(task -> task.getState().equals(Constants.TaskState.RUNNING.toString()))
-                .sorted(byElapsedTimeDecrease).iterator();
+            .filter(task -> task.getState().equals(Constants.TaskState.RUNNING.toString()))
+            .sorted(byElapsedTimeDecrease).iterator();
         i = 0;
         while (taskIteratorDecrease.hasNext() && i < jobExtractorConfig.topAndBottomTaskByElapsedTime) {
             MRTask mrTask = taskIteratorDecrease.next();
@@ -560,12 +560,12 @@ public class MRJobParser implements Runnable {
                     //we must flush entities before delete from zk in case of missing finish state of jobs
                     //delete from zk if needed
                     mrJobEntityMap.keySet()
-                            .stream()
-                            .filter(
-                                jobId -> mrJobEntityMap.get(jobId).getCurrentState().equals(Constants.AppState.FINISHED.toString())
-                                    || mrJobEntityMap.get(jobId).getCurrentState().equals(Constants.AppState.FAILED.toString()))
-                            .forEach(
-                                jobId -> this.runningJobManager.delete(app.getId(), jobId));
+                        .stream()
+                        .filter(
+                            jobId -> mrJobEntityMap.get(jobId).getCurrentState().equals(Constants.AppState.FINISHED.toString())
+                                || mrJobEntityMap.get(jobId).getCurrentState().equals(Constants.AppState.FAILED.toString()))
+                        .forEach(
+                            jobId -> this.runningJobManager.delete(app.getId(), jobId));
                 }
 
                 LOG.info("finish process yarn application " + app.getId());
@@ -575,4 +575,4 @@ public class MRJobParser implements Runnable {
             }
         }
     }
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/b54a63e0/eagle-jpm/eagle-jpm-mr-running/src/main/java/org/apache/eagle/jpm/mr/running/recover/MRRunningJobManager.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-mr-running/src/main/java/org/apache/eagle/jpm/mr/running/recover/MRRunningJobManager.java b/eagle-jpm/eagle-jpm-mr-running/src/main/java/org/apache/eagle/jpm/mr/running/recover/MRRunningJobManager.java
index 75650b7..50b4726 100644
--- a/eagle-jpm/eagle-jpm-mr-running/src/main/java/org/apache/eagle/jpm/mr/running/recover/MRRunningJobManager.java
+++ b/eagle-jpm/eagle-jpm-mr-running/src/main/java/org/apache/eagle/jpm/mr/running/recover/MRRunningJobManager.java
@@ -18,11 +18,13 @@
 
 package org.apache.eagle.jpm.mr.running.recover;
 
-import org.apache.eagle.jpm.mr.running.config.MRRunningConfigManager;
+import org.apache.eagle.jpm.mr.running.MRRunningJobConfig;
 import org.apache.eagle.jpm.mr.runningentity.JobExecutionAPIEntity;
 import org.apache.eagle.jpm.util.jobrecover.RunningJobManager;
-import org.apache.eagle.jpm.util.resourcefetch.model.AppInfo;
+
 import org.apache.commons.lang3.tuple.Pair;
+import org.apache.eagle.jpm.util.resourcefetch.model.AppInfo;
+
 import java.io.Serializable;
 import java.util.HashMap;
 import java.util.Map;
@@ -30,7 +32,7 @@ import java.util.Map;
 public class MRRunningJobManager implements Serializable {
     private RunningJobManager runningJobManager;
 
-    public MRRunningJobManager(MRRunningConfigManager.ZKStateConfig config) {
+    public MRRunningJobManager(MRRunningJobConfig.ZKStateConfig config) {
         this.runningJobManager = new RunningJobManager(config.zkQuorum,
                 config.zkSessionTimeoutMs, config.zkRetryTimes, config.zkRetryInterval, config.zkRoot);
     }

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/b54a63e0/eagle-jpm/eagle-jpm-mr-running/src/main/java/org/apache/eagle/jpm/mr/running/storm/MRRunningJobFetchSpout.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-mr-running/src/main/java/org/apache/eagle/jpm/mr/running/storm/MRRunningJobFetchSpout.java b/eagle-jpm/eagle-jpm-mr-running/src/main/java/org/apache/eagle/jpm/mr/running/storm/MRRunningJobFetchSpout.java
index ebb9144..268912c 100644
--- a/eagle-jpm/eagle-jpm-mr-running/src/main/java/org/apache/eagle/jpm/mr/running/storm/MRRunningJobFetchSpout.java
+++ b/eagle-jpm/eagle-jpm-mr-running/src/main/java/org/apache/eagle/jpm/mr/running/storm/MRRunningJobFetchSpout.java
@@ -18,13 +18,7 @@
 
 package org.apache.eagle.jpm.mr.running.storm;
 
-import backtype.storm.spout.SpoutOutputCollector;
-import backtype.storm.task.TopologyContext;
-import backtype.storm.topology.OutputFieldsDeclarer;
-import backtype.storm.topology.base.BaseRichSpout;
-import backtype.storm.tuple.Fields;
-import backtype.storm.tuple.Values;
-import org.apache.eagle.jpm.mr.running.config.MRRunningConfigManager;
+import org.apache.eagle.jpm.mr.running.MRRunningJobConfig;
 import org.apache.eagle.jpm.mr.running.recover.MRRunningJobManager;
 import org.apache.eagle.jpm.mr.runningentity.JobExecutionAPIEntity;
 import org.apache.eagle.jpm.util.Constants;
@@ -32,6 +26,13 @@ import org.apache.eagle.jpm.util.Utils;
 import org.apache.eagle.jpm.util.resourcefetch.RMResourceFetcher;
 import org.apache.eagle.jpm.util.resourcefetch.ResourceFetcher;
 import org.apache.eagle.jpm.util.resourcefetch.model.AppInfo;
+
+import backtype.storm.spout.SpoutOutputCollector;
+import backtype.storm.task.TopologyContext;
+import backtype.storm.topology.OutputFieldsDeclarer;
+import backtype.storm.topology.base.BaseRichSpout;
+import backtype.storm.tuple.Fields;
+import backtype.storm.tuple.Values;
 import org.apache.zookeeper.KeeperException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -40,17 +41,18 @@ import java.util.*;
 
 public class MRRunningJobFetchSpout extends BaseRichSpout {
     private static final Logger LOG = LoggerFactory.getLogger(MRRunningJobFetchSpout.class);
-    private MRRunningConfigManager.JobExtractorConfig jobExtractorConfig;
-    private MRRunningConfigManager.EndpointConfig endpointConfig;
-    private MRRunningConfigManager.ZKStateConfig zkStateConfig;
+    private MRRunningJobConfig.JobExtractorConfig jobExtractorConfig;
+    private MRRunningJobConfig.EndpointConfig endpointConfig;
+    private MRRunningJobConfig.ZKStateConfig zkStateConfig;
     private ResourceFetcher resourceFetcher;
     private SpoutOutputCollector collector;
     private boolean init;
     private transient MRRunningJobManager runningJobManager;
     private Set<String> runningYarnApps;
-    public MRRunningJobFetchSpout(MRRunningConfigManager.JobExtractorConfig jobExtractorConfig,
-                                  MRRunningConfigManager.EndpointConfig endpointConfig,
-                                  MRRunningConfigManager.ZKStateConfig zkStateConfig) {
+
+    public MRRunningJobFetchSpout(MRRunningJobConfig.JobExtractorConfig jobExtractorConfig,
+                                  MRRunningJobConfig.EndpointConfig endpointConfig,
+                                  MRRunningJobConfig.ZKStateConfig zkStateConfig) {
         this.jobExtractorConfig = jobExtractorConfig;
         this.endpointConfig = endpointConfig;
         this.zkStateConfig = zkStateConfig;
@@ -168,4 +170,4 @@ public class MRRunningJobFetchSpout extends BaseRichSpout {
     @Override
     public void close() {
     }
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/b54a63e0/eagle-jpm/eagle-jpm-mr-running/src/main/java/org/apache/eagle/jpm/mr/running/storm/MRRunningJobParseBolt.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-mr-running/src/main/java/org/apache/eagle/jpm/mr/running/storm/MRRunningJobParseBolt.java b/eagle-jpm/eagle-jpm-mr-running/src/main/java/org/apache/eagle/jpm/mr/running/storm/MRRunningJobParseBolt.java
index 0dccd70..3174eb1 100644
--- a/eagle-jpm/eagle-jpm-mr-running/src/main/java/org/apache/eagle/jpm/mr/running/storm/MRRunningJobParseBolt.java
+++ b/eagle-jpm/eagle-jpm-mr-running/src/main/java/org/apache/eagle/jpm/mr/running/storm/MRRunningJobParseBolt.java
@@ -18,12 +18,7 @@
 
 package org.apache.eagle.jpm.mr.running.storm;
 
-import backtype.storm.task.OutputCollector;
-import backtype.storm.task.TopologyContext;
-import backtype.storm.topology.OutputFieldsDeclarer;
-import backtype.storm.topology.base.BaseRichBolt;
-import backtype.storm.tuple.Tuple;
-import org.apache.eagle.jpm.mr.running.config.MRRunningConfigManager;
+import org.apache.eagle.jpm.mr.running.MRRunningJobConfig;
 import org.apache.eagle.jpm.mr.running.parser.MRJobParser;
 import org.apache.eagle.jpm.mr.running.recover.MRRunningJobManager;
 import org.apache.eagle.jpm.mr.runningentity.JobExecutionAPIEntity;
@@ -31,6 +26,12 @@ import org.apache.eagle.jpm.util.Constants;
 import org.apache.eagle.jpm.util.resourcefetch.RMResourceFetcher;
 import org.apache.eagle.jpm.util.resourcefetch.ResourceFetcher;
 import org.apache.eagle.jpm.util.resourcefetch.model.AppInfo;
+
+import backtype.storm.task.OutputCollector;
+import backtype.storm.task.TopologyContext;
+import backtype.storm.topology.OutputFieldsDeclarer;
+import backtype.storm.tuple.Tuple;
+import backtype.storm.topology.base.BaseRichBolt;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -41,19 +42,20 @@ import java.util.concurrent.Executors;
 public class MRRunningJobParseBolt extends BaseRichBolt {
     private static final Logger LOG = LoggerFactory.getLogger(MRRunningJobParseBolt.class);
 
-    private MRRunningConfigManager.EndpointConfig endpointConfig;
-    private MRRunningConfigManager.JobExtractorConfig jobExtractorConfig;
-    private MRRunningConfigManager.ZKStateConfig zkStateConfig;
+    private MRRunningJobConfig.EndpointConfig endpointConfig;
+    private MRRunningJobConfig.JobExtractorConfig jobExtractorConfig;
+    private MRRunningJobConfig.ZKStateConfig zkStateConfig;
     private ExecutorService executorService;
     private Map<String, MRJobParser> runningMRParsers;
     private transient MRRunningJobManager runningJobManager;
-    private MRRunningConfigManager.EagleServiceConfig eagleServiceConfig;
+    private MRRunningJobConfig.EagleServiceConfig eagleServiceConfig;
     private ResourceFetcher resourceFetcher;
     private List<String> configKeys;
-    public MRRunningJobParseBolt(MRRunningConfigManager.EagleServiceConfig eagleServiceConfig,
-                                 MRRunningConfigManager.EndpointConfig endpointConfig,
-                                 MRRunningConfigManager.JobExtractorConfig jobExtractorConfig,
-                                 MRRunningConfigManager.ZKStateConfig zkStateConfig,
+
+    public MRRunningJobParseBolt(MRRunningJobConfig.EagleServiceConfig eagleServiceConfig,
+                                 MRRunningJobConfig.EndpointConfig endpointConfig,
+                                 MRRunningJobConfig.JobExtractorConfig jobExtractorConfig,
+                                 MRRunningJobConfig.ZKStateConfig zkStateConfig,
                                  List<String> configKeys) {
         this.eagleServiceConfig = eagleServiceConfig;
         this.endpointConfig = endpointConfig;
@@ -96,8 +98,8 @@ public class MRRunningJobParseBolt extends BaseRichBolt {
                     LOG.info("remove parser {}", appId);
                 });
 
-        if (appInfo.getState().equals(Constants.AppState.FINISHED.toString()) ||
-                applicationParser.status() == MRJobParser.ParserStatus.FINISHED) {
+        if (appInfo.getState().equals(Constants.AppState.FINISHED.toString())
+            || applicationParser.status() == MRJobParser.ParserStatus.FINISHED) {
             applicationParser.setStatus(MRJobParser.ParserStatus.RUNNING);
             executorService.execute(applicationParser);
         }

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/b54a63e0/eagle-jpm/eagle-jpm-mr-running/src/main/resources/META-INF/providers/org.apache.eagle.jpm.mr.running.MRRunningJobApplicationProvider.xml
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-mr-running/src/main/resources/META-INF/providers/org.apache.eagle.jpm.mr.running.MRRunningJobApplicationProvider.xml b/eagle-jpm/eagle-jpm-mr-running/src/main/resources/META-INF/providers/org.apache.eagle.jpm.mr.running.MRRunningJobApplicationProvider.xml
new file mode 100644
index 0000000..5f809f3
--- /dev/null
+++ b/eagle-jpm/eagle-jpm-mr-running/src/main/resources/META-INF/providers/org.apache.eagle.jpm.mr.running.MRRunningJobApplicationProvider.xml
@@ -0,0 +1,175 @@
+<?xml version="1.0" encoding="UTF-8" ?>
+<!--
+  ~ Licensed to the Apache Software Foundation (ASF) under one or more
+  ~ contributor license agreements.  See the NOTICE file distributed with
+  ~ this work for additional information regarding copyright ownership.
+  ~ The ASF licenses this file to You under the Apache License, Version 2.0
+  ~ (the "License"); you may not use this file except in compliance with
+  ~ the License.  You may obtain a copy of the License at
+  ~
+  ~    http://www.apache.org/licenses/LICENSE-2.0
+  ~
+  ~ Unless required by applicable law or agreed to in writing, software
+  ~ distributed under the License is distributed on an "AS IS" BASIS,
+  ~ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  ~ See the License for the specific language governing permissions and
+  ~ limitations under the License.
+  -->
+
+<application>
+    <type>MR_RUNNING_JOB_APP</type>
+    <name>MR Running Job Monitoring</name>
+    <version>0.5.0-incubating</version>
+    <appClass>org.apache.eagle.jpm.mr.running.MRRunningJobApplication</appClass>
+    <viewPath>/apps/jpm</viewPath>
+    <configuration>
+        <!-- org.apache.eagle.jpm.spark.running.SparkRunningJobAppConfig -->
+        <property>
+            <name>envContextConfig.env</name>
+            <value>local</value>
+            <displayName>Environment</displayName>
+            <description>Execution environment</description>
+        </property>
+        <property>
+            <name>zookeeperConfig.zkQuorum</name>
+            <displayName>zkQuorum</displayName>
+            <description>Zookeeper Quorum</description>
+            <value>sandbox.hortonworks.com:2181</value>
+        </property>
+        <property>
+            <name>zookeeperConfig.zkPort</name>
+            <displayName>zkPort</displayName>
+            <description>Zookeeper Port</description>
+            <value>2181</value>
+        </property>
+        <property>
+            <name>zookeeperConfig.zkSessionTimeoutMs</name>
+            <displayName>zkSessionTimeoutMs</displayName>
+            <description>Zookeeper session timeoutMs</description>
+            <value>15000</value>
+        </property>
+        <property>
+            <name>zookeeperConfig.zkRetryTimes</name>
+            <displayName>zkRetryTimes</displayName>
+            <description>zookeeperConfig.zkRetryTimes</description>
+            <value>3</value>
+        </property>
+        <property>
+            <name>zookeeperConfig.zkRetryInterval</name>
+            <displayName>zkRetryInterval</displayName>
+            <description>zookeeperConfig.zkRetryInterval</description>
+            <value>20000</value>
+        </property>
+        <property>
+            <name>zookeeperConfig.zkRoot</name>
+            <value>/apps/mr/running</value>
+        </property>
+        <property>
+            <name>eagleProps.eagleService.host</name>
+            <description>eagleProps.eagleService.host</description>
+            <value>sandbox.hortonworks.com</value>
+        </property>
+        <property>
+            <name>eagleProps.eagleService.port</name>
+            <description>eagleProps.eagleService.port</description>
+            <value>9099</value>
+        </property>
+        <property>
+            <name>eagleProps.eagleService.username</name>
+            <description>eagleProps.eagleService.username</description>
+            <value>admin</value>
+        </property>
+        <property>
+            <name>eagleProps.eagleService.password</name>
+            <description>eagleProps.eagleService.password</description>
+            <value>secret</value>
+        </property>
+        <property>
+            <name>eagleProps.eagleService.readTimeOutSeconds</name>
+            <description>eagleProps.eagleService.readTimeOutSeconds</description>
+            <value>20</value>
+        </property>
+        <property>
+            <name>eagleProps.eagleService.maxFlushNum</name>
+            <description>eagleProps.eagleService.maxFlushNum</description>
+            <value>500</value>
+        </property>
+        <property>
+            <name>jobExtractorConfig.site</name>
+            <description>jobExtractorConfig.site</description>
+            <value>sandbox</value>
+        </property>
+        <property>
+            <name>jobExtractorConfig.fetchRunningJobInterval</name>
+            <description>jobExtractorConfig.fetchRunningJobInterval</description>
+            <value>15</value>
+        </property>
+        <property>
+            <name>jobExtractorConfig.parseThreadPoolSize</name>
+            <description>jobExtractorConfig.parseThreadPoolSize</description>
+            <value>5</value>
+        </property>
+        <property>
+            <name>jobExtractorConfig.topAndBottomTaskByElapsedTime</name>
+            <description>jobExtractorConfig.topAndBottomTaskByElapsedTime</description>
+            <value>5</value>
+        </property>
+        <property>
+            <name>dataSourceConfig.rmUrls</name>
+            <description>dataSourceConfig.rmUrls</description>
+            <value>http://sandbox.hortonworks.com:50030</value>
+        </property>
+        <property>
+            <name>dataSourceConfig.nnEndpoint</name>
+            <description>dataSourceConfig.nnEndpoint</description>
+            <value>hdfs://sandbox.hortonworks.com:8020</value>
+        </property>
+        <property>
+            <name>dataSourceConfig.keytab</name>
+            <description>dataSourceConfig.keytab</description>
+            <value></value>
+        </property>
+        <property>
+            <name>dataSourceConfig.principal</name>
+            <description>dataSourceConfig.principal</description>
+            <value></value>
+        </property>
+        <property>
+            <name>dataSourceConfig.rmUrls</name>
+            <description>dataSourceConfig.rmUrls</description>
+            <value>http://sandbox.hortonworks.com:8088</value>
+        </property>
+    </configuration>
+    <docs>
+        <install>
+            # Step 1: Create source kafka topic named "${site}_example_source_topic"
+
+            ./bin/kafka-topics.sh --create --topic example_source_topic --replication-factor 1 --replication 1
+
+            # Step 2: Set up data collector to flow data into kafka topic in
+
+            ./bin/logstash -f log_collector.conf
+
+            ## `log_collector.conf` sample as following:
+
+            input {
+
+            }
+            filter {
+
+            }
+            output{
+
+            }
+
+            # Step 3: start application
+
+            # Step 4: monitor with featured portal or alert with policies
+        </install>
+        <uninstall>
+            # Step 1: stop and uninstall application
+            # Step 2: delete kafka topic named "${site}_example_source_topic"
+            # Step 3: stop logstash
+        </uninstall>
+    </docs>
+</application>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/b54a63e0/eagle-jpm/eagle-jpm-mr-running/src/main/resources/META-INF/services/org.apache.eagle.app.spi.ApplicationProvider
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-mr-running/src/main/resources/META-INF/services/org.apache.eagle.app.spi.ApplicationProvider b/eagle-jpm/eagle-jpm-mr-running/src/main/resources/META-INF/services/org.apache.eagle.app.spi.ApplicationProvider
new file mode 100644
index 0000000..61ec08c
--- /dev/null
+++ b/eagle-jpm/eagle-jpm-mr-running/src/main/resources/META-INF/services/org.apache.eagle.app.spi.ApplicationProvider
@@ -0,0 +1,16 @@
+# 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.
+
+org.apache.eagle.jpm.mr.running.MRRunningJobApplicationProvider
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/b54a63e0/eagle-jpm/eagle-jpm-mr-running/src/main/resources/application.conf
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-mr-running/src/main/resources/application.conf b/eagle-jpm/eagle-jpm-mr-running/src/main/resources/application.conf
index f15fc2d..4b6d4fe 100644
--- a/eagle-jpm/eagle-jpm-mr-running/src/main/resources/application.conf
+++ b/eagle-jpm/eagle-jpm-mr-running/src/main/resources/application.conf
@@ -14,10 +14,10 @@
 # limitations under the License.
 
 {
+  "appId":"mrRunningJob",
+  "mode":"LOCAL",
   "envContextConfig" : {
     "env" : "local",
-    "topologyName" : "mrRunningJob",
-    "stormConfigFile" : "storm.yaml",
     "parallelismConfig" : {
       "mrRunningJobFetchSpout" : 1,
       "mrRunningJobParseBolt" : 10
@@ -44,11 +44,9 @@
     "zkRetryTimes" : 3,
     "zkRetryInterval" : 20000
   },
-
   "dataSourceConfig" : {
-    "rmUrls": ["http://sandbox.hortonworks.com:50030"]
+    "rmUrls": "http://sandbox.hortonworks.com:50030"
   },
-
   "eagleProps" : {
     "mailHost" : "abc.com",
     "mailDebug" : "true",

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/b54a63e0/eagle-jpm/eagle-jpm-mr-running/src/test/java/org/apache/eagle/jpm/mr/running/MRRunningJobApplicationProviderTest.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-mr-running/src/test/java/org/apache/eagle/jpm/mr/running/MRRunningJobApplicationProviderTest.java b/eagle-jpm/eagle-jpm-mr-running/src/test/java/org/apache/eagle/jpm/mr/running/MRRunningJobApplicationProviderTest.java
new file mode 100644
index 0000000..2b7d429
--- /dev/null
+++ b/eagle-jpm/eagle-jpm-mr-running/src/test/java/org/apache/eagle/jpm/mr/running/MRRunningJobApplicationProviderTest.java
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.eagle.jpm.mr.running;
+
+import com.google.inject.Inject;
+import org.apache.eagle.app.test.AppJUnitRunner;
+import org.apache.eagle.app.test.ApplicationSimulator;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+
+@RunWith(AppJUnitRunner.class)
+public class MRRunningJobApplicationProviderTest {
+    @Inject
+    private
+    ApplicationSimulator simulator;
+
+    @Test
+    public void testStartAsManagedApplication(){
+        simulator.start(MRRunningJobApplicationProvider.class);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/b54a63e0/eagle-jpm/eagle-jpm-mr-running/src/test/java/org/apache/eagle/jpm/mr/running/MRRunningJobApplicationTest.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-mr-running/src/test/java/org/apache/eagle/jpm/mr/running/MRRunningJobApplicationTest.java b/eagle-jpm/eagle-jpm-mr-running/src/test/java/org/apache/eagle/jpm/mr/running/MRRunningJobApplicationTest.java
new file mode 100644
index 0000000..3ec9089
--- /dev/null
+++ b/eagle-jpm/eagle-jpm-mr-running/src/test/java/org/apache/eagle/jpm/mr/running/MRRunningJobApplicationTest.java
@@ -0,0 +1,27 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.eagle.jpm.mr.running;
+
+import com.typesafe.config.ConfigFactory;
+import org.junit.Test;
+
+public class MRRunningJobApplicationTest {
+    @Test
+    public void testRunApplicationWithCLI(){
+        new MRRunningJobApplication().run(ConfigFactory.load());
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/b54a63e0/eagle-jpm/eagle-jpm-service/src/main/java/org/apache/eagle/service/jpm/MRJobExecutionResource.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-service/src/main/java/org/apache/eagle/service/jpm/MRJobExecutionResource.java b/eagle-jpm/eagle-jpm-service/src/main/java/org/apache/eagle/service/jpm/MRJobExecutionResource.java
index be90456..3e487ae 100644
--- a/eagle-jpm/eagle-jpm-service/src/main/java/org/apache/eagle/service/jpm/MRJobExecutionResource.java
+++ b/eagle-jpm/eagle-jpm-service/src/main/java/org/apache/eagle/service/jpm/MRJobExecutionResource.java
@@ -18,28 +18,30 @@
 
 package org.apache.eagle.service.jpm;
 
-import org.apache.commons.lang.time.StopWatch;
+
+import static org.apache.eagle.jpm.util.MRJobTagName.JOB_ID;
+
 import org.apache.eagle.jpm.mr.runningentity.TaskExecutionAPIEntity;
 import org.apache.eagle.jpm.util.Constants;
 import org.apache.eagle.log.base.taggedlog.TaggedLogAPIEntity;
 import org.apache.eagle.log.entity.GenericServiceAPIResponseEntity;
 import org.apache.eagle.service.generic.GenericEntityServiceResource;
+
+import org.apache.commons.lang.time.StopWatch;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import java.util.*;
 import javax.ws.rs.*;
 import javax.ws.rs.core.MediaType;
-import java.util.*;
-
-import static org.apache.eagle.jpm.util.MRJobTagName.JOB_ID;
 
 @Path("mrJobs")
 public class MRJobExecutionResource {
     GenericEntityServiceResource resource = new GenericEntityServiceResource();
-    public final static String ELAPSEDMS = "elapsedms";
-    public final static String TOTAL_RESULTS = "totalResults";
+    public static final String ELAPSEDMS = "elapsedms";
+    public static final String TOTAL_RESULTS = "totalResults";
 
-    private final static Logger LOG = LoggerFactory.getLogger(MRJobExecutionResource.class);
+    private static final Logger LOG = LoggerFactory.getLogger(MRJobExecutionResource.class);
 
     @GET
     @Produces(MediaType.APPLICATION_JSON)
@@ -57,7 +59,7 @@ public class MRJobExecutionResource {
         List<TaggedLogAPIEntity> jobs = new ArrayList<>();
         List<TaggedLogAPIEntity> finishedJobs = new ArrayList<>();
         Set<String> jobIds = new HashSet<>();
-        Map<String,Object> meta = new HashMap<>();
+        final Map<String,Object> meta = new HashMap<>();
         StopWatch stopWatch = new StopWatch();
 
         stopWatch.start();
@@ -134,7 +136,7 @@ public class MRJobExecutionResource {
         List<TaggedLogAPIEntity> jobs = new ArrayList<>();
         Set<String> jobIds = new HashSet<>();
         String condition = buildCondition(jobId, jobDefId, site);
-        int pageSize = Integer.MAX_VALUE;
+        final int pageSize = Integer.MAX_VALUE;
         if (condition == null) {
             response.setException(new Exception("Search condition is empty"));
             response.setSuccess(false);
@@ -142,7 +144,7 @@ public class MRJobExecutionResource {
         }
         LOG.debug("search condition=" + condition);
 
-        Map<String,Object> meta = new HashMap<>();
+        final Map<String,Object> meta = new HashMap<>();
         StopWatch stopWatch = new StopWatch();
         stopWatch.start();
         String queryFormat = "%s[%s]{*}";
@@ -245,11 +247,11 @@ public class MRJobExecutionResource {
 
         List<Long> times = parseTimeList(timeList);
         String query = String.format("%s[@site=\"%s\" AND @jobId=\"%s\"]{*}", Constants.JPA_TASK_EXECUTION_SERVICE_NAME, site, jobId);
-        GenericServiceAPIResponseEntity<org.apache.eagle.jpm.mr.historyentity.TaskExecutionAPIEntity> history_res =
+        GenericServiceAPIResponseEntity<org.apache.eagle.jpm.mr.historyentity.TaskExecutionAPIEntity> historyRes =
                 resource.search(query,  null, null, Integer.MAX_VALUE, null, false, true,  0L, 0, true, 0, null, false);
-        if (history_res.isSuccess() && history_res.getObj() != null && history_res.getObj().size() > 0) {
+        if (historyRes.isSuccess() && historyRes.getObj() != null && historyRes.getObj().size() > 0) {
             initTaskCountList(runningTaskCount, finishedTaskCount, times, new HistoryTaskComparator());
-            for (org.apache.eagle.jpm.mr.historyentity.TaskExecutionAPIEntity o : history_res.getObj()) {
+            for (org.apache.eagle.jpm.mr.historyentity.TaskExecutionAPIEntity o : historyRes.getObj()) {
                 int index = getPosition(times, o.getDuration());
                 MRJobTaskGroupResponse.UnitTaskCount counter = finishedTaskCount.get(index);
                 counter.taskCount++;
@@ -257,11 +259,11 @@ public class MRJobExecutionResource {
             }
         } else {
             query = String.format("%s[@site=\"%s\" AND @jobId=\"%s\"]{*}", Constants.JPA_RUNNING_TASK_EXECUTION_SERVICE_NAME, site, jobId);
-            GenericServiceAPIResponseEntity<TaskExecutionAPIEntity> running_res =
+            GenericServiceAPIResponseEntity<TaskExecutionAPIEntity> runningRes =
                     resource.search(query,  null, null, Integer.MAX_VALUE, null, false, true,  0L, 0, true, 0, null, false);
-            if (running_res.isSuccess() && running_res.getObj() != null) {
+            if (runningRes.isSuccess() && runningRes.getObj() != null) {
                 initTaskCountList(runningTaskCount, finishedTaskCount, times, new RunningTaskComparator());
-                for (TaskExecutionAPIEntity o : running_res.getObj()) {
+                for (TaskExecutionAPIEntity o : runningRes.getObj()) {
                     int index = getPosition(times, o.getDuration());
                     if (o.getTaskStatus().equalsIgnoreCase(Constants.TaskState.RUNNING.toString())) {
                         MRJobTaskGroupResponse.UnitTaskCount counter = runningTaskCount.get(index);

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/b54a63e0/eagle-jpm/eagle-jpm-spark-running/src/main/java/org/apache/eagle/jpm/spark/running/storm/SparkRunningJobFetchSpout.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-spark-running/src/main/java/org/apache/eagle/jpm/spark/running/storm/SparkRunningJobFetchSpout.java b/eagle-jpm/eagle-jpm-spark-running/src/main/java/org/apache/eagle/jpm/spark/running/storm/SparkRunningJobFetchSpout.java
index ccdfe79..7162bac 100644
--- a/eagle-jpm/eagle-jpm-spark-running/src/main/java/org/apache/eagle/jpm/spark/running/storm/SparkRunningJobFetchSpout.java
+++ b/eagle-jpm/eagle-jpm-spark-running/src/main/java/org/apache/eagle/jpm/spark/running/storm/SparkRunningJobFetchSpout.java
@@ -18,12 +18,6 @@
 
 package org.apache.eagle.jpm.spark.running.storm;
 
-import backtype.storm.spout.SpoutOutputCollector;
-import backtype.storm.task.TopologyContext;
-import backtype.storm.topology.OutputFieldsDeclarer;
-import backtype.storm.topology.base.BaseRichSpout;
-import backtype.storm.tuple.Fields;
-import backtype.storm.tuple.Values;
 import org.apache.eagle.jpm.spark.running.SparkRunningJobAppConfig;
 import org.apache.eagle.jpm.spark.running.entities.SparkAppEntity;
 import org.apache.eagle.jpm.spark.running.recover.SparkRunningJobManager;
@@ -31,6 +25,13 @@ import org.apache.eagle.jpm.util.Constants;
 import org.apache.eagle.jpm.util.resourcefetch.RMResourceFetcher;
 import org.apache.eagle.jpm.util.resourcefetch.ResourceFetcher;
 import org.apache.eagle.jpm.util.resourcefetch.model.AppInfo;
+
+import backtype.storm.spout.SpoutOutputCollector;
+import backtype.storm.task.TopologyContext;
+import backtype.storm.topology.OutputFieldsDeclarer;
+import backtype.storm.topology.base.BaseRichSpout;
+import backtype.storm.tuple.Fields;
+import backtype.storm.tuple.Values;
 import org.apache.zookeeper.KeeperException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -143,6 +144,7 @@ public class SparkRunningJobFetchSpout extends BaseRichSpout {
             try {
                 Thread.sleep(jobExtractorConfig.fetchRunningJobInterval * 1000);
             } catch (Exception e) {
+                // ignored
             }
         }
     }

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/b54a63e0/eagle-jpm/eagle-jpm-spark-running/src/main/java/org/apache/eagle/jpm/spark/running/storm/SparkRunningJobParseBolt.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-spark-running/src/main/java/org/apache/eagle/jpm/spark/running/storm/SparkRunningJobParseBolt.java b/eagle-jpm/eagle-jpm-spark-running/src/main/java/org/apache/eagle/jpm/spark/running/storm/SparkRunningJobParseBolt.java
index c41804b..a497e29 100644
--- a/eagle-jpm/eagle-jpm-spark-running/src/main/java/org/apache/eagle/jpm/spark/running/storm/SparkRunningJobParseBolt.java
+++ b/eagle-jpm/eagle-jpm-spark-running/src/main/java/org/apache/eagle/jpm/spark/running/storm/SparkRunningJobParseBolt.java
@@ -53,6 +53,7 @@ public class SparkRunningJobParseBolt extends BaseRichBolt {
     private ExecutorService executorService;
     private Map<String, SparkApplicationParser> runningSparkParsers;
     private ResourceFetcher resourceFetcher;
+
     public SparkRunningJobParseBolt(SparkRunningJobAppConfig.ZKStateConfig zkStateConfig,
                                     SparkRunningJobAppConfig.EagleServiceConfig eagleServiceConfig,
                                     SparkRunningJobAppConfig.EndpointConfig endpointConfig,
@@ -94,8 +95,8 @@ public class SparkRunningJobParseBolt extends BaseRichBolt {
                     LOG.info("remove parser {}", appId);
                 });
 
-        if (appInfo.getState().equals(Constants.AppState.FINISHED.toString()) ||
-                applicationParser.status() == SparkApplicationParser.ParserStatus.FINISHED) {
+        if (appInfo.getState().equals(Constants.AppState.FINISHED.toString())
+            || applicationParser.status() == SparkApplicationParser.ParserStatus.FINISHED) {
             applicationParser.setStatus(SparkApplicationParser.ParserStatus.RUNNING);
             executorService.execute(applicationParser);
         }


[24/52] [abbrv] incubator-eagle git commit: [EAGLE-504] Fix JSONUtils to avoid null pointer exceptions when getting json values.

Posted by yo...@apache.org.
[EAGLE-504] Fix JSONUtils to avoid null pointer exceptions when getting json values.

Author: pkuwm <ih...@gmail.com>

Closes #396 from pkuwm/EAGLE-504.


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

Branch: refs/heads/master
Commit: 3cc183017da19889cff6bb2c57ecfe41dfbc71e8
Parents: 4f4fd0c
Author: pkuwm <ih...@gmail.com>
Authored: Mon Aug 29 13:57:49 2016 +0800
Committer: Hao Chen <ha...@apache.org>
Committed: Mon Aug 29 13:57:49 2016 +0800

----------------------------------------------------------------------
 .../jpm/spark/crawl/JHFSparkEventReader.java    | 165 +++++++++----------
 .../eagle/jpm/spark/crawl/JHFSparkParser.java   |  10 +-
 eagle-jpm/eagle-jpm-util/pom.xml                |   4 +
 .../org/apache/eagle/jpm/util/JSONUtil.java     |  66 --------
 .../org/apache/eagle/jpm/util/JSONUtils.java    | 108 ++++++++++++
 5 files changed, 197 insertions(+), 156 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/3cc18301/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/spark/crawl/JHFSparkEventReader.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/spark/crawl/JHFSparkEventReader.java b/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/spark/crawl/JHFSparkEventReader.java
index 1cd5a77..fe02da5 100644
--- a/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/spark/crawl/JHFSparkEventReader.java
+++ b/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/spark/crawl/JHFSparkEventReader.java
@@ -149,17 +149,17 @@ public class JHFSparkEventReader {
         entities.addAll(this.executors.values());
         entities.add(this.app);
 
-        long appStartTime = JSONUtil.getLong(event, "Timestamp");
+        long appStartTime = JSONUtils.getLong(event, "Timestamp", lastEventTime);
         for (TaggedLogAPIEntity entity : entities) {
-            entity.getTags().put(SparkJobTagName.SPARK_APP_ID.toString(), JSONUtil.getString(event, "App ID"));
-            entity.getTags().put(SparkJobTagName.SPARK_APP_NAME.toString(), JSONUtil.getString(event, "App Name"));
+            entity.getTags().put(SparkJobTagName.SPARK_APP_ID.toString(), JSONUtils.getString(event, "App ID"));
+            entity.getTags().put(SparkJobTagName.SPARK_APP_NAME.toString(), JSONUtils.getString(event, "App Name"));
             // In yarn-client mode, attemptId is not available in the log, so we set attemptId = 1.
-            String attemptId = isClientMode(this.app.getConfig()) ? "1" : JSONUtil.getString(event, "App Attempt ID");
+            String attemptId = isClientMode(this.app.getConfig()) ? "1" : JSONUtils.getString(event, "App Attempt ID");
             entity.getTags().put(SparkJobTagName.SPARK_APP_ATTEMPT_ID.toString(), attemptId);
             // the second argument of getNormalizeName() is changed to null because the original code contains sensitive text
             // original second argument looks like: this.app.getConfig().getConfig().get("xxx"), "xxx" is the sensitive text
-            entity.getTags().put(SparkJobTagName.SPARK_APP_NORM_NAME.toString(), this.getNormalizedName(JSONUtil.getString(event, "App Name"), null));
-            entity.getTags().put(SparkJobTagName.SPARK_USER.toString(), JSONUtil.getString(event, "User"));
+            entity.getTags().put(SparkJobTagName.SPARK_APP_NORM_NAME.toString(), this.getNormalizedName(JSONUtils.getString(event, "App Name"), null));
+            entity.getTags().put(SparkJobTagName.SPARK_USER.toString(), JSONUtils.getString(event, "User"));
 
             entity.setTimestamp(appStartTime);
         }
@@ -170,25 +170,25 @@ public class JHFSparkEventReader {
 
     private void handleExecutorAdd(JSONObject event) throws Exception {
         String executorID = (String) event.get("Executor ID");
-        long executorAddTime = JSONUtil.getLong(event, "Timestamp");
+        long executorAddTime = JSONUtils.getLong(event, "Timestamp", lastEventTime);
         this.lastEventTime = executorAddTime;
         SparkExecutor executor = this.initiateExecutor(executorID, executorAddTime);
 
-        JSONObject executorInfo = JSONUtil.getJSONObject(event, "Executor Info");
+        JSONObject executorInfo = JSONUtils.getJSONObject(event, "Executor Info");
 
     }
 
     private void handleBlockManagerAdd(JSONObject event) throws Exception {
-        long maxMemory = JSONUtil.getLong(event, "Maximum Memory");
-        long timestamp = JSONUtil.getLong(event, "Timestamp");
+        long maxMemory = JSONUtils.getLong(event, "Maximum Memory");
+        long timestamp = JSONUtils.getLong(event, "Timestamp", lastEventTime);
         this.lastEventTime = timestamp;
-        JSONObject blockInfo = JSONUtil.getJSONObject(event, "Block Manager ID");
-        String executorID = JSONUtil.getString(blockInfo, "Executor ID");
-        String hostport = String.format("%s:%s", JSONUtil.getString(blockInfo, "Host"), JSONUtil.getLong(blockInfo, "Port"));
+        JSONObject blockInfo = JSONUtils.getJSONObject(event, "Block Manager ID");
+        String executorID = JSONUtils.getString(blockInfo, "Executor ID");
+        String hostAndPort = JSONUtils.getString(blockInfo, "Host") + ":" + JSONUtils.getLong(blockInfo, "Port");
 
         SparkExecutor executor = this.initiateExecutor(executorID, timestamp);
         executor.setMaxMemory(maxMemory);
-        executor.setHostPort(hostport);
+        executor.setHostPort(hostAndPort);
     }
 
     private void handleTaskStart(JSONObject event) {
@@ -196,47 +196,47 @@ public class JHFSparkEventReader {
     }
 
     private void handleTaskEnd(JSONObject event) {
-        JSONObject taskInfo = JSONUtil.getJSONObject(event, "Task Info");
-        int taskId = JSONUtil.getInt(taskInfo, "Task ID");
+        JSONObject taskInfo = JSONUtils.getJSONObject(event, "Task Info");
+        int taskId = JSONUtils.getInt(taskInfo, "Task ID");
         SparkTask task = tasks.get(taskId);
         if (task == null) {
             return;
         }
 
-        task.setFailed(JSONUtil.getBoolean(taskInfo, "Failed"));
-        JSONObject taskMetrics = JSONUtil.getJSONObject(event, "Task Metrics");
+        task.setFailed(JSONUtils.getBoolean(taskInfo, "Failed"));
+        JSONObject taskMetrics = JSONUtils.getJSONObject(event, "Task Metrics");
         if (null != taskMetrics) {
-            task.setExecutorDeserializeTime(JSONUtil.getLong(taskMetrics, "Executor Deserialize Time"));
-            task.setExecutorRunTime(JSONUtil.getLong(taskMetrics, "Executor Run Time"));
-            task.setJvmGcTime(JSONUtil.getLong(taskMetrics, "JVM GC Time"));
-            task.setResultSize(JSONUtil.getLong(taskMetrics, "Result Size"));
-            task.setResultSerializationTime(JSONUtil.getLong(taskMetrics, "Result Serialization Time"));
-            task.setMemoryBytesSpilled(JSONUtil.getLong(taskMetrics, "Memory Bytes Spilled"));
-            task.setDiskBytesSpilled(JSONUtil.getLong(taskMetrics, "Disk Bytes Spilled"));
-
-            JSONObject inputMetrics = JSONUtil.getJSONObject(taskMetrics, "Input Metrics");
+            task.setExecutorDeserializeTime(JSONUtils.getLong(taskMetrics, "Executor Deserialize Time", lastEventTime));
+            task.setExecutorRunTime(JSONUtils.getLong(taskMetrics, "Executor Run Time", lastEventTime));
+            task.setJvmGcTime(JSONUtils.getLong(taskMetrics, "JVM GC Time", lastEventTime));
+            task.setResultSize(JSONUtils.getLong(taskMetrics, "Result Size"));
+            task.setResultSerializationTime(JSONUtils.getLong(taskMetrics, "Result Serialization Time", lastEventTime));
+            task.setMemoryBytesSpilled(JSONUtils.getLong(taskMetrics, "Memory Bytes Spilled"));
+            task.setDiskBytesSpilled(JSONUtils.getLong(taskMetrics, "Disk Bytes Spilled"));
+
+            JSONObject inputMetrics = JSONUtils.getJSONObject(taskMetrics, "Input Metrics");
             if (null != inputMetrics) {
-                task.setInputBytes(JSONUtil.getLong(inputMetrics, "Bytes Read"));
-                task.setInputRecords(JSONUtil.getLong(inputMetrics, "Records Read"));
+                task.setInputBytes(JSONUtils.getLong(inputMetrics, "Bytes Read"));
+                task.setInputRecords(JSONUtils.getLong(inputMetrics, "Records Read"));
             }
 
-            JSONObject outputMetrics = JSONUtil.getJSONObject(taskMetrics, "Output Metrics");
+            JSONObject outputMetrics = JSONUtils.getJSONObject(taskMetrics, "Output Metrics");
             if (null != outputMetrics) {
-                task.setOutputBytes(JSONUtil.getLong(outputMetrics, "Bytes Written"));
-                task.setOutputRecords(JSONUtil.getLong(outputMetrics, "Records Written"));
+                task.setOutputBytes(JSONUtils.getLong(outputMetrics, "Bytes Written"));
+                task.setOutputRecords(JSONUtils.getLong(outputMetrics, "Records Written"));
             }
 
-            JSONObject shuffleWriteMetrics = JSONUtil.getJSONObject(taskMetrics, "Shuffle Write Metrics");
+            JSONObject shuffleWriteMetrics = JSONUtils.getJSONObject(taskMetrics, "Shuffle Write Metrics");
             if (null != shuffleWriteMetrics) {
-                task.setShuffleWriteBytes(JSONUtil.getLong(shuffleWriteMetrics, "Shuffle Bytes Written"));
-                task.setShuffleWriteRecords(JSONUtil.getLong(shuffleWriteMetrics, "Shuffle Records Written"));
+                task.setShuffleWriteBytes(JSONUtils.getLong(shuffleWriteMetrics, "Shuffle Bytes Written"));
+                task.setShuffleWriteRecords(JSONUtils.getLong(shuffleWriteMetrics, "Shuffle Records Written"));
             }
 
-            JSONObject shuffleReadMetrics = JSONUtil.getJSONObject(taskMetrics, "Shuffle Read Metrics");
+            JSONObject shuffleReadMetrics = JSONUtils.getJSONObject(taskMetrics, "Shuffle Read Metrics");
             if (null != shuffleReadMetrics) {
-                task.setShuffleReadLocalBytes(JSONUtil.getLong(shuffleReadMetrics, "Local Bytes Read"));
-                task.setShuffleReadRemoteBytes(JSONUtil.getLong(shuffleReadMetrics, "Remote Bytes Read"));
-                task.setShuffleReadRecords(JSONUtil.getLong(shuffleReadMetrics, "Total Records Read"));
+                task.setShuffleReadLocalBytes(JSONUtils.getLong(shuffleReadMetrics, "Local Bytes Read"));
+                task.setShuffleReadRemoteBytes(JSONUtils.getLong(shuffleReadMetrics, "Remote Bytes Read"));
+                task.setShuffleReadRecords(JSONUtils.getLong(shuffleReadMetrics, "Total Records Read"));
             }
         } else {
             //for tasks success without task metrics, save in the end if no other information
@@ -257,25 +257,25 @@ public class JHFSparkEventReader {
         task.setTags(new HashMap<>(this.app.getTags()));
         task.setTimestamp(app.getTimestamp());
 
-        task.getTags().put(SparkJobTagName.SPARK_SATGE_ID.toString(), Long.toString(JSONUtil.getLong(event, "Stage ID")));
-        task.getTags().put(SparkJobTagName.SPARK_STAGE_ATTEMPT_ID.toString(), Long.toString(JSONUtil.getLong(event, "Stage Attempt ID")));
+        task.getTags().put(SparkJobTagName.SPARK_SATGE_ID.toString(), Long.toString(JSONUtils.getLong(event, "Stage ID")));
+        task.getTags().put(SparkJobTagName.SPARK_STAGE_ATTEMPT_ID.toString(), Long.toString(JSONUtils.getLong(event, "Stage Attempt ID")));
 
-        JSONObject taskInfo = JSONUtil.getJSONObject(event, "Task Info");
-        int taskId = JSONUtil.getInt(taskInfo, "Task ID");
+        JSONObject taskInfo = JSONUtils.getJSONObject(event, "Task Info");
+        int taskId = JSONUtils.getInt(taskInfo, "Task ID");
         task.setTaskId(taskId);
 
-        task.getTags().put(SparkJobTagName.SPARK_TASK_INDEX.toString(), Integer.toString(JSONUtil.getInt(taskInfo, "Index")));
-        task.getTags().put(SparkJobTagName.SPARK_TASK_ATTEMPT_ID.toString(), Integer.toString(JSONUtil.getInt(taskInfo, "Attempt")));
-        long launchTime = JSONUtil.getLong(taskInfo, "Launch Time");
+        task.getTags().put(SparkJobTagName.SPARK_TASK_INDEX.toString(), Integer.toString(JSONUtils.getInt(taskInfo, "Index")));
+        task.getTags().put(SparkJobTagName.SPARK_TASK_ATTEMPT_ID.toString(), Integer.toString(JSONUtils.getInt(taskInfo, "Attempt")));
+        long launchTime = JSONUtils.getLong(taskInfo, "Launch Time", lastEventTime);
         this.lastEventTime = launchTime;
         if (taskId == 0) {
             this.setFirstTaskLaunchTime(launchTime);
         }
         task.setLaunchTime(launchTime);
-        task.setExecutorId(JSONUtil.getString(taskInfo, "Executor ID"));
-        task.setHost(JSONUtil.getString(taskInfo, "Host"));
-        task.setTaskLocality(JSONUtil.getString(taskInfo, "Locality"));
-        task.setSpeculative(JSONUtil.getBoolean(taskInfo, "Speculative"));
+        task.setExecutorId(JSONUtils.getString(taskInfo, "Executor ID"));
+        task.setHost(JSONUtils.getString(taskInfo, "Host"));
+        task.setTaskLocality(JSONUtils.getString(taskInfo, "Locality"));
+        task.setSpeculative(JSONUtils.getBoolean(taskInfo, "Speculative"));
 
         tasks.put(task.getTaskId(), task);
         return task;
@@ -285,18 +285,14 @@ public class JHFSparkEventReader {
         this.firstTaskLaunchTime = launchTime;
     }
 
-    private long getFirstTaskLaunchTime() {
-        return this.firstTaskLaunchTime;
-    }
-
     private void handleJobStart(JSONObject event) {
         SparkJob job = new SparkJob();
         job.setTags(new HashMap<>(this.app.getTags()));
         job.setTimestamp(app.getTimestamp());
 
-        int jobId = JSONUtil.getInt(event, "Job ID");
+        int jobId = JSONUtils.getInt(event, "Job ID");
         job.getTags().put(SparkJobTagName.SPARK_JOB_ID.toString(), Integer.toString(jobId));
-        long submissionTime = JSONUtil.getLong(event, "Submission Time");
+        long submissionTime = JSONUtils.getLong(event, "Submission Time", lastEventTime);
         job.setSubmissionTime(submissionTime);
         this.lastEventTime = submissionTime;
 
@@ -307,22 +303,23 @@ public class JHFSparkEventReader {
         this.jobs.put(jobId, job);
         this.jobStageMap.put(jobId, new HashSet<String>());
 
-        JSONArray stages = JSONUtil.getJSONArray(event, "Stage Infos");
-        job.setNumStages(stages.size());
-        for (int i = 0; i < stages.size(); i++) {
+        JSONArray stages = JSONUtils.getJSONArray(event, "Stage Infos");
+        int stagesSize = (stages == null ? 0 : stages.size());
+        job.setNumStages(stagesSize);
+        for (int i = 0; i < stagesSize; i++) {
             JSONObject stageInfo = (JSONObject) stages.get(i);
-            int stageId = JSONUtil.getInt(stageInfo, "Stage ID");
-            int stageAttemptId = JSONUtil.getInt(stageInfo, "Stage Attempt ID");
-            String stageName = JSONUtil.getString(stageInfo, "Stage Name");
-            int numTasks = JSONUtil.getInt(stageInfo, "Number of Tasks");
+            int stageId = JSONUtils.getInt(stageInfo, "Stage ID");
+            int stageAttemptId = JSONUtils.getInt(stageInfo, "Stage Attempt ID");
+            String stageName = JSONUtils.getString(stageInfo, "Stage Name");
+            int numTasks = JSONUtils.getInt(stageInfo, "Number of Tasks");
             this.initiateStage(jobId, stageId, stageAttemptId, stageName, numTasks);
         }
     }
 
     private void handleStageSubmit(JSONObject event) {
-        JSONObject stageInfo = JSONUtil.getJSONObject(event, "Stage Info");
-        int stageId = JSONUtil.getInt(stageInfo, "Stage ID");
-        int stageAttemptId = JSONUtil.getInt(stageInfo, "Stage Attempt ID");
+        JSONObject stageInfo = JSONUtils.getJSONObject(event, "Stage Info");
+        int stageId = JSONUtils.getInt(stageInfo, "Stage ID");
+        int stageAttemptId = JSONUtils.getInt(stageInfo, "Stage Attempt ID");
         String key = this.generateStageKey(Integer.toString(stageId), Integer.toString(stageAttemptId));
         stageTaskStatusMap.put(key, new HashMap<Integer, Boolean>());
 
@@ -333,28 +330,26 @@ public class JHFSparkEventReader {
                 SparkStage stage = stages.get(baseAttempt);
                 String jobId = stage.getTags().get(SparkJobTagName.SPARK_JOB_ID.toString());
 
-                String stageName = JSONUtil.getString(event, "Stage Name");
-                int numTasks = JSONUtil.getInt(stageInfo, "Number of Tasks");
+                String stageName = JSONUtils.getString(event, "Stage Name");
+                int numTasks = JSONUtils.getInt(stageInfo, "Number of Tasks");
                 this.initiateStage(Integer.parseInt(jobId), stageId, stageAttemptId, stageName, numTasks);
             }
         }
     }
 
     private void handleStageComplete(JSONObject event) {
-        JSONObject stageInfo = JSONUtil.getJSONObject(event, "Stage Info");
-        int stageId = JSONUtil.getInt(stageInfo, "Stage ID");
-        int stageAttemptId = JSONUtil.getInt(stageInfo, "Stage Attempt ID");
+        JSONObject stageInfo = JSONUtils.getJSONObject(event, "Stage Info");
+        int stageId = JSONUtils.getInt(stageInfo, "Stage ID");
+        int stageAttemptId = JSONUtils.getInt(stageInfo, "Stage Attempt ID");
         String key = this.generateStageKey(Integer.toString(stageId), Integer.toString(stageAttemptId));
         SparkStage stage = stages.get(key);
 
         // If "Submission Time" is not available, use the "Launch Time" of "Task ID" = 0.
-        Long submissionTime = JSONUtil.getLong(stageInfo, "Submission Time");
-        if (submissionTime == null) {
-            submissionTime = this.getFirstTaskLaunchTime();
-        }
+        Long submissionTime = JSONUtils.getLong(stageInfo, "Submission Time", firstTaskLaunchTime);
+
         stage.setSubmitTime(submissionTime);
 
-        long completeTime = JSONUtil.getLong(stageInfo, "Completion Time");
+        long completeTime = JSONUtils.getLong(stageInfo, "Completion Time", lastEventTime);
         stage.setCompleteTime(completeTime);
         this.lastEventTime = completeTime;
 
@@ -366,23 +361,23 @@ public class JHFSparkEventReader {
     }
 
     private void handleExecutorRemoved(JSONObject event) {
-        String executorID = JSONUtil.getString(event, "Executor ID");
+        String executorID = JSONUtils.getString(event, "Executor ID");
         SparkExecutor executor = executors.get(executorID);
-        long removedTime = JSONUtil.getLong(event, "Timestamp");
+        long removedTime = JSONUtils.getLong(event, "Timestamp", lastEventTime);
         executor.setEndTime(removedTime);
         this.lastEventTime = removedTime;
     }
 
     private void handleJobEnd(JSONObject event) {
-        int jobId = JSONUtil.getInt(event, "Job ID");
+        int jobId = JSONUtils.getInt(event, "Job ID");
         SparkJob job = jobs.get(jobId);
 
-        long completionTime = JSONUtil.getLong(event, "Completion Time");
+        long completionTime = JSONUtils.getLong(event, "Completion Time", lastEventTime);
         job.setCompletionTime(completionTime);
         this.lastEventTime = completionTime;
 
-        JSONObject jobResult = JSONUtil.getJSONObject(event, "Job Result");
-        String result = JSONUtil.getString(jobResult, "Result");
+        JSONObject jobResult = JSONUtils.getJSONObject(event, "Job Result");
+        String result = JSONUtils.getString(jobResult, "Result");
         if (result.equalsIgnoreCase("JobSucceeded")) {
             job.setStatus(SparkEntityConstant.SparkJobStatus.SUCCEEDED.toString());
         } else {
@@ -391,7 +386,7 @@ public class JHFSparkEventReader {
     }
 
     private void handleAppEnd(JSONObject event) {
-        long endTime = JSONUtil.getLong(event, "Timestamp");
+        long endTime = JSONUtils.getLong(event, "Timestamp", lastEventTime);
         app.setEndTime(endTime);
         this.lastEventTime = endTime;
     }
@@ -634,7 +629,7 @@ public class JHFSparkEventReader {
 
 
     private String generateStageKey(String stageId, String stageAttemptId) {
-        return String.format("%s-%s", stageId, stageAttemptId);
+        return stageId + "-" + stageAttemptId;
     }
 
     private void initiateStage(int jobId, int stageId, int stageAttemptId, String name, int numTasks) {
@@ -679,7 +674,7 @@ public class JHFSparkEventReader {
     }
 
     private void flushEntities(Object entity, boolean forceFlush) {
-        this.flushEntities(Arrays.asList(entity), forceFlush);
+        this.flushEntities(Collections.singletonList(entity), forceFlush);
     }
 
     private void flushEntities(Collection entities, boolean forceFlush) {

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/3cc18301/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/spark/crawl/JHFSparkParser.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/spark/crawl/JHFSparkParser.java b/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/spark/crawl/JHFSparkParser.java
index 05cdd7e..2ba3c73 100644
--- a/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/spark/crawl/JHFSparkParser.java
+++ b/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/spark/crawl/JHFSparkParser.java
@@ -45,13 +45,13 @@ public class JHFSparkParser implements JHFParserBase {
         try (BufferedReader reader = new BufferedReader(new InputStreamReader(is))) {
             for (String line = reader.readLine(); line != null; line = reader.readLine()) {
                 isValidJson = true;
-                try {
-                    JSONObject eventObj = parseAndValidateJSON(line);
-                    if (isValidJson) {
+                JSONObject eventObj = parseAndValidateJSON(line);
+                if (isValidJson) {
+                    try {
                         this.eventReader.read(eventObj);
+                    } catch (Exception e) {
+                        logger.error("Fail to read eventObj. Exception: " + e);
                     }
-                } catch(Exception e) {
-                    logger.error(String.format("Fail to parse %s.", line), e);
                 }
             }
 

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/3cc18301/eagle-jpm/eagle-jpm-util/pom.xml
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-util/pom.xml b/eagle-jpm/eagle-jpm-util/pom.xml
index 8868dbe..e424e49 100644
--- a/eagle-jpm/eagle-jpm-util/pom.xml
+++ b/eagle-jpm/eagle-jpm-util/pom.xml
@@ -37,6 +37,10 @@
             <version>1.1.1</version>
         </dependency>
         <dependency>
+            <groupId>org.json</groupId>
+            <artifactId>json</artifactId>
+        </dependency>
+        <dependency>
             <groupId>org.apache.storm</groupId>
             <artifactId>storm-core</artifactId>
             <version>${storm.version}</version>

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/3cc18301/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/JSONUtil.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/JSONUtil.java b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/JSONUtil.java
deleted file mode 100644
index 9804a3b..0000000
--- a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/JSONUtil.java
+++ /dev/null
@@ -1,66 +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.eagle.jpm.util;
-
-import org.json.simple.JSONArray;
-import org.json.simple.JSONObject;
-
-public class JSONUtil {
-
-    public static String getString(JSONObject obj, String field) {
-        if (obj.containsKey(field)) {
-            return (String) obj.get(field);
-        }
-        return null;
-    }
-
-    public static Integer getInt(JSONObject obj, String field) {
-        if (obj.containsKey(field)) {
-            return ((Long) obj.get(field)).intValue();
-        }
-        return null;
-    }
-
-    public static Long getLong(JSONObject obj, String field) {
-        if (obj.containsKey(field)) {
-            return (Long) obj.get(field);
-        }
-        return null;
-    }
-
-    public static Boolean getBoolean(JSONObject obj, String field) {
-        if (obj.containsKey(field)) {
-            return (Boolean) obj.get(field);
-        }
-        return null;
-    }
-
-    public static JSONObject getJSONObject(JSONObject obj, String field) {
-        if (obj.containsKey(field)) {
-            return (JSONObject) obj.get(field);
-        }
-        return null;
-    }
-
-    public static JSONArray getJSONArray(JSONObject obj, String field) {
-        if (obj.containsKey(field)) {
-            return (JSONArray) obj.get(field);
-        }
-        return null;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/3cc18301/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/JSONUtils.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/JSONUtils.java b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/JSONUtils.java
new file mode 100644
index 0000000..38500b0
--- /dev/null
+++ b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/JSONUtils.java
@@ -0,0 +1,108 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.eagle.jpm.util;
+
+import org.apache.commons.lang.StringUtils;
+import org.json.JSONException;
+import org.json.simple.JSONArray;
+import org.json.simple.JSONObject;
+
+public class JSONUtils {
+
+    public static String getString(JSONObject obj, String field) {
+        if (obj == null || StringUtils.isEmpty(field)) {
+            return null;
+        }
+
+        try {
+            return obj.get(field).toString();
+        } catch (JSONException e) {
+            e.printStackTrace();
+            return null;
+        }
+    }
+
+    public static int getInt(JSONObject obj, String field) {
+        if (obj == null || StringUtils.isEmpty(field)) {
+            return 0;
+        }
+
+        try {
+            return (int) obj.get(field);
+        } catch (JSONException e) {
+            e.printStackTrace();
+            return 0;
+        }
+    }
+
+    public static long getLong(JSONObject obj, String field) {
+        return getLong(obj, field, 0L);
+    }
+
+    public static long getLong(JSONObject obj, String field, long defaultValue) {
+        if (obj == null || StringUtils.isEmpty(field)) {
+            return defaultValue;
+        }
+
+        try {
+            return (long) obj.get(field);
+        } catch (JSONException e) {
+            e.printStackTrace();
+            return defaultValue;
+        }
+    }
+
+    public static Boolean getBoolean(JSONObject obj, String field) {
+        if (obj == null || StringUtils.isEmpty(field)) {
+            return false;
+        }
+
+        try {
+            return (Boolean) obj.get(field);
+        } catch (JSONException e) {
+            e.printStackTrace();
+            return false;
+        }
+    }
+
+    public static JSONObject getJSONObject(JSONObject obj, String field) {
+        if (obj == null || StringUtils.isEmpty(field)) {
+            return null;
+        }
+
+        try {
+            return (JSONObject) obj.get(field);
+        } catch (JSONException e) {
+            e.printStackTrace();
+            return null;
+        }
+    }
+
+    public static JSONArray getJSONArray(JSONObject obj, String field) {
+        if (obj == null || StringUtils.isEmpty(field)) {
+            return null;
+        }
+
+        try {
+            return (JSONArray) obj.get(field);
+        } catch (JSONException e) {
+            e.printStackTrace();
+            return null;
+        }
+    }
+}


[29/52] [abbrv] incubator-eagle git commit: [EAGLE-515] refactor zk manager to singleton

Posted by yo...@apache.org.
[EAGLE-515] refactor zk manager to singleton

Author: wujinhu <wu...@126.com>

Closes #411 from wujinhu/EAGLE-515.


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

Branch: refs/heads/master
Commit: c940f56c27296c5fec657bd240e96c5522199262
Parents: 8de69a8
Author: wujinhu <wu...@126.com>
Authored: Thu Sep 1 11:26:10 2016 +0800
Committer: Hao Chen <ha...@apache.org>
Committed: Thu Sep 1 11:26:10 2016 +0800

----------------------------------------------------------------------
 .../mr/history/crawler/JHFCrawlerDriverImpl.java   | 17 +++++++----------
 .../JobEntityCreationEagleServiceListener.java     |  4 +---
 .../jpm/mr/history/storm/JobHistorySpout.java      | 10 ++++------
 .../mr/history/zkres/JobHistoryZKStateManager.java |  8 +++++++-
 4 files changed, 19 insertions(+), 20 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/c940f56c/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/crawler/JHFCrawlerDriverImpl.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/crawler/JHFCrawlerDriverImpl.java b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/crawler/JHFCrawlerDriverImpl.java
index 1a17751..077f4e1 100644
--- a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/crawler/JHFCrawlerDriverImpl.java
+++ b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/crawler/JHFCrawlerDriverImpl.java
@@ -20,7 +20,7 @@ package org.apache.eagle.jpm.mr.history.crawler;
 
 import org.apache.eagle.jpm.mr.history.MRHistoryJobConfig;
 import org.apache.eagle.jpm.mr.history.parser.EagleJobStatus;
-import org.apache.eagle.jpm.mr.history.zkres.JobHistoryZKStateLCM;
+import org.apache.eagle.jpm.mr.history.zkres.JobHistoryZKStateManager;
 import org.apache.eagle.jpm.mr.historyentity.JobCountEntity;
 import org.apache.eagle.jpm.util.JobIdFilter;
 import org.apache.commons.lang3.tuple.Pair;
@@ -58,7 +58,6 @@ public class JHFCrawlerDriverImpl implements JHFCrawlerDriver {
     private JHFInputStreamCallback reader;
     protected boolean zeroBasedMonth = true;
 
-    private JobHistoryZKStateLCM zkStateLcm;
     private JobHistoryLCM jhfLCM;
     private JobIdFilter jobFilter;
     private int partitionId;
@@ -69,7 +68,6 @@ public class JHFCrawlerDriverImpl implements JHFCrawlerDriver {
     public JHFCrawlerDriverImpl(MRHistoryJobConfig.EagleServiceConfig eagleServiceConfig,
                                 MRHistoryJobConfig.JobExtractorConfig jobExtractorConfig,
                                 MRHistoryJobConfig.ControlConfig controlConfig, JHFInputStreamCallback reader,
-                                JobHistoryZKStateLCM zkStateLCM,
                                 JobHistoryLCM historyLCM, JobIdFilter jobFilter, int partitionId) throws Exception {
         this.eagleServiceConfig = eagleServiceConfig;
         this.jobExtractorConfig = jobExtractorConfig;
@@ -80,7 +78,6 @@ public class JHFCrawlerDriverImpl implements JHFCrawlerDriver {
         }
         this.reader = reader;
         jhfLCM = historyLCM;//new JobHistoryDAOImpl(jobHistoryConfig);
-        this.zkStateLcm = zkStateLCM;
         this.partitionId = partitionId;
         this.jobFilter = jobFilter;
         timeZone = TimeZone.getTimeZone(controlConfig.timeZone);
@@ -187,7 +184,7 @@ public class JHFCrawlerDriverImpl implements JHFCrawlerDriver {
                     jobHistoryFile,
                 reader);
         }
-        zkStateLcm.addProcessedJob(String.format(FORMAT_JOB_PROCESS_DATE,
+        JobHistoryZKStateManager.instance().addProcessedJob(String.format(FORMAT_JOB_PROCESS_DATE,
                 this.processDate.year,
                 this.processDate.month + 1,
                 this.processDate.day),
@@ -202,7 +199,7 @@ public class JHFCrawlerDriverImpl implements JHFCrawlerDriver {
     private void updateProcessDate() throws Exception {
         String line = String.format(FORMAT_JOB_PROCESS_DATE, this.processDate.year,
                 this.processDate.month + 1, this.processDate.day);
-        zkStateLcm.updateProcessedDate(partitionId, line);
+        JobHistoryZKStateManager.instance().updateProcessedDate(partitionId, line);
     }
 
     private int getActualMonth(int month) {
@@ -220,7 +217,7 @@ public class JHFCrawlerDriverImpl implements JHFCrawlerDriver {
     }
 
     private void readAndCacheLastProcessedDate() throws Exception {
-        String lastProcessedDate = zkStateLcm.readProcessedDate(partitionId);
+        String lastProcessedDate = JobHistoryZKStateManager.instance().readProcessedDate(partitionId);
         Matcher m = PATTERN_JOB_PROCESS_DATE.matcher(lastProcessedDate);
         if (m.find() && m.groupCount() == 3) {
             this.processDate.year = Integer.parseInt(m.group(1));
@@ -233,7 +230,7 @@ public class JHFCrawlerDriverImpl implements JHFCrawlerDriver {
         GregorianCalendar cal = new GregorianCalendar(timeZone);
         cal.set(this.processDate.year, this.processDate.month, this.processDate.day, 0, 0, 0);
         cal.add(Calendar.DATE, 1);
-        List<String> list = zkStateLcm.readProcessedJobs(String.format(FORMAT_JOB_PROCESS_DATE, cal.get(Calendar.YEAR),
+        List<String> list = JobHistoryZKStateManager.instance().readProcessedJobs(String.format(FORMAT_JOB_PROCESS_DATE, cal.get(Calendar.YEAR),
                 cal.get(Calendar.MONTH) + 1, cal.get(Calendar.DAY_OF_MONTH)));
         if (list != null) {
             this.processedJobFileNames = new HashSet<>(list);
@@ -241,7 +238,7 @@ public class JHFCrawlerDriverImpl implements JHFCrawlerDriver {
     }
 
     private void flushJobCount() throws Exception {
-        List<Pair<String, String>> jobs = zkStateLcm.getProcessedJobs(
+        List<Pair<String, String>> jobs = JobHistoryZKStateManager.instance().getProcessedJobs(
             String.format(FORMAT_JOB_PROCESS_DATE, this.processDate.year, this.processDate.month + 1, this.processDate.day)
         );
         JobCountEntity entity = new JobCountEntity();
@@ -300,7 +297,7 @@ public class JHFCrawlerDriverImpl implements JHFCrawlerDriver {
         cal.add(Calendar.DATE, -1 - PROCESSED_JOB_KEEP_DAYS);
         String line = String.format(FORMAT_JOB_PROCESS_DATE, cal.get(Calendar.YEAR),
                 cal.get(Calendar.MONTH) + 1, cal.get(Calendar.DAY_OF_MONTH));
-        zkStateLcm.truncateProcessedJob(line);
+        JobHistoryZKStateManager.instance().truncateProcessedJob(line);
     }
 
     private boolean isToday() {

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/c940f56c/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JobEntityCreationEagleServiceListener.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JobEntityCreationEagleServiceListener.java b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JobEntityCreationEagleServiceListener.java
index 520fbbc..30eeb54 100644
--- a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JobEntityCreationEagleServiceListener.java
+++ b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JobEntityCreationEagleServiceListener.java
@@ -90,13 +90,12 @@ public class JobEntityCreationEagleServiceListener implements HistoryJobEntityCr
             eagleServiceConfig.password);
 
         client.getJerseyClient().setReadTimeout(jobExtractorConfig.readTimeoutSeconds * 1000);
-        JobHistoryZKStateManager zkState = new JobHistoryZKStateManager(configManager.getZkStateConfig());
         logger.info("start flushing entities of total number " + list.size());
         for (int i = 0; i < list.size(); i++) {
             JobBaseAPIEntity entity = list.get(i);
             if (entity instanceof JobExecutionAPIEntity) {
                 jobs.add((JobExecutionAPIEntity) entity);
-                zkState.updateProcessedJob(timeStamp2Date(entity.getTimestamp()),
+                JobHistoryZKStateManager.instance().updateProcessedJob(timeStamp2Date(entity.getTimestamp()),
                     entity.getTags().get(MRJobTagName.JOB_ID.toString()),
                     ((JobExecutionAPIEntity) entity).getCurrentState());
             } else if (entity instanceof JobEventAPIEntity) {
@@ -107,7 +106,6 @@ public class JobEntityCreationEagleServiceListener implements HistoryJobEntityCr
                 taskAttemptExecs.add((TaskAttemptExecutionAPIEntity) entity);
             }
         }
-        zkState.close();
         GenericServiceAPIResponseEntity result;
         if (jobs.size() > 0) {
             logger.info("flush JobExecutionAPIEntity of number " + jobs.size());

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/c940f56c/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/storm/JobHistorySpout.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/storm/JobHistorySpout.java b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/storm/JobHistorySpout.java
index 402f93e..04283d3 100644
--- a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/storm/JobHistorySpout.java
+++ b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/storm/JobHistorySpout.java
@@ -87,7 +87,6 @@ public class JobHistorySpout extends BaseRichSpout {
 
     private int partitionId;
     private int numTotalPartitions;
-    private transient JobHistoryZKStateManager zkState;
     private transient JHFCrawlerDriver driver;
     private JobHistoryContentFilter contentFilter;
     private JobHistorySpoutCollectorInterceptor interceptor;
@@ -143,8 +142,8 @@ public class JobHistorySpout extends BaseRichSpout {
             throw new IllegalStateException(e);
         }
         JobIdFilter jobIdFilter = new JobIdFilterByPartition(partitioner, numTotalPartitions, partitionId);
-        zkState = new JobHistoryZKStateManager(configManager.getZkStateConfig());
-        zkState.ensureJobPartitions(numTotalPartitions);
+        JobHistoryZKStateManager.instance().init(configManager.getZkStateConfig());
+        JobHistoryZKStateManager.instance().ensureJobPartitions(numTotalPartitions);
         interceptor.setSpoutOutputCollector(collector);
 
         try {
@@ -154,7 +153,6 @@ public class JobHistorySpout extends BaseRichSpout {
                 configManager.getJobExtractorConfig(),
                 configManager.getControlConfig(),
                 callback,
-                zkState,
                 jhfLCM,
                 jobIdFilter,
                 partitionId);
@@ -168,7 +166,7 @@ public class JobHistorySpout extends BaseRichSpout {
     public void nextTuple() {
         try {
             Long modifiedTime = driver.crawl();
-            zkState.updateProcessedTimeStamp(partitionId, modifiedTime);
+            JobHistoryZKStateManager.instance().updateProcessedTimeStamp(partitionId, modifiedTime);
             updateProcessedTimeStamp(modifiedTime);
         } catch (Exception ex) {
             LOG.error("fail crawling job history file and continue ...", ex);
@@ -223,7 +221,7 @@ public class JobHistorySpout extends BaseRichSpout {
         //update latest process time
         long minTimeStamp = modifiedTime;
         for (int i = 1; i < numTotalPartitions; i++) {
-            long time = zkState.readProcessedTimeStamp(i);
+            long time = JobHistoryZKStateManager.instance().readProcessedTimeStamp(i);
             if (time <= minTimeStamp) {
                 minTimeStamp = time;
             }

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/c940f56c/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/zkres/JobHistoryZKStateManager.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/zkres/JobHistoryZKStateManager.java b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/zkres/JobHistoryZKStateManager.java
index c61d05a..2e64da3 100644
--- a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/zkres/JobHistoryZKStateManager.java
+++ b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/zkres/JobHistoryZKStateManager.java
@@ -47,6 +47,8 @@ public class JobHistoryZKStateManager implements JobHistoryZKStateLCM {
 
     public static final int BACKOFF_DAYS = 0;
 
+    private static JobHistoryZKStateManager jobHistoryZKStateManager = new JobHistoryZKStateManager();
+
     private CuratorFramework newCurator(ZKStateConfig config) throws Exception {
         return CuratorFrameworkFactory.newClient(
             config.zkQuorum,
@@ -56,7 +58,11 @@ public class JobHistoryZKStateManager implements JobHistoryZKStateLCM {
         );
     }
 
-    public JobHistoryZKStateManager(ZKStateConfig config) {
+    public static JobHistoryZKStateManager instance() {
+        return jobHistoryZKStateManager;
+    }
+
+    public void init(ZKStateConfig config) {
         this.zkRoot = config.zkRoot;
 
         try {


[33/52] [abbrv] incubator-eagle git commit: [EAGLE-518] add Job counter metrics for mr history job

Posted by yo...@apache.org.
[EAGLE-518] add Job counter metrics for mr history job

Author: wujinhu <wu...@126.com>

Closes #414 from wujinhu/EAGLE-518.


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

Branch: refs/heads/master
Commit: b2b16b745e210eb80929d970820c322d557d93e0
Parents: 9488afc
Author: wujinhu <wu...@126.com>
Authored: Fri Sep 2 11:48:54 2016 +0800
Committer: Qingwen Zhao <qi...@gmail.com>
Committed: Fri Sep 2 11:48:54 2016 +0800

----------------------------------------------------------------------
 .../history/crawler/JHFCrawlerDriverImpl.java   | 53 +++---------
 .../metrics/JobCountMetricsGenerator.java       | 88 ++++++++++++++++++++
 .../JobExecutionMetricsCreationListener.java    | 75 +++++++++++++++++
 .../JobEntityCreationEagleServiceListener.java  | 13 +++
 .../AbstractMetricsCreationListener.java        | 42 ----------
 .../JobExecutionMetricsCreationListener.java    |  9 +-
 .../TaskExecutionMetricsCreationListener.java   |  3 +-
 .../org/apache/eagle/jpm/util/Constants.java    |  4 +
 .../AbstractMetricsCreationListener.java        | 42 ++++++++++
 9 files changed, 239 insertions(+), 90 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/b2b16b74/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/crawler/JHFCrawlerDriverImpl.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/crawler/JHFCrawlerDriverImpl.java b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/crawler/JHFCrawlerDriverImpl.java
index 077f4e1..2f326fe 100644
--- a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/crawler/JHFCrawlerDriverImpl.java
+++ b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/crawler/JHFCrawlerDriverImpl.java
@@ -19,6 +19,7 @@
 package org.apache.eagle.jpm.mr.history.crawler;
 
 import org.apache.eagle.jpm.mr.history.MRHistoryJobConfig;
+import org.apache.eagle.jpm.mr.history.metrics.JobCountMetricsGenerator;
 import org.apache.eagle.jpm.mr.history.parser.EagleJobStatus;
 import org.apache.eagle.jpm.mr.history.zkres.JobHistoryZKStateManager;
 import org.apache.eagle.jpm.mr.historyentity.JobCountEntity;
@@ -62,15 +63,12 @@ public class JHFCrawlerDriverImpl implements JHFCrawlerDriver {
     private JobIdFilter jobFilter;
     private int partitionId;
     private TimeZone timeZone;
-    private MRHistoryJobConfig.EagleServiceConfig eagleServiceConfig;
-    private MRHistoryJobConfig.JobExtractorConfig jobExtractorConfig;
+    private JobCountMetricsGenerator jobCountMetricsGenerator;
 
     public JHFCrawlerDriverImpl(MRHistoryJobConfig.EagleServiceConfig eagleServiceConfig,
                                 MRHistoryJobConfig.JobExtractorConfig jobExtractorConfig,
                                 MRHistoryJobConfig.ControlConfig controlConfig, JHFInputStreamCallback reader,
                                 JobHistoryLCM historyLCM, JobIdFilter jobFilter, int partitionId) throws Exception {
-        this.eagleServiceConfig = eagleServiceConfig;
-        this.jobExtractorConfig = jobExtractorConfig;
         this.zeroBasedMonth = controlConfig.zeroBasedMonth;
         this.dryRun = controlConfig.dryRun;
         if (this.dryRun)  {
@@ -81,6 +79,7 @@ public class JHFCrawlerDriverImpl implements JHFCrawlerDriver {
         this.partitionId = partitionId;
         this.jobFilter = jobFilter;
         timeZone = TimeZone.getTimeZone(controlConfig.timeZone);
+        jobCountMetricsGenerator = new JobCountMetricsGenerator(eagleServiceConfig, jobExtractorConfig, timeZone);
     }
 
     /**
@@ -191,7 +190,13 @@ public class JHFCrawlerDriverImpl implements JHFCrawlerDriver {
                 jobHistoryFile);
         processedJobFileNames.add(jobHistoryFile);
 
-        flushJobCount();
+        jobCountMetricsGenerator.flush(
+            String.format(FORMAT_JOB_PROCESS_DATE,
+                this.processDate.year,
+                this.processDate.month + 1,
+                this.processDate.day),
+            this.processDate.year, this.processDate.month, this.processDate.day
+        );
         Long modifiedTime = item.getLeft();
         return modifiedTime;
     }
@@ -237,44 +242,6 @@ public class JHFCrawlerDriverImpl implements JHFCrawlerDriver {
         }
     }
 
-    private void flushJobCount() throws Exception {
-        List<Pair<String, String>> jobs = JobHistoryZKStateManager.instance().getProcessedJobs(
-            String.format(FORMAT_JOB_PROCESS_DATE, this.processDate.year, this.processDate.month + 1, this.processDate.day)
-        );
-        JobCountEntity entity = new JobCountEntity();
-        entity.setTotal(jobs.size());
-        entity.setFail(0);
-        jobs.stream().filter(job -> !job.getRight().equals(EagleJobStatus.SUCCEEDED.toString())).forEach(
-            job -> entity.setFail(1 + entity.getFail())
-        );
-
-        IEagleServiceClient client = new EagleServiceClientImpl(
-            eagleServiceConfig.eagleServiceHost,
-            eagleServiceConfig.eagleServicePort,
-            eagleServiceConfig.username,
-            eagleServiceConfig.password);
-
-
-        GregorianCalendar cal = new GregorianCalendar(this.processDate.year, this.processDate.month, this.processDate.day, 0, 0, 0);
-        cal.setTimeZone(timeZone);
-        entity.setTimestamp(cal.getTimeInMillis());
-        @SuppressWarnings("serial")
-        Map<String, String> baseTags = new HashMap<String, String>() {
-            {
-                put("site", jobExtractorConfig.site);
-            }
-        };
-        entity.setTags(baseTags);
-        List<JobCountEntity> entities = new ArrayList<>();
-        entities.add(entity);
-
-        LOG.info("start flushing entities of total number " + entities.size());
-        client.create(entities);
-        LOG.info("finish flushing entities of total number " + entities.size());
-        client.getJerseyClient().destroy();
-        client.close();
-    }
-
     private void advanceOneDay() throws Exception {
         //flushJobCount();
         GregorianCalendar cal = new GregorianCalendar(timeZone);

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/b2b16b74/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/metrics/JobCountMetricsGenerator.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/metrics/JobCountMetricsGenerator.java b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/metrics/JobCountMetricsGenerator.java
new file mode 100644
index 0000000..0e0e5e9
--- /dev/null
+++ b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/metrics/JobCountMetricsGenerator.java
@@ -0,0 +1,88 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ *
+ */
+
+package org.apache.eagle.jpm.mr.history.metrics;
+
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.eagle.jpm.mr.history.MRHistoryJobConfig;
+import org.apache.eagle.jpm.mr.history.parser.EagleJobStatus;
+import org.apache.eagle.jpm.mr.history.zkres.JobHistoryZKStateManager;
+import org.apache.eagle.jpm.util.Constants;
+import org.apache.eagle.log.entity.GenericMetricEntity;
+import org.apache.eagle.service.client.IEagleServiceClient;
+import org.apache.eagle.service.client.impl.EagleServiceClientImpl;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.*;
+
+public class JobCountMetricsGenerator {
+    private static final Logger LOG = LoggerFactory.getLogger(JobCountMetricsGenerator.class);
+
+    private MRHistoryJobConfig.EagleServiceConfig eagleServiceConfig;
+    private MRHistoryJobConfig.JobExtractorConfig jobExtractorConfig;
+    private TimeZone timeZone;
+
+    public JobCountMetricsGenerator(MRHistoryJobConfig.EagleServiceConfig eagleServiceConfig,
+                                    MRHistoryJobConfig.JobExtractorConfig jobExtractorConfig,
+                                    TimeZone timeZone) {
+        this.eagleServiceConfig = eagleServiceConfig;
+        this.jobExtractorConfig = jobExtractorConfig;
+        this.timeZone = timeZone;
+    }
+
+    public void flush(String date, int year, int month, int day) throws Exception {
+        List<Pair<String, String>> jobs = JobHistoryZKStateManager.instance().getProcessedJobs(date);
+        int total = jobs.size();
+        int fail = 0;
+        for (Pair<String, String> job : jobs) {
+            if (!job.getRight().equals(EagleJobStatus.SUCCEEDED.toString())) {
+                ++fail;
+            }
+        }
+
+        IEagleServiceClient client = new EagleServiceClientImpl(
+            eagleServiceConfig.eagleServiceHost,
+            eagleServiceConfig.eagleServicePort,
+            eagleServiceConfig.username,
+            eagleServiceConfig.password);
+
+
+        GregorianCalendar cal = new GregorianCalendar(year, month, day);
+        cal.setTimeZone(timeZone);
+        GenericMetricEntity metricEntity = new GenericMetricEntity();
+        metricEntity.setTimestamp(cal.getTimeInMillis());
+        metricEntity.setPrefix(Constants.JOB_COUNT_PER_DAY);
+        metricEntity.setValue(new double[]{total, fail});
+        @SuppressWarnings("serial")
+        Map<String, String> baseTags = new HashMap<String, String>() {
+            {
+                put("site", jobExtractorConfig.site);
+            }
+        };
+        metricEntity.setTags(baseTags);
+        List<GenericMetricEntity> entities = new ArrayList<>();
+        entities.add(metricEntity);
+
+        LOG.info("start flushing entities of total number " + entities.size());
+        client.create(entities);
+        LOG.info("finish flushing entities of total number " + entities.size());
+        client.getJerseyClient().destroy();
+        client.close();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/b2b16b74/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/metrics/JobExecutionMetricsCreationListener.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/metrics/JobExecutionMetricsCreationListener.java b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/metrics/JobExecutionMetricsCreationListener.java
new file mode 100644
index 0000000..2129bed
--- /dev/null
+++ b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/metrics/JobExecutionMetricsCreationListener.java
@@ -0,0 +1,75 @@
+/*
+ * 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.eagle.jpm.mr.history.metrics;
+
+import org.apache.eagle.jpm.mr.historyentity.JobExecutionAPIEntity;
+import org.apache.eagle.jpm.util.Constants;
+import org.apache.eagle.log.entity.GenericMetricEntity;
+import org.apache.eagle.jpm.util.metrics.AbstractMetricsCreationListener;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+public class JobExecutionMetricsCreationListener extends AbstractMetricsCreationListener<JobExecutionAPIEntity> {
+
+    @Override
+    public List<GenericMetricEntity> generateMetrics(JobExecutionAPIEntity entity) {
+        List<GenericMetricEntity> metrics = new ArrayList<>();
+        if (entity != null) {
+            Long timeStamp = entity.getTimestamp();
+            Map<String, String> tags = entity.getTags();
+            metrics.add(metricWrapper(timeStamp,
+                Constants.JOB_EXECUTION_TIME,
+                new double[]{entity.getDurationTime()},
+                tags));
+
+            metrics.add(metricWrapper(
+                timeStamp,
+                Constants.MAP_COUNT_RATIO,
+                new double[]{entity.getNumTotalMaps(), 1.0 * entity.getNumFailedMaps() / entity.getNumTotalMaps()},
+                tags));
+
+            metrics.add(metricWrapper(
+                timeStamp,
+                Constants.REDUCE_COUNT_RATIO,
+                new double[]{entity.getNumTotalReduces(), 1.0 * entity.getNumFailedReduces() / entity.getNumTotalReduces()},
+                tags));
+
+            org.apache.eagle.jpm.util.jobcounter.JobCounters jobCounters = entity.getJobCounters();
+            if (jobCounters != null && jobCounters.getCounters() != null) {
+                for (Map<String, Long> metricGroup : jobCounters.getCounters().values()) {
+                    for (Map.Entry<String, Long> entry : metricGroup.entrySet()) {
+                        String metricName = entry.getKey().toLowerCase();
+                        metrics.add(metricWrapper(timeStamp, "history." + metricName, new double[]{entry.getValue()}, tags));
+                    }
+                }
+            }
+        }
+        return metrics;
+    }
+
+    @Override
+    public String buildMetricName(String field) {
+        return String.format(Constants.hadoopMetricFormat, Constants.JOB_LEVEL, field);
+    }
+
+
+}
+

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/b2b16b74/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JobEntityCreationEagleServiceListener.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JobEntityCreationEagleServiceListener.java b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JobEntityCreationEagleServiceListener.java
index 30eeb54..623a776 100644
--- a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JobEntityCreationEagleServiceListener.java
+++ b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JobEntityCreationEagleServiceListener.java
@@ -19,9 +19,11 @@
 package org.apache.eagle.jpm.mr.history.parser;
 
 import org.apache.eagle.jpm.mr.history.MRHistoryJobConfig;
+import org.apache.eagle.jpm.mr.history.metrics.JobExecutionMetricsCreationListener;
 import org.apache.eagle.jpm.mr.history.zkres.JobHistoryZKStateManager;
 import org.apache.eagle.jpm.mr.historyentity.*;
 import org.apache.eagle.jpm.util.MRJobTagName;
+import org.apache.eagle.log.entity.GenericMetricEntity;
 import org.apache.eagle.log.entity.GenericServiceAPIResponseEntity;
 import org.apache.eagle.service.client.IEagleServiceClient;
 import org.apache.eagle.service.client.impl.EagleServiceClientImpl;
@@ -43,6 +45,7 @@ public class JobEntityCreationEagleServiceListener implements HistoryJobEntityCr
     List<JobEventAPIEntity> jobEvents = new ArrayList<>();
     List<TaskExecutionAPIEntity> taskExecs = new ArrayList<>();
     List<TaskAttemptExecutionAPIEntity> taskAttemptExecs = new ArrayList<>();
+    private JobExecutionMetricsCreationListener jobExecutionMetricsCreationListener = new JobExecutionMetricsCreationListener();
     private TimeZone timeZone;
 
     public JobEntityCreationEagleServiceListener(MRHistoryJobConfig configManager) {
@@ -91,6 +94,7 @@ public class JobEntityCreationEagleServiceListener implements HistoryJobEntityCr
 
         client.getJerseyClient().setReadTimeout(jobExtractorConfig.readTimeoutSeconds * 1000);
         logger.info("start flushing entities of total number " + list.size());
+        List<GenericMetricEntity> metricEntities = new ArrayList<>();
         for (int i = 0; i < list.size(); i++) {
             JobBaseAPIEntity entity = list.get(i);
             if (entity instanceof JobExecutionAPIEntity) {
@@ -98,6 +102,8 @@ public class JobEntityCreationEagleServiceListener implements HistoryJobEntityCr
                 JobHistoryZKStateManager.instance().updateProcessedJob(timeStamp2Date(entity.getTimestamp()),
                     entity.getTags().get(MRJobTagName.JOB_ID.toString()),
                     ((JobExecutionAPIEntity) entity).getCurrentState());
+
+                metricEntities.addAll(jobExecutionMetricsCreationListener.generateMetrics((JobExecutionAPIEntity)entity));
             } else if (entity instanceof JobEventAPIEntity) {
                 jobEvents.add((JobEventAPIEntity) entity);
             } else if (entity instanceof TaskExecutionAPIEntity) {
@@ -113,6 +119,12 @@ public class JobEntityCreationEagleServiceListener implements HistoryJobEntityCr
             checkResult(result);
             jobs.clear();
         }
+        if (metricEntities.size() > 0) {
+            logger.info("flush job metrics of number " + metricEntities.size());
+            result = client.create(metricEntities);
+            checkResult(result);
+            metricEntities.clear();
+        }
         if (jobEvents.size() > 0) {
             logger.info("flush JobEventAPIEntity of number " + jobEvents.size());
             result = client.create(jobEvents);
@@ -131,6 +143,7 @@ public class JobEntityCreationEagleServiceListener implements HistoryJobEntityCr
             checkResult(result);
             taskAttemptExecs.clear();
         }
+
         logger.info("finish flushing entities of total number " + list.size());
         list.clear();
         client.getJerseyClient().destroy();

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/b2b16b74/eagle-jpm/eagle-jpm-mr-running/src/main/java/org/apache/eagle/jpm/mr/running/parser/metrics/AbstractMetricsCreationListener.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-mr-running/src/main/java/org/apache/eagle/jpm/mr/running/parser/metrics/AbstractMetricsCreationListener.java b/eagle-jpm/eagle-jpm-mr-running/src/main/java/org/apache/eagle/jpm/mr/running/parser/metrics/AbstractMetricsCreationListener.java
deleted file mode 100644
index 8634b6a..0000000
--- a/eagle-jpm/eagle-jpm-mr-running/src/main/java/org/apache/eagle/jpm/mr/running/parser/metrics/AbstractMetricsCreationListener.java
+++ /dev/null
@@ -1,42 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- *  Unless required by applicable law or agreed to in writing, software
- *  distributed under the License is distributed on an "AS IS" BASIS,
- *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- *  See the License for the specific language governing permissions and
- *  limitations under the License.
- *
- */
-
-package org.apache.eagle.jpm.mr.running.parser.metrics;
-
-import org.apache.eagle.log.base.taggedlog.TaggedLogAPIEntity;
-import org.apache.eagle.log.entity.GenericMetricEntity;
-
-import java.util.List;
-import java.util.Map;
-
-public abstract class AbstractMetricsCreationListener<E extends TaggedLogAPIEntity> {
-
-    public abstract List<GenericMetricEntity> generateMetrics(E entity);
-
-    protected abstract String buildMetricName(String field);
-
-    protected GenericMetricEntity metricWrapper(Long timestamp, String field, double value, Map<String, String> tags) {
-        String metricName = buildMetricName(field);
-        GenericMetricEntity metricEntity = new GenericMetricEntity();
-        metricEntity.setTimestamp(timestamp);
-        metricEntity.setTags(tags);
-        metricEntity.setPrefix(metricName);
-        metricEntity.setValue(new double[]{value});
-        return metricEntity;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/b2b16b74/eagle-jpm/eagle-jpm-mr-running/src/main/java/org/apache/eagle/jpm/mr/running/parser/metrics/JobExecutionMetricsCreationListener.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-mr-running/src/main/java/org/apache/eagle/jpm/mr/running/parser/metrics/JobExecutionMetricsCreationListener.java b/eagle-jpm/eagle-jpm-mr-running/src/main/java/org/apache/eagle/jpm/mr/running/parser/metrics/JobExecutionMetricsCreationListener.java
index 76d2a19..8b30d45 100644
--- a/eagle-jpm/eagle-jpm-mr-running/src/main/java/org/apache/eagle/jpm/mr/running/parser/metrics/JobExecutionMetricsCreationListener.java
+++ b/eagle-jpm/eagle-jpm-mr-running/src/main/java/org/apache/eagle/jpm/mr/running/parser/metrics/JobExecutionMetricsCreationListener.java
@@ -21,6 +21,7 @@ package org.apache.eagle.jpm.mr.running.parser.metrics;
 import org.apache.eagle.jpm.mr.runningentity.JobExecutionAPIEntity;
 import org.apache.eagle.jpm.util.Constants;
 import org.apache.eagle.log.entity.GenericMetricEntity;
+import org.apache.eagle.jpm.util.metrics.AbstractMetricsCreationListener;
 
 import java.util.ArrayList;
 import java.util.List;
@@ -34,15 +35,15 @@ public class JobExecutionMetricsCreationListener extends AbstractMetricsCreation
         if (entity != null) {
             Long currentTime = System.currentTimeMillis();
             Map<String, String> tags = entity.getTags();
-            metrics.add(metricWrapper(currentTime, Constants.ALLOCATED_MB, entity.getAllocatedMB(), tags));
-            metrics.add(metricWrapper(currentTime, Constants.ALLOCATED_VCORES, entity.getAllocatedVCores(), tags));
-            metrics.add(metricWrapper(currentTime, Constants.RUNNING_CONTAINERS, entity.getRunningContainers(), tags));
+            metrics.add(metricWrapper(currentTime, Constants.ALLOCATED_MB, new double[]{entity.getAllocatedMB()}, tags));
+            metrics.add(metricWrapper(currentTime, Constants.ALLOCATED_VCORES, new double[]{entity.getAllocatedVCores()}, tags));
+            metrics.add(metricWrapper(currentTime, Constants.RUNNING_CONTAINERS, new double[]{entity.getRunningContainers()}, tags));
             org.apache.eagle.jpm.util.jobcounter.JobCounters jobCounters = entity.getJobCounters();
             if (jobCounters != null && jobCounters.getCounters() != null) {
                 for (Map<String, Long> metricGroup : jobCounters.getCounters().values()) {
                     for (Map.Entry<String, Long> entry : metricGroup.entrySet()) {
                         String metricName = entry.getKey().toLowerCase();
-                        metrics.add(metricWrapper(currentTime, metricName, entry.getValue(), tags));
+                        metrics.add(metricWrapper(currentTime, metricName, new double[]{entry.getValue()}, tags));
                     }
                 }
             }

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/b2b16b74/eagle-jpm/eagle-jpm-mr-running/src/main/java/org/apache/eagle/jpm/mr/running/parser/metrics/TaskExecutionMetricsCreationListener.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-mr-running/src/main/java/org/apache/eagle/jpm/mr/running/parser/metrics/TaskExecutionMetricsCreationListener.java b/eagle-jpm/eagle-jpm-mr-running/src/main/java/org/apache/eagle/jpm/mr/running/parser/metrics/TaskExecutionMetricsCreationListener.java
index d0b0d57..9f22a7f 100644
--- a/eagle-jpm/eagle-jpm-mr-running/src/main/java/org/apache/eagle/jpm/mr/running/parser/metrics/TaskExecutionMetricsCreationListener.java
+++ b/eagle-jpm/eagle-jpm-mr-running/src/main/java/org/apache/eagle/jpm/mr/running/parser/metrics/TaskExecutionMetricsCreationListener.java
@@ -21,6 +21,7 @@ package org.apache.eagle.jpm.mr.running.parser.metrics;
 import org.apache.eagle.jpm.mr.runningentity.TaskExecutionAPIEntity;
 import org.apache.eagle.jpm.util.Constants;
 import org.apache.eagle.log.entity.GenericMetricEntity;
+import org.apache.eagle.jpm.util.metrics.AbstractMetricsCreationListener;
 
 import java.util.ArrayList;
 import java.util.List;
@@ -33,7 +34,7 @@ public class TaskExecutionMetricsCreationListener extends AbstractMetricsCreatio
         if (entity != null) {
             Long currentTime = System.currentTimeMillis();
             Map<String, String> tags = entity.getTags();
-            metrics.add(metricWrapper(currentTime, Constants.TASK_EXECUTION_TIME, entity.getDuration(), tags));
+            metrics.add(metricWrapper(currentTime, Constants.TASK_EXECUTION_TIME, new double[]{entity.getDuration()}, tags));
         }
         return metrics;
     }

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/b2b16b74/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/Constants.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/Constants.java b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/Constants.java
index ec56eac..e18fe07 100644
--- a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/Constants.java
+++ b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/Constants.java
@@ -172,7 +172,11 @@ public class Constants {
     public static final String ALLOCATED_VCORES = "allocatedvcores";
     public static final String RUNNING_CONTAINERS = "runningcontainers";
     public static final String TASK_EXECUTION_TIME = "taskduration";
+    public static final String JOB_EXECUTION_TIME = "jobduration";
+    public static final String MAP_COUNT_RATIO = "map.count.ratio";
+    public static final String REDUCE_COUNT_RATIO = "reduce.count.ratio";
     public static final String JOB_LEVEL = "job";
     public static final String TASK_LEVEL = "task";
+    public static final String JOB_COUNT_PER_DAY = "hadoop.job.day.count";
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/b2b16b74/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/metrics/AbstractMetricsCreationListener.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/metrics/AbstractMetricsCreationListener.java b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/metrics/AbstractMetricsCreationListener.java
new file mode 100644
index 0000000..dd61432
--- /dev/null
+++ b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/metrics/AbstractMetricsCreationListener.java
@@ -0,0 +1,42 @@
+/*
+ * 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.eagle.jpm.util.metrics;
+
+import org.apache.eagle.log.base.taggedlog.TaggedLogAPIEntity;
+import org.apache.eagle.log.entity.GenericMetricEntity;
+
+import java.util.List;
+import java.util.Map;
+
+public abstract class AbstractMetricsCreationListener<E extends TaggedLogAPIEntity> {
+
+    public abstract List<GenericMetricEntity> generateMetrics(E entity);
+
+    protected abstract String buildMetricName(String field);
+
+    protected GenericMetricEntity metricWrapper(Long timestamp, String field, double[] values, Map<String, String> tags) {
+        String metricName = buildMetricName(field);
+        GenericMetricEntity metricEntity = new GenericMetricEntity();
+        metricEntity.setTimestamp(timestamp);
+        metricEntity.setTags(tags);
+        metricEntity.setPrefix(metricName);
+        metricEntity.setValue(values);
+        return metricEntity;
+    }
+}


[42/52] [abbrv] incubator-eagle git commit: [Enhancement] eagle alert engine policy handler extensible for customized handler

Posted by yo...@apache.org.
[Enhancement] eagle alert engine policy handler extensible for customized handler

Author : ralphsu


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

Branch: refs/heads/master
Commit: f81ae06217235a04745997f70ddcd1c4d92707e9
Parents: fce6ae3
Author: Ralph, Su <su...@gmail.com>
Authored: Wed Sep 7 13:50:32 2016 +0800
Committer: Ralph, Su <su...@gmail.com>
Committed: Wed Sep 7 13:50:32 2016 +0800

----------------------------------------------------------------------
 .../org/apache/eagle/alert/utils/JsonUtils.java |  2 +-
 .../engine/evaluator/PolicyHandlerContext.java  | 13 ++++-
 .../impl/PolicyGroupEvaluatorImpl.java          | 13 ++---
 .../engine/evaluator/PoilcyExtendedTest.java    | 51 ++++++++++++++++++++
 .../src/test/resources/extend_policy.json       | 22 +++++++++
 5 files changed, 92 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/f81ae062/eagle-core/eagle-alert-parent/eagle-alert/alert-common/src/main/java/org/apache/eagle/alert/utils/JsonUtils.java
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-alert-parent/eagle-alert/alert-common/src/main/java/org/apache/eagle/alert/utils/JsonUtils.java b/eagle-core/eagle-alert-parent/eagle-alert/alert-common/src/main/java/org/apache/eagle/alert/utils/JsonUtils.java
index de0c48b..09ee5f4 100644
--- a/eagle-core/eagle-alert-parent/eagle-alert/alert-common/src/main/java/org/apache/eagle/alert/utils/JsonUtils.java
+++ b/eagle-core/eagle-alert-parent/eagle-alert/alert-common/src/main/java/org/apache/eagle/alert/utils/JsonUtils.java
@@ -16,10 +16,10 @@
  */
 package org.apache.eagle.alert.utils;
 
+import org.codehaus.jackson.map.ObjectMapper;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import com.fasterxml.jackson.databind.ObjectMapper;
 
 /**
  * @since May 1, 2016

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/f81ae062/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/main/java/org/apache/eagle/alert/engine/evaluator/PolicyHandlerContext.java
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/main/java/org/apache/eagle/alert/engine/evaluator/PolicyHandlerContext.java b/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/main/java/org/apache/eagle/alert/engine/evaluator/PolicyHandlerContext.java
index 285ca13..335b237 100644
--- a/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/main/java/org/apache/eagle/alert/engine/evaluator/PolicyHandlerContext.java
+++ b/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/main/java/org/apache/eagle/alert/engine/evaluator/PolicyHandlerContext.java
@@ -1,8 +1,8 @@
 package org.apache.eagle.alert.engine.evaluator;
 
-import org.apache.eagle.alert.engine.coordinator.PolicyDefinition;
-
 import backtype.storm.metric.api.MultiCountMetric;
+import com.typesafe.config.Config;
+import org.apache.eagle.alert.engine.coordinator.PolicyDefinition;
 
 /**
  * Licensed to the Apache Software Foundation (ASF) under one or more
@@ -25,6 +25,7 @@ public class PolicyHandlerContext {
     private PolicyGroupEvaluator policyEvaluator;
     private MultiCountMetric policyCounter;
     private String policyEvaluatorId;
+    private Config config;
 
     public PolicyDefinition getPolicyDefinition() {
         return policyDefinition;
@@ -57,4 +58,12 @@ public class PolicyHandlerContext {
     public void setPolicyEvaluatorId(String policyEvaluatorId) {
         this.policyEvaluatorId = policyEvaluatorId;
     }
+
+    public Config getConfig() {
+        return config;
+    }
+
+    public void setConfig(Config config) {
+        this.config = config;
+    }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/f81ae062/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/main/java/org/apache/eagle/alert/engine/evaluator/impl/PolicyGroupEvaluatorImpl.java
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/main/java/org/apache/eagle/alert/engine/evaluator/impl/PolicyGroupEvaluatorImpl.java b/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/main/java/org/apache/eagle/alert/engine/evaluator/impl/PolicyGroupEvaluatorImpl.java
index 068df90..26ae19e 100644
--- a/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/main/java/org/apache/eagle/alert/engine/evaluator/impl/PolicyGroupEvaluatorImpl.java
+++ b/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/main/java/org/apache/eagle/alert/engine/evaluator/impl/PolicyGroupEvaluatorImpl.java
@@ -138,12 +138,13 @@ public class PolicyGroupEvaluatorImpl implements PolicyGroupEvaluator {
             policies.put(policy.getName(), policy);
             CompositePolicyHandler handler = new CompositePolicyHandler(sds);
             try {
-                PolicyHandlerContext context = new PolicyHandlerContext();
-                context.setPolicyCounter(this.context.counter());
-                context.setPolicyDefinition(policy);
-                context.setPolicyEvaluator(this);
-                context.setPolicyEvaluatorId(policyEvaluatorId);
-                handler.prepare(collector, context);
+                PolicyHandlerContext handlerContext = new PolicyHandlerContext();
+                handlerContext.setPolicyCounter(this.context.counter());
+                handlerContext.setPolicyDefinition(policy);
+                handlerContext.setPolicyEvaluator(this);
+                handlerContext.setPolicyEvaluatorId(policyEvaluatorId);
+                handlerContext.setConfig(this.context.config());
+                handler.prepare(collector, handlerContext);
                 handlers.put(policy.getName(), handler);
             } catch (Exception e) {
                 LOG.error(e.getMessage(), e);

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/f81ae062/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/test/java/org/apache/eagle/alert/engine/evaluator/PoilcyExtendedTest.java
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/test/java/org/apache/eagle/alert/engine/evaluator/PoilcyExtendedTest.java b/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/test/java/org/apache/eagle/alert/engine/evaluator/PoilcyExtendedTest.java
new file mode 100644
index 0000000..dd75efb
--- /dev/null
+++ b/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/test/java/org/apache/eagle/alert/engine/evaluator/PoilcyExtendedTest.java
@@ -0,0 +1,51 @@
+package org.apache.eagle.alert.engine.evaluator;
+
+import org.apache.eagle.alert.engine.coordinator.PolicyDefinition;
+import org.codehaus.jackson.JsonNode;
+import org.codehaus.jackson.map.ObjectMapper;
+import org.codehaus.jackson.node.ArrayNode;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.util.List;
+import java.util.Map;
+
+/**
+ * Created on 9/7/16.
+ */
+public class PoilcyExtendedTest {
+
+    private static final ObjectMapper mapper = new ObjectMapper();
+
+    @Test
+    public void test() throws Exception {
+        ArrayNode arrayNode = (ArrayNode)
+                mapper.readTree(PoilcyExtendedTest.class.getResourceAsStream("/extend_policy.json"));
+        Assert.assertEquals(1, arrayNode.size());
+        for (JsonNode node : arrayNode) {
+            PolicyDefinition definition = mapper.readValue(node, PolicyDefinition.class);
+
+            Assert.assertNotNull(definition);
+            Assert.assertNotNull(definition.getName());
+            Assert.assertNotNull(definition.getDefinition());
+
+            Assert.assertEquals(PolicyStreamHandlers.CUSTOMIZED_ENGINE, definition.getDefinition().getType());
+            Assert.assertNotNull(definition.getDefinition().getProperties());
+
+            Assert.assertTrue(definition.getDefinition().getProperties().containsKey("parentKey"));
+            Map pkSetting = (Map) definition.getDefinition().getProperties().get("parentKey");
+            Assert.assertTrue(pkSetting.containsKey("syslogStream"));
+
+            Map syslogStreamSetting = (Map) pkSetting.get("syslogStream");
+            Assert.assertTrue(syslogStreamSetting.containsKey("pattern"));
+            Assert.assertEquals("%s-%s", syslogStreamSetting.get("pattern"));
+
+            Assert.assertTrue(syslogStreamSetting.containsKey("columns"));
+            Assert.assertEquals(3, ((List) syslogStreamSetting.get("columns")).size());
+
+            break;
+        }
+
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/f81ae062/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/test/resources/extend_policy.json
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/test/resources/extend_policy.json b/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/test/resources/extend_policy.json
new file mode 100644
index 0000000..3a5bd86
--- /dev/null
+++ b/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/test/resources/extend_policy.json
@@ -0,0 +1,22 @@
+[
+  {
+    "name": "extend-policy",
+    "definition": {
+      "type": "Custom",
+      "handlerClass": "i.o.x.Handler",
+      "value": "",
+      "properties": {
+        "parentKey": {
+          "syslogStream": {
+            "pattern": "%s-%s",
+            "columns": [
+              "a",
+              "b",
+              "d"
+            ]
+          }
+        }
+      }
+    }
+  }
+]


[37/52] [abbrv] incubator-eagle git commit: [EAGLE-524] aggregation framework-job level metrics aggregation

Posted by yo...@apache.org.
[EAGLE-524] aggregation framework-job level metrics aggregation

Author: wujinhu <wu...@126.com>

Closes #419 from wujinhu/aggregation.


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

Branch: refs/heads/master
Commit: 8774b85cd97b7b4c386c1b04cff36c13b3bb82d3
Parents: b66e27b
Author: wujinhu <wu...@126.com>
Authored: Tue Sep 6 17:22:42 2016 +0800
Committer: Qingwen Zhao <qi...@gmail.com>
Committed: Tue Sep 6 17:22:42 2016 +0800

----------------------------------------------------------------------
 .../jpm/mr/history/MRHistoryJobApplication.java |   2 +-
 .../jpm/mr/history/MRHistoryJobConfig.java      |  24 +---
 .../crawler/DefaultJHFInputStreamCallback.java  |  11 +-
 .../history/crawler/JHFCrawlerDriverImpl.java   |  12 +-
 .../metrics/JobCountMetricsGenerator.java       |  18 +--
 .../metrics/JobCounterMetricsGenerator.java     | 133 +++++++++++++++++++
 .../JobExecutionMetricsCreationListener.java    |   4 +-
 .../mr/history/parser/JHFEventReaderBase.java   |  16 ++-
 .../mr/history/parser/JHFMRVer1EventReader.java |   5 +-
 .../mr/history/parser/JHFMRVer2EventReader.java |   5 +-
 .../jpm/mr/history/parser/JHFParserFactory.java |  22 +--
 ...JobConfigurationCreationServiceListener.java |  16 +--
 .../JobEntityCreationEagleServiceListener.java  |  22 ++-
 .../parser/TaskAttemptCounterListener.java      |  16 +--
 .../mr/history/parser/TaskFailureListener.java  |  16 +--
 .../jpm/mr/history/storm/JobHistorySpout.java   |  37 +++---
 .../org/apache/eagle/jpm/util/Constants.java    |   2 +
 17 files changed, 228 insertions(+), 133 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/8774b85c/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/MRHistoryJobApplication.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/MRHistoryJobApplication.java b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/MRHistoryJobApplication.java
index 08607a1..beec938 100644
--- a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/MRHistoryJobApplication.java
+++ b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/MRHistoryJobApplication.java
@@ -67,7 +67,7 @@ public class MRHistoryJobApplication extends StormApplication {
         }
         topologyBuilder.setSpout(
             spoutName,
-            new JobHistorySpout(filter, appConfig),
+            new JobHistorySpout(filter, config),
             parallelism
         ).setNumTasks(tasks);
         return topologyBuilder.createTopology();

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/8774b85c/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/MRHistoryJobConfig.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/MRHistoryJobConfig.java b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/MRHistoryJobConfig.java
index c0943de..4ac875b 100644
--- a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/MRHistoryJobConfig.java
+++ b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/MRHistoryJobConfig.java
@@ -127,29 +127,19 @@ public class MRHistoryJobConfig implements Serializable {
         this.controlConfig = new ControlConfig();
         this.jobExtractorConfig = new JobExtractorConfig();
         this.eagleServiceConfig = new EagleServiceConfig();
-    }
-
-    public static MRHistoryJobConfig getInstance(String[] args) {
-        manager.init(args);
-        return manager;
+        this.config = null;
     }
 
     public static MRHistoryJobConfig getInstance(Config config) {
-        manager.init(config);
+        if (config != null && manager.config == null) {
+            manager.init(config);
+        }
+
         return manager;
     }
 
-    /**
-     * read configuration file and load hbase config etc.
-     */
-    private void init(String[] args) {
-        // TODO: Probably we can remove the properties file path check in future
-        try {
-            LOG.info("Loading from configuration file");
-            init(new ConfigOptionParser().load(args));
-        } catch (Exception e) {
-            LOG.error("failed to load config");
-        }
+    public static MRHistoryJobConfig get() {
+        return getInstance(null);
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/8774b85c/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/crawler/DefaultJHFInputStreamCallback.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/crawler/DefaultJHFInputStreamCallback.java b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/crawler/DefaultJHFInputStreamCallback.java
index 87cd4e0..14b93af 100644
--- a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/crawler/DefaultJHFInputStreamCallback.java
+++ b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/crawler/DefaultJHFInputStreamCallback.java
@@ -33,20 +33,17 @@ public class DefaultJHFInputStreamCallback implements JHFInputStreamCallback {
 
 
     private JobHistoryContentFilter filter;
-    private MRHistoryJobConfig configManager;
 
-    public DefaultJHFInputStreamCallback(JobHistoryContentFilter filter, MRHistoryJobConfig configManager, EagleOutputCollector eagleCollector) {
+    public DefaultJHFInputStreamCallback(JobHistoryContentFilter filter, EagleOutputCollector eagleCollector) {
         this.filter = filter;
-        this.configManager = configManager;
     }
 
     @Override
     public void onInputStream(InputStream jobFileInputStream, org.apache.hadoop.conf.Configuration conf) throws Exception {
-        final MRHistoryJobConfig.JobExtractorConfig jobExtractorConfig = configManager.getJobExtractorConfig();
         @SuppressWarnings("serial")
         Map<String, String> baseTags = new HashMap<String, String>() {
             {
-                put("site", jobExtractorConfig.site);
+                put("site", MRHistoryJobConfig.get().getJobExtractorConfig().site);
             }
         };
 
@@ -55,9 +52,7 @@ public class DefaultJHFInputStreamCallback implements JHFInputStreamCallback {
             jobFileInputStream.close();
         } else {
             //get parser and parse, do not need to emit data now
-            JHFParserBase parser = JHFParserFactory.getParser(configManager,
-                    baseTags,
-                    conf, filter);
+            JHFParserBase parser = JHFParserFactory.getParser(baseTags, conf, filter);
             parser.parse(jobFileInputStream);
             jobFileInputStream.close();
         }

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/8774b85c/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/crawler/JHFCrawlerDriverImpl.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/crawler/JHFCrawlerDriverImpl.java b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/crawler/JHFCrawlerDriverImpl.java
index 2f326fe..55ffc19 100644
--- a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/crawler/JHFCrawlerDriverImpl.java
+++ b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/crawler/JHFCrawlerDriverImpl.java
@@ -65,12 +65,10 @@ public class JHFCrawlerDriverImpl implements JHFCrawlerDriver {
     private TimeZone timeZone;
     private JobCountMetricsGenerator jobCountMetricsGenerator;
 
-    public JHFCrawlerDriverImpl(MRHistoryJobConfig.EagleServiceConfig eagleServiceConfig,
-                                MRHistoryJobConfig.JobExtractorConfig jobExtractorConfig,
-                                MRHistoryJobConfig.ControlConfig controlConfig, JHFInputStreamCallback reader,
+    public JHFCrawlerDriverImpl(JHFInputStreamCallback reader,
                                 JobHistoryLCM historyLCM, JobIdFilter jobFilter, int partitionId) throws Exception {
-        this.zeroBasedMonth = controlConfig.zeroBasedMonth;
-        this.dryRun = controlConfig.dryRun;
+        this.zeroBasedMonth = MRHistoryJobConfig.get().getControlConfig().zeroBasedMonth;
+        this.dryRun = MRHistoryJobConfig.get().getControlConfig().dryRun;
         if (this.dryRun)  {
             LOG.info("this is a dry run");
         }
@@ -78,8 +76,8 @@ public class JHFCrawlerDriverImpl implements JHFCrawlerDriver {
         jhfLCM = historyLCM;//new JobHistoryDAOImpl(jobHistoryConfig);
         this.partitionId = partitionId;
         this.jobFilter = jobFilter;
-        timeZone = TimeZone.getTimeZone(controlConfig.timeZone);
-        jobCountMetricsGenerator = new JobCountMetricsGenerator(eagleServiceConfig, jobExtractorConfig, timeZone);
+        timeZone = TimeZone.getTimeZone(MRHistoryJobConfig.get().getControlConfig().timeZone);
+        jobCountMetricsGenerator = new JobCountMetricsGenerator(timeZone);
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/8774b85c/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/metrics/JobCountMetricsGenerator.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/metrics/JobCountMetricsGenerator.java b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/metrics/JobCountMetricsGenerator.java
index 0e0e5e9..642170d 100644
--- a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/metrics/JobCountMetricsGenerator.java
+++ b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/metrics/JobCountMetricsGenerator.java
@@ -34,15 +34,9 @@ import java.util.*;
 public class JobCountMetricsGenerator {
     private static final Logger LOG = LoggerFactory.getLogger(JobCountMetricsGenerator.class);
 
-    private MRHistoryJobConfig.EagleServiceConfig eagleServiceConfig;
-    private MRHistoryJobConfig.JobExtractorConfig jobExtractorConfig;
     private TimeZone timeZone;
 
-    public JobCountMetricsGenerator(MRHistoryJobConfig.EagleServiceConfig eagleServiceConfig,
-                                    MRHistoryJobConfig.JobExtractorConfig jobExtractorConfig,
-                                    TimeZone timeZone) {
-        this.eagleServiceConfig = eagleServiceConfig;
-        this.jobExtractorConfig = jobExtractorConfig;
+    public JobCountMetricsGenerator(TimeZone timeZone) {
         this.timeZone = timeZone;
     }
 
@@ -57,10 +51,10 @@ public class JobCountMetricsGenerator {
         }
 
         IEagleServiceClient client = new EagleServiceClientImpl(
-            eagleServiceConfig.eagleServiceHost,
-            eagleServiceConfig.eagleServicePort,
-            eagleServiceConfig.username,
-            eagleServiceConfig.password);
+            MRHistoryJobConfig.get().getEagleServiceConfig().eagleServiceHost,
+            MRHistoryJobConfig.get().getEagleServiceConfig().eagleServicePort,
+            MRHistoryJobConfig.get().getEagleServiceConfig().username,
+            MRHistoryJobConfig.get().getEagleServiceConfig().password);
 
 
         GregorianCalendar cal = new GregorianCalendar(year, month, day);
@@ -72,7 +66,7 @@ public class JobCountMetricsGenerator {
         @SuppressWarnings("serial")
         Map<String, String> baseTags = new HashMap<String, String>() {
             {
-                put("site", jobExtractorConfig.site);
+                put("site", MRHistoryJobConfig.get().getJobExtractorConfig().site);
             }
         };
         metricEntity.setTags(baseTags);

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/8774b85c/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/metrics/JobCounterMetricsGenerator.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/metrics/JobCounterMetricsGenerator.java b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/metrics/JobCounterMetricsGenerator.java
new file mode 100644
index 0000000..6291b37
--- /dev/null
+++ b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/metrics/JobCounterMetricsGenerator.java
@@ -0,0 +1,133 @@
+/*
+ * 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.eagle.jpm.mr.history.metrics;
+
+import org.apache.eagle.jpm.mr.history.MRHistoryJobConfig;
+import org.apache.eagle.jpm.mr.historyentity.TaskExecutionAPIEntity;
+import org.apache.eagle.jpm.util.Constants;
+import org.apache.eagle.jpm.util.MRJobTagName;
+import org.apache.eagle.jpm.util.jobcounter.JobCounters;
+import org.apache.eagle.log.entity.GenericMetricEntity;
+import org.apache.eagle.service.client.IEagleServiceClient;
+import org.apache.eagle.service.client.impl.EagleServiceClientImpl;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+public class JobCounterMetricsGenerator {
+    private static final Logger LOG = LoggerFactory.getLogger(JobCounterMetricsGenerator.class);
+    private static final int BATCH_SIZE = 1000;
+
+    private List<List<GenericMetricEntity>> metricEntities = new ArrayList<>();
+    //metric, time, value
+    private Map<String, Map<Long, Long>> metricValueByMinute = new HashMap<>();
+
+    private List<GenericMetricEntity> lastEntitiesBatch;
+    private Map<String, String> baseTags;
+
+    public JobCounterMetricsGenerator() {
+        this.lastEntitiesBatch = null;
+    }
+
+    public void setBaseTags(Map<String, String> tags) {
+        this.baseTags = tags;
+    }
+
+    public void taskExecutionEntityCreated(TaskExecutionAPIEntity taskExecutionAPIEntity) {
+        JobCounters jobCounters = taskExecutionAPIEntity.getJobCounters();
+        if (jobCounters == null || jobCounters.getCounters() == null) {
+            LOG.warn("found null job counters, task {}", taskExecutionAPIEntity.getTags().get(MRJobTagName.TASK_ID.toString()));
+            return;
+        }
+
+        long duration = taskExecutionAPIEntity.getDuration();
+        long startTime = taskExecutionAPIEntity.getStartTime();
+        long endTime = taskExecutionAPIEntity.getEndTime();
+
+        Map<String, Map<String, Long>> counters = jobCounters.getCounters();
+        for (String groupName : counters.keySet()) {
+            Map<String, Long> metricValues = counters.get(groupName);
+            for (String metric : metricValues.keySet()) {
+                if (!metricValueByMinute.containsKey(metric)) {
+                    metricValueByMinute.put(metric, new HashMap<>());
+                }
+                Long value = metricValues.get(metric);
+                double avg = value * 1.0 / duration;
+                for (long i = startTime; i <= endTime;) {
+                    long timeStamp = i / 60000L * 60000L;
+                    if (!metricValueByMinute.get(metric).containsKey(timeStamp)) {
+                        metricValueByMinute.get(metric).put(timeStamp, 0L);
+                    }
+                    long valueByEachMinute = metricValueByMinute.get(metric).get(timeStamp);
+                    if (endTime >= timeStamp + 60000L) {
+                        metricValueByMinute.get(metric).put(timeStamp, valueByEachMinute + (long)(avg * (timeStamp + 60000L - i)));
+                    } else {
+                        metricValueByMinute.get(metric).put(timeStamp, valueByEachMinute + (long)(avg * (endTime - timeStamp)));
+                    }
+
+                    i = timeStamp + 60000L;
+                }
+            }
+        }
+    }
+
+    private String buildMetricName(String field) {
+        return String.format(Constants.HADOOP_HISTORY_MINUTE_METRIC_FORMAT, Constants.JOB_LEVEL, field);
+    }
+
+    public void flush() throws Exception {
+        for (String metric : metricValueByMinute.keySet()) {
+            Map<Long, Long> valueByMinute = metricValueByMinute.get(metric);
+            for (Long timeStamp : valueByMinute.keySet()) {
+                GenericMetricEntity metricEntity = new GenericMetricEntity();
+                metricEntity.setTimestamp(timeStamp);
+                metricEntity.setPrefix(buildMetricName(metric.toLowerCase()));
+                metricEntity.setValue(new double[] {valueByMinute.get(timeStamp)});
+                metricEntity.setTags(this.baseTags);
+
+                if (this.lastEntitiesBatch == null || this.lastEntitiesBatch.size() > BATCH_SIZE) {
+                    this.lastEntitiesBatch = new ArrayList<>();
+                    metricEntities.add(this.lastEntitiesBatch);
+                }
+
+                this.lastEntitiesBatch.add(metricEntity);
+            }
+        }
+
+        IEagleServiceClient client = new EagleServiceClientImpl(
+            MRHistoryJobConfig.get().getEagleServiceConfig().eagleServiceHost,
+            MRHistoryJobConfig.get().getEagleServiceConfig().eagleServicePort,
+            MRHistoryJobConfig.get().getEagleServiceConfig().username,
+            MRHistoryJobConfig.get().getEagleServiceConfig().password);
+
+        for (List<GenericMetricEntity> entities : metricEntities) {
+            LOG.info("start flushing entities of total number " + entities.size());
+            client.create(entities);
+            LOG.info("finish flushing entities of total number " + entities.size());
+            entities.clear();
+        }
+        client.getJerseyClient().destroy();
+        client.close();
+        metricEntities.clear();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/8774b85c/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/metrics/JobExecutionMetricsCreationListener.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/metrics/JobExecutionMetricsCreationListener.java b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/metrics/JobExecutionMetricsCreationListener.java
index 2129bed..d7e8fcc 100644
--- a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/metrics/JobExecutionMetricsCreationListener.java
+++ b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/metrics/JobExecutionMetricsCreationListener.java
@@ -57,7 +57,7 @@ public class JobExecutionMetricsCreationListener extends AbstractMetricsCreation
                 for (Map<String, Long> metricGroup : jobCounters.getCounters().values()) {
                     for (Map.Entry<String, Long> entry : metricGroup.entrySet()) {
                         String metricName = entry.getKey().toLowerCase();
-                        metrics.add(metricWrapper(timeStamp, "history." + metricName, new double[]{entry.getValue()}, tags));
+                        metrics.add(metricWrapper(timeStamp, metricName, new double[]{entry.getValue()}, tags));
                     }
                 }
             }
@@ -67,7 +67,7 @@ public class JobExecutionMetricsCreationListener extends AbstractMetricsCreation
 
     @Override
     public String buildMetricName(String field) {
-        return String.format(Constants.hadoopMetricFormat, Constants.JOB_LEVEL, field);
+        return String.format(Constants.HADOOP_HISTORY_TOTAL_METRIC_FORMAT, Constants.JOB_LEVEL, field);
     }
 
 

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/8774b85c/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JHFEventReaderBase.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JHFEventReaderBase.java b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JHFEventReaderBase.java
index 1570956..d33c26b 100644
--- a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JHFEventReaderBase.java
+++ b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JHFEventReaderBase.java
@@ -18,9 +18,9 @@
 
 package org.apache.eagle.jpm.mr.history.parser;
 
-import org.apache.commons.io.FilenameUtils;
-import org.apache.eagle.jpm.mr.history.MRHistoryJobConfig.JobHistoryEndpointConfig;
+import org.apache.eagle.jpm.mr.history.MRHistoryJobConfig;
 import org.apache.eagle.jpm.mr.history.crawler.JobHistoryContentFilter;
+import org.apache.eagle.jpm.mr.history.metrics.JobCounterMetricsGenerator;
 import org.apache.eagle.jpm.mr.history.parser.JHFMRVer1Parser.Keys;
 import org.apache.eagle.jpm.mr.historyentity.*;
 import org.apache.eagle.jpm.util.Constants;
@@ -69,7 +69,6 @@ public abstract class JHFEventReaderBase extends JobEntityCreationPublisher impl
     protected String queueName;
     protected Long jobLaunchTime;
     protected JobHistoryContentFilter filter;
-    private JobHistoryEndpointConfig jobHistoryEndpointConfig;
 
     protected final List<HistoryJobEntityLifecycleListener> jobEntityLifecycleListeners = new ArrayList<>();
 
@@ -78,6 +77,8 @@ public abstract class JHFEventReaderBase extends JobEntityCreationPublisher impl
     private long sumMapTaskDuration;
     private long sumReduceTaskDuration;
 
+    private JobCounterMetricsGenerator jobCounterMetricsGenerator;
+
     public Constants.JobType fetchJobType(Configuration config) {
         if (config.get(Constants.JobConfiguration.CASCADING_JOB) != null) {
             return Constants.JobType.CASCADING;
@@ -101,9 +102,8 @@ public abstract class JHFEventReaderBase extends JobEntityCreationPublisher impl
      *
      * @param baseTags
      */
-    public JHFEventReaderBase(Map<String, String> baseTags, Configuration configuration, JobHistoryContentFilter filter, JobHistoryEndpointConfig jobHistoryEndpointConfig) {
+    public JHFEventReaderBase(Map<String, String> baseTags, Configuration configuration, JobHistoryContentFilter filter) {
         this.filter = filter;
-        this.jobHistoryEndpointConfig = jobHistoryEndpointConfig;
 
         this.baseTags = baseTags;
         jobSubmitEventEntity = new JobEventAPIEntity();
@@ -134,6 +134,7 @@ public abstract class JHFEventReaderBase extends JobEntityCreationPublisher impl
         }
         this.sumMapTaskDuration = 0L;
         this.sumReduceTaskDuration = 0L;
+        this.jobCounterMetricsGenerator = new JobCounterMetricsGenerator();
     }
 
     public void register(HistoryJobEntityLifecycleListener lifecycleListener) {
@@ -148,6 +149,7 @@ public abstract class JHFEventReaderBase extends JobEntityCreationPublisher impl
         }
         try {
             flush();
+            this.jobCounterMetricsGenerator.flush();
         } catch (Exception ex) {
             throw new IOException(ex);
         }
@@ -162,7 +164,7 @@ public abstract class JHFEventReaderBase extends JobEntityCreationPublisher impl
     }
 
     private String buildJobTrackingUrl(String jobId) {
-        String jobTrackingUrlBase = this.jobHistoryEndpointConfig.mrHistoryServerUrl + "/jobhistory/job/";
+        String jobTrackingUrlBase = MRHistoryJobConfig.getInstance(null).getJobHistoryEndpointConfig().mrHistoryServerUrl + "/jobhistory/job/";
         try {
             URI oldUri = new URI(jobTrackingUrlBase);
             URI resolved = oldUri.resolve(jobId);
@@ -303,6 +305,7 @@ public abstract class JHFEventReaderBase extends JobEntityCreationPublisher impl
             } else {
                 jobExecutionEntity.setAvgReduceTaskDuration(this.sumReduceTaskDuration * 1.0 / numTotalReduces);
             }
+            this.jobCounterMetricsGenerator.setBaseTags(jobExecutionEntity.getTags());
             entityCreated(jobExecutionEntity);
         }
     }
@@ -401,6 +404,7 @@ public abstract class JHFEventReaderBase extends JobEntityCreationPublisher impl
             }
 
             entityCreated(entity);
+            this.jobCounterMetricsGenerator.taskExecutionEntityCreated(entity);
             //_taskStartTime.remove(taskID); // clean this taskID
         } else if ((recType == RecordTypes.MapAttempt || recType == RecordTypes.ReduceAttempt) && startTime != null) { // task attempt start
             taskAttemptStartTime.put(taskAttemptID, Long.valueOf(startTime));

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/8774b85c/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JHFMRVer1EventReader.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JHFMRVer1EventReader.java b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JHFMRVer1EventReader.java
index 0e9458a..e20836f 100644
--- a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JHFMRVer1EventReader.java
+++ b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JHFMRVer1EventReader.java
@@ -18,7 +18,6 @@
 
 package org.apache.eagle.jpm.mr.history.parser;
 
-import org.apache.eagle.jpm.mr.history.MRHistoryJobConfig.JobHistoryEndpointConfig;
 import org.apache.eagle.jpm.mr.history.crawler.JobHistoryContentFilter;
 import org.apache.eagle.jpm.mr.history.parser.JHFMRVer1Parser.Keys;
 import org.apache.eagle.jpm.mr.historyentity.JobExecutionAPIEntity;
@@ -49,8 +48,8 @@ public class JHFMRVer1EventReader extends JHFEventReaderBase implements JHFMRVer
      *
      * @param baseTags
      */
-    public JHFMRVer1EventReader(Map<String, String> baseTags, Configuration configuration, JobHistoryContentFilter filter, JobHistoryEndpointConfig jobHistoryEndpointConfig) {
-        super(baseTags, configuration, filter, jobHistoryEndpointConfig);
+    public JHFMRVer1EventReader(Map<String, String> baseTags, Configuration configuration, JobHistoryContentFilter filter) {
+        super(baseTags, configuration, filter);
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/8774b85c/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JHFMRVer2EventReader.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JHFMRVer2EventReader.java b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JHFMRVer2EventReader.java
index 74f84f6..0919aa0 100644
--- a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JHFMRVer2EventReader.java
+++ b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JHFMRVer2EventReader.java
@@ -18,7 +18,6 @@
 
 package org.apache.eagle.jpm.mr.history.parser;
 
-import org.apache.eagle.jpm.mr.history.MRHistoryJobConfig.JobHistoryEndpointConfig;
 import org.apache.eagle.jpm.mr.history.crawler.JobHistoryContentFilter;
 import org.apache.eagle.jpm.mr.history.parser.JHFMRVer1Parser.Keys;
 import org.apache.eagle.jpm.util.jobcounter.JobCounters;
@@ -44,8 +43,8 @@ public class JHFMRVer2EventReader extends JHFEventReaderBase {
      *
      * @throws IOException
      */
-    public JHFMRVer2EventReader(Map<String, String> baseTags, Configuration configuration, JobHistoryContentFilter filter, JobHistoryEndpointConfig jobHistoryEndpointConfig) {
-        super(baseTags, configuration, filter, jobHistoryEndpointConfig);
+    public JHFMRVer2EventReader(Map<String, String> baseTags, Configuration configuration, JobHistoryContentFilter filter) {
+        super(baseTags, configuration, filter);
     }
 
     @SuppressWarnings("deprecation")

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/8774b85c/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JHFParserFactory.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JHFParserFactory.java b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JHFParserFactory.java
index 386d50c..56fd956 100755
--- a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JHFParserFactory.java
+++ b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JHFParserFactory.java
@@ -30,8 +30,8 @@ public class JHFParserFactory {
 
     private static final Logger LOG = LoggerFactory.getLogger(JHFParserFactory.class);
 
-    public static JHFParserBase getParser(MRHistoryJobConfig configManager, Map<String, String> baseTags, Configuration configuration, JobHistoryContentFilter filter) {
-        String format = configManager.getJobExtractorConfig().mrVersion;
+    public static JHFParserBase getParser(Map<String, String> baseTags, Configuration configuration, JobHistoryContentFilter filter) {
+        String format = MRHistoryJobConfig.get().getJobExtractorConfig().mrVersion;
         JHFParserBase parser;
         JHFFormat f;
         try {
@@ -46,21 +46,21 @@ public class JHFParserFactory {
 
         switch (f) {
             case MRVer2:
-                JHFMRVer2EventReader reader2 = new JHFMRVer2EventReader(baseTags, configuration, filter, configManager.getJobHistoryEndpointConfig());
-                reader2.addListener(new JobEntityCreationEagleServiceListener(configManager));
-                reader2.addListener(new TaskFailureListener(configManager));
-                reader2.addListener(new TaskAttemptCounterListener(configManager));
-                reader2.addListener(new JobConfigurationCreationServiceListener(configManager));
+                JHFMRVer2EventReader reader2 = new JHFMRVer2EventReader(baseTags, configuration, filter);
+                reader2.addListener(new JobEntityCreationEagleServiceListener());
+                reader2.addListener(new TaskFailureListener());
+                reader2.addListener(new TaskAttemptCounterListener());
+                reader2.addListener(new JobConfigurationCreationServiceListener());
 
                 reader2.register(new JobEntityLifecycleAggregator());
                 parser = new JHFMRVer2Parser(reader2);
                 break;
             case MRVer1:
             default:
-                JHFMRVer1EventReader reader1 = new JHFMRVer1EventReader(baseTags, configuration, filter, configManager.getJobHistoryEndpointConfig());
-                reader1.addListener(new JobEntityCreationEagleServiceListener(configManager));
-                reader1.addListener(new TaskFailureListener(configManager));
-                reader1.addListener(new TaskAttemptCounterListener(configManager));
+                JHFMRVer1EventReader reader1 = new JHFMRVer1EventReader(baseTags, configuration, filter);
+                reader1.addListener(new JobEntityCreationEagleServiceListener());
+                reader1.addListener(new TaskFailureListener());
+                reader1.addListener(new TaskAttemptCounterListener());
 
                 reader1.register(new JobEntityLifecycleAggregator());
                 parser = new JHFMRVer1Parser(reader1);

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/8774b85c/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JobConfigurationCreationServiceListener.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JobConfigurationCreationServiceListener.java b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JobConfigurationCreationServiceListener.java
index 7293c89..bf93432 100644
--- a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JobConfigurationCreationServiceListener.java
+++ b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JobConfigurationCreationServiceListener.java
@@ -32,11 +32,9 @@ import java.util.List;
 public class JobConfigurationCreationServiceListener implements HistoryJobEntityLifecycleListener {
     private static final Logger logger = LoggerFactory.getLogger(JobConfigurationCreationServiceListener.class);
     private static final int MAX_RETRY_TIMES = 3;
-    private MRHistoryJobConfig configManager;
     private JobConfigurationAPIEntity jobConfigurationEntity;
 
-    public JobConfigurationCreationServiceListener(MRHistoryJobConfig configManager) {
-        this.configManager = configManager;
+    public JobConfigurationCreationServiceListener() {
     }
 
     @Override
@@ -55,15 +53,13 @@ public class JobConfigurationCreationServiceListener implements HistoryJobEntity
 
     @Override
     public void flush() throws Exception {
-        MRHistoryJobConfig.EagleServiceConfig eagleServiceConfig = configManager.getEagleServiceConfig();
-        MRHistoryJobConfig.JobExtractorConfig jobExtractorConfig = configManager.getJobExtractorConfig();
         IEagleServiceClient client = new EagleServiceClientImpl(
-            eagleServiceConfig.eagleServiceHost,
-            eagleServiceConfig.eagleServicePort,
-            eagleServiceConfig.username,
-            eagleServiceConfig.password);
+            MRHistoryJobConfig.get().getEagleServiceConfig().eagleServiceHost,
+            MRHistoryJobConfig.get().getEagleServiceConfig().eagleServicePort,
+            MRHistoryJobConfig.get().getEagleServiceConfig().username,
+            MRHistoryJobConfig.get().getEagleServiceConfig().password);
 
-        client.getJerseyClient().setReadTimeout(jobExtractorConfig.readTimeoutSeconds * 1000);
+        client.getJerseyClient().setReadTimeout(MRHistoryJobConfig.get().getJobExtractorConfig().readTimeoutSeconds * 1000);
         List<JobConfigurationAPIEntity> list = new ArrayList<>();
         list.add(jobConfigurationEntity);
 

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/8774b85c/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JobEntityCreationEagleServiceListener.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JobEntityCreationEagleServiceListener.java b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JobEntityCreationEagleServiceListener.java
index 623a776..74368a5 100644
--- a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JobEntityCreationEagleServiceListener.java
+++ b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JobEntityCreationEagleServiceListener.java
@@ -40,7 +40,6 @@ public class JobEntityCreationEagleServiceListener implements HistoryJobEntityCr
     private static final int BATCH_SIZE = 1000;
     private int batchSize;
     private List<JobBaseAPIEntity> list = new ArrayList<>();
-    private MRHistoryJobConfig configManager;
     List<JobExecutionAPIEntity> jobs = new ArrayList<>();
     List<JobEventAPIEntity> jobEvents = new ArrayList<>();
     List<TaskExecutionAPIEntity> taskExecs = new ArrayList<>();
@@ -48,17 +47,16 @@ public class JobEntityCreationEagleServiceListener implements HistoryJobEntityCr
     private JobExecutionMetricsCreationListener jobExecutionMetricsCreationListener = new JobExecutionMetricsCreationListener();
     private TimeZone timeZone;
 
-    public JobEntityCreationEagleServiceListener(MRHistoryJobConfig configManager) {
-        this(configManager, BATCH_SIZE);
+    public JobEntityCreationEagleServiceListener() {
+        this(BATCH_SIZE);
     }
 
-    public JobEntityCreationEagleServiceListener(MRHistoryJobConfig configManager, int batchSize) {
-        this.configManager = configManager;
+    public JobEntityCreationEagleServiceListener(int batchSize) {
         if (batchSize <= 0) {
             throw new IllegalArgumentException("batchSize must be greater than 0 when it is provided");
         }
         this.batchSize = batchSize;
-        timeZone = TimeZone.getTimeZone(configManager.getControlConfig().timeZone);
+        timeZone = TimeZone.getTimeZone(MRHistoryJobConfig.get().getControlConfig().timeZone);
     }
 
     @Override
@@ -84,15 +82,13 @@ public class JobEntityCreationEagleServiceListener implements HistoryJobEntityCr
      */
     @Override
     public void flush() throws Exception {
-        MRHistoryJobConfig.EagleServiceConfig eagleServiceConfig = configManager.getEagleServiceConfig();
-        MRHistoryJobConfig.JobExtractorConfig jobExtractorConfig = configManager.getJobExtractorConfig();
         IEagleServiceClient client = new EagleServiceClientImpl(
-            eagleServiceConfig.eagleServiceHost,
-            eagleServiceConfig.eagleServicePort,
-            eagleServiceConfig.username,
-            eagleServiceConfig.password);
+            MRHistoryJobConfig.get().getEagleServiceConfig().eagleServiceHost,
+            MRHistoryJobConfig.get().getEagleServiceConfig().eagleServicePort,
+            MRHistoryJobConfig.get().getEagleServiceConfig().username,
+            MRHistoryJobConfig.get().getEagleServiceConfig().password);
 
-        client.getJerseyClient().setReadTimeout(jobExtractorConfig.readTimeoutSeconds * 1000);
+        client.getJerseyClient().setReadTimeout(MRHistoryJobConfig.get().getJobExtractorConfig().readTimeoutSeconds * 1000);
         logger.info("start flushing entities of total number " + list.size());
         List<GenericMetricEntity> metricEntities = new ArrayList<>();
         for (int i = 0; i < list.size(); i++) {

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/8774b85c/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/TaskAttemptCounterListener.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/TaskAttemptCounterListener.java b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/TaskAttemptCounterListener.java
index efc43c5..ef7c8e9 100755
--- a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/TaskAttemptCounterListener.java
+++ b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/TaskAttemptCounterListener.java
@@ -34,10 +34,8 @@ public class TaskAttemptCounterListener implements HistoryJobEntityCreationListe
     private static final Logger logger = LoggerFactory.getLogger(TaskAttemptCounterListener.class);
     private static final int BATCH_SIZE = 1000;
     private Map<CounterKey, CounterValue> counters = new HashMap<>();
-    private MRHistoryJobConfig configManager;
 
-    public TaskAttemptCounterListener(MRHistoryJobConfig configManager) {
-        this.configManager = configManager;
+    public TaskAttemptCounterListener() {
     }
 
     private static class CounterKey {
@@ -112,15 +110,13 @@ public class TaskAttemptCounterListener implements HistoryJobEntityCreationListe
 
     @Override
     public void flush() throws Exception {
-        MRHistoryJobConfig.EagleServiceConfig eagleServiceConfig = configManager.getEagleServiceConfig();
-        MRHistoryJobConfig.JobExtractorConfig jobExtractorConfig = configManager.getJobExtractorConfig();
         IEagleServiceClient client = new EagleServiceClientImpl(
-            eagleServiceConfig.eagleServiceHost,
-            eagleServiceConfig.eagleServicePort,
-            eagleServiceConfig.username,
-            eagleServiceConfig.password);
+            MRHistoryJobConfig.get().getEagleServiceConfig().eagleServiceHost,
+            MRHistoryJobConfig.get().getEagleServiceConfig().eagleServicePort,
+            MRHistoryJobConfig.get().getEagleServiceConfig().username,
+            MRHistoryJobConfig.get().getEagleServiceConfig().password);
 
-        client.getJerseyClient().setReadTimeout(jobExtractorConfig.readTimeoutSeconds * 1000);
+        client.getJerseyClient().setReadTimeout(MRHistoryJobConfig.get().getJobExtractorConfig().readTimeoutSeconds * 1000);
         List<TaskAttemptCounterAPIEntity> list = new ArrayList<>();
         logger.info("start flushing TaskAttemptCounter entities of total number " + counters.size());
         // create entity

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/8774b85c/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/TaskFailureListener.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/TaskFailureListener.java b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/TaskFailureListener.java
index f95eaa2..1a7a5fc 100755
--- a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/TaskFailureListener.java
+++ b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/TaskFailureListener.java
@@ -44,10 +44,8 @@ public class TaskFailureListener implements HistoryJobEntityCreationListener {
 
     private final List<TaskFailureCountAPIEntity> failureTasks = new ArrayList<TaskFailureCountAPIEntity>();
     private final MRErrorClassifier classifier;
-    private MRHistoryJobConfig configManager;
 
-    public TaskFailureListener(MRHistoryJobConfig configManager) {
-        this.configManager = configManager;
+    public TaskFailureListener() {
         InputStream is = null;
         try {
             is = TaskFailureListener.class.getClassLoader().getResourceAsStream(MR_ERROR_CATEGORY_CONFIG_FILE_NAME);
@@ -109,15 +107,13 @@ public class TaskFailureListener implements HistoryJobEntityCreationListener {
 
     @Override
     public void flush() throws Exception {
-        MRHistoryJobConfig.EagleServiceConfig eagleServiceConfig = configManager.getEagleServiceConfig();
-        MRHistoryJobConfig.JobExtractorConfig jobExtractorConfig = configManager.getJobExtractorConfig();
         IEagleServiceClient client = new EagleServiceClientImpl(
-            eagleServiceConfig.eagleServiceHost,
-            eagleServiceConfig.eagleServicePort,
-            eagleServiceConfig.username,
-            eagleServiceConfig.password);
+            MRHistoryJobConfig.get().getEagleServiceConfig().eagleServiceHost,
+            MRHistoryJobConfig.get().getEagleServiceConfig().eagleServicePort,
+            MRHistoryJobConfig.get().getEagleServiceConfig().username,
+            MRHistoryJobConfig.get().getEagleServiceConfig().password);
 
-        client.getJerseyClient().setReadTimeout(jobExtractorConfig.readTimeoutSeconds * 1000);
+        client.getJerseyClient().setReadTimeout(MRHistoryJobConfig.get().getJobExtractorConfig().readTimeoutSeconds * 1000);
 
         int tried = 0;
         while (tried <= MAX_RETRY_TIMES) {

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/8774b85c/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/storm/JobHistorySpout.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/storm/JobHistorySpout.java b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/storm/JobHistorySpout.java
index 04283d3..da98e0d 100644
--- a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/storm/JobHistorySpout.java
+++ b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/storm/JobHistorySpout.java
@@ -18,6 +18,7 @@
 
 package org.apache.eagle.jpm.mr.history.storm;
 
+import com.typesafe.config.Config;
 import org.apache.eagle.jpm.mr.history.MRHistoryJobConfig;
 import org.apache.eagle.jpm.mr.history.crawler.*;
 import org.apache.eagle.jpm.mr.history.zkres.JobHistoryZKStateManager;
@@ -91,22 +92,22 @@ public class JobHistorySpout extends BaseRichSpout {
     private JobHistoryContentFilter contentFilter;
     private JobHistorySpoutCollectorInterceptor interceptor;
     private JHFInputStreamCallback callback;
-    private MRHistoryJobConfig configManager;
     private JobHistoryLCM jhfLCM;
     private static final int MAX_RETRY_TIMES = 3;
+    private Config config;
 
-    public JobHistorySpout(JobHistoryContentFilter filter, MRHistoryJobConfig configManager) {
-        this(filter, configManager, new JobHistorySpoutCollectorInterceptor());
+    public JobHistorySpout(JobHistoryContentFilter filter, Config config) {
+        this(filter, config, new JobHistorySpoutCollectorInterceptor());
     }
 
     /**
      * mostly this constructor signature is for unit test purpose as you can put customized interceptor here.
      */
-    public JobHistorySpout(JobHistoryContentFilter filter, MRHistoryJobConfig configManager, JobHistorySpoutCollectorInterceptor adaptor) {
+    public JobHistorySpout(JobHistoryContentFilter filter, Config config, JobHistorySpoutCollectorInterceptor adaptor) {
         this.contentFilter = filter;
-        this.configManager = configManager;
+        this.config = config;
         this.interceptor = adaptor;
-        callback = new DefaultJHFInputStreamCallback(contentFilter, configManager, interceptor);
+        callback = new DefaultJHFInputStreamCallback(contentFilter, interceptor);
     }
 
     private int calculatePartitionId(TopologyContext context) {
@@ -127,13 +128,14 @@ public class JobHistorySpout extends BaseRichSpout {
     @Override
     public void open(Map conf, TopologyContext context,
                      final SpoutOutputCollector collector) {
+        MRHistoryJobConfig.getInstance(config);
         partitionId = calculatePartitionId(context);
         // sanity verify 0<=partitionId<=numTotalPartitions-1
         if (partitionId < 0 || partitionId > numTotalPartitions) {
             throw new IllegalStateException("partitionId should be less than numTotalPartitions with partitionId "
                 + partitionId + " and numTotalPartitions " + numTotalPartitions);
         }
-        Class<? extends JobIdPartitioner> partitionerCls = configManager.getControlConfig().partitionerCls;
+        Class<? extends JobIdPartitioner> partitionerCls = MRHistoryJobConfig.get().getControlConfig().partitionerCls;
         JobIdPartitioner partitioner;
         try {
             partitioner = partitionerCls.newInstance();
@@ -142,16 +144,13 @@ public class JobHistorySpout extends BaseRichSpout {
             throw new IllegalStateException(e);
         }
         JobIdFilter jobIdFilter = new JobIdFilterByPartition(partitioner, numTotalPartitions, partitionId);
-        JobHistoryZKStateManager.instance().init(configManager.getZkStateConfig());
+        JobHistoryZKStateManager.instance().init(MRHistoryJobConfig.get().getZkStateConfig());
         JobHistoryZKStateManager.instance().ensureJobPartitions(numTotalPartitions);
         interceptor.setSpoutOutputCollector(collector);
 
         try {
-            jhfLCM = new JobHistoryDAOImpl(configManager.getJobHistoryEndpointConfig());
+            jhfLCM = new JobHistoryDAOImpl(MRHistoryJobConfig.get().getJobHistoryEndpointConfig());
             driver = new JHFCrawlerDriverImpl(
-                configManager.getEagleServiceConfig(),
-                configManager.getJobExtractorConfig(),
-                configManager.getControlConfig(),
                 callback,
                 jhfLCM,
                 jobIdFilter,
@@ -232,11 +231,9 @@ public class JobHistorySpout extends BaseRichSpout {
         }
 
         LOG.info("update process time stamp {}", minTimeStamp);
-        final MRHistoryJobConfig.EagleServiceConfig eagleServiceConfig = configManager.getEagleServiceConfig();
-        final MRHistoryJobConfig.JobExtractorConfig jobExtractorConfig = configManager.getJobExtractorConfig();
         Map<String, String> baseTags = new HashMap<String, String>() {
             {
-                put("site", jobExtractorConfig.site);
+                put("site", MRHistoryJobConfig.get().getJobExtractorConfig().site);
             }
         };
         JobProcessTimeStampEntity entity = new JobProcessTimeStampEntity();
@@ -245,12 +242,12 @@ public class JobHistorySpout extends BaseRichSpout {
         entity.setTags(baseTags);
 
         IEagleServiceClient client = new EagleServiceClientImpl(
-            eagleServiceConfig.eagleServiceHost,
-            eagleServiceConfig.eagleServicePort,
-            eagleServiceConfig.username,
-            eagleServiceConfig.password);
+            MRHistoryJobConfig.get().getEagleServiceConfig().eagleServiceHost,
+            MRHistoryJobConfig.get().getEagleServiceConfig().eagleServicePort,
+            MRHistoryJobConfig.get().getEagleServiceConfig().username,
+            MRHistoryJobConfig.get().getEagleServiceConfig().password);
 
-        client.getJerseyClient().setReadTimeout(jobExtractorConfig.readTimeoutSeconds * 1000);
+        client.getJerseyClient().setReadTimeout(MRHistoryJobConfig.get().getJobExtractorConfig().readTimeoutSeconds * 1000);
 
         List<JobProcessTimeStampEntity> entities = new ArrayList<>();
         entities.add(entity);

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/8774b85c/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/Constants.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/Constants.java b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/Constants.java
index e18fe07..5a60ee3 100644
--- a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/Constants.java
+++ b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/Constants.java
@@ -179,4 +179,6 @@ public class Constants {
     public static final String TASK_LEVEL = "task";
     public static final String JOB_COUNT_PER_DAY = "hadoop.job.day.count";
 
+    public static final String HADOOP_HISTORY_TOTAL_METRIC_FORMAT = "hadoop.%s.history.%s";
+    public static final String HADOOP_HISTORY_MINUTE_METRIC_FORMAT = "hadoop.%s.history.minute.%s";
 }


[44/52] [abbrv] incubator-eagle git commit: [Fix] : Have alert stream event only refer to the policy id, not the policy definition

Posted by yo...@apache.org.
[Fix] : Have alert stream event only refer to the policy id, not the policy definition

Author: ralphsu


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

Branch: refs/heads/master
Commit: 3324279e42e17c0a063922be355a135cce8c975c
Parents: 0a2ce9e
Author: Ralph, Su <su...@gmail.com>
Authored: Wed Sep 7 18:11:15 2016 +0800
Committer: Ralph, Su <su...@gmail.com>
Committed: Wed Sep 7 18:11:15 2016 +0800

----------------------------------------------------------------------
 .../alert/engine/model/AlertStreamEvent.java      | 18 ++++++------------
 .../evaluator/impl/AlertStreamCallback.java       |  2 +-
 .../evaluator/nodata/NoDataPolicyHandler.java     | 11 +++--------
 .../nodata/NoDataPolicyTimeBatchHandler.java      | 10 +++-------
 .../alert/engine/utils/AlertStreamUtils.java      |  2 +-
 .../engine/router/TestAlertPublisherBolt.java     |  4 ++--
 6 files changed, 16 insertions(+), 31 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/3324279e/eagle-core/eagle-alert-parent/eagle-alert/alert-common/src/main/java/org/apache/eagle/alert/engine/model/AlertStreamEvent.java
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-alert-parent/eagle-alert/alert-common/src/main/java/org/apache/eagle/alert/engine/model/AlertStreamEvent.java b/eagle-core/eagle-alert-parent/eagle-alert/alert-common/src/main/java/org/apache/eagle/alert/engine/model/AlertStreamEvent.java
index 6e784c5..13881a1 100644
--- a/eagle-core/eagle-alert-parent/eagle-alert/alert-common/src/main/java/org/apache/eagle/alert/engine/model/AlertStreamEvent.java
+++ b/eagle-core/eagle-alert-parent/eagle-alert/alert-common/src/main/java/org/apache/eagle/alert/engine/model/AlertStreamEvent.java
@@ -17,7 +17,6 @@
 package org.apache.eagle.alert.engine.model;
 
 import org.apache.commons.lang3.StringUtils;
-import org.apache.eagle.alert.engine.coordinator.PolicyDefinition;
 import org.apache.eagle.alert.engine.coordinator.StreamDefinition;
 import org.apache.eagle.alert.utils.DateTimeUtil;
 
@@ -30,22 +29,17 @@ import java.util.List;
 public class AlertStreamEvent extends StreamEvent {
     private static final long serialVersionUID = 2392131134670106397L;
 
-    // TODO: Keep policy name only instead of policy entity
-    private PolicyDefinition policy;
+    private String policyId;
     private StreamDefinition schema;
     private String createdBy;
     private long createdTime;
 
-    public PolicyDefinition getPolicy() {
-        return policy;
-    }
-
-    public void setPolicy(PolicyDefinition policy) {
-        this.policy = policy;
+    public void setPolicyId(String policyId) {
+        this.policyId = policyId;
     }
 
     public String getPolicyId() {
-        return policy.getName();
+        return policyId;
     }
 
     @Override
@@ -58,8 +52,8 @@ public class AlertStreamEvent extends StreamEvent {
                 dataStrings.add(null);
             }
         }
-        return String.format("AlertStreamEvent[stream=%S,timestamp=%s,data=[%s], policy=%s, createdBy=%s, metaVersion=%s]",
-                this.getStreamId(), DateTimeUtil.millisecondsToHumanDateWithMilliseconds(this.getTimestamp()), StringUtils.join(dataStrings,","),this.getPolicy().getName(),this.getCreatedBy(),this.getMetaVersion());
+        return String.format("AlertStreamEvent[stream=%S,timestamp=%s,data=[%s], policyId=%s, createdBy=%s, metaVersion=%s]",
+                this.getStreamId(), DateTimeUtil.millisecondsToHumanDateWithMilliseconds(this.getTimestamp()), StringUtils.join(dataStrings,","),this.getPolicyId(),this.getCreatedBy(),this.getMetaVersion());
     }
 
     public String getCreatedBy() {

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/3324279e/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/main/java/org/apache/eagle/alert/engine/evaluator/impl/AlertStreamCallback.java
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/main/java/org/apache/eagle/alert/engine/evaluator/impl/AlertStreamCallback.java b/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/main/java/org/apache/eagle/alert/engine/evaluator/impl/AlertStreamCallback.java
index 50e3377..b9a109c 100644
--- a/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/main/java/org/apache/eagle/alert/engine/evaluator/impl/AlertStreamCallback.java
+++ b/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/main/java/org/apache/eagle/alert/engine/evaluator/impl/AlertStreamCallback.java
@@ -68,7 +68,7 @@ public class AlertStreamCallback extends StreamCallback {
             event.setTimestamp(e.getTimestamp());
             event.setData(e.getData());
             event.setStreamId(outputStream);
-            event.setPolicy(context.getPolicyDefinition());
+            event.setPolicyId(context.getPolicyDefinition().getName());
             if (this.context.getPolicyEvaluator() != null) {
                 event.setCreatedBy(context.getPolicyEvaluator().getName());
             }

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/3324279e/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/main/java/org/apache/eagle/alert/engine/evaluator/nodata/NoDataPolicyHandler.java
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/main/java/org/apache/eagle/alert/engine/evaluator/nodata/NoDataPolicyHandler.java b/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/main/java/org/apache/eagle/alert/engine/evaluator/nodata/NoDataPolicyHandler.java
index 9ad7529..8f37b93 100644
--- a/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/main/java/org/apache/eagle/alert/engine/evaluator/nodata/NoDataPolicyHandler.java
+++ b/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/main/java/org/apache/eagle/alert/engine/evaluator/nodata/NoDataPolicyHandler.java
@@ -16,13 +16,6 @@
  */
 package org.apache.eagle.alert.engine.evaluator.nodata;
 
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-
 import org.apache.commons.collections.CollectionUtils;
 import org.apache.eagle.alert.engine.Collector;
 import org.apache.eagle.alert.engine.coordinator.PolicyDefinition;
@@ -36,6 +29,8 @@ import org.joda.time.Period;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import java.util.*;
+
 /**
  * Since 6/28/16.
  * No Data Policy engine
@@ -190,7 +185,7 @@ public class NoDataPolicyHandler implements PolicyStreamHandler{
         event.setTimestamp(timestamp);
         event.setData(triggerEvent);
         event.setStreamId(policyDef.getOutputStreams().get(0));
-        event.setPolicy(context.getPolicyDefinition());
+        event.setPolicyId(context.getPolicyDefinition().getName());
         if (this.context.getPolicyEvaluator() != null) {
             event.setCreatedBy(context.getPolicyEvaluator().getName());
         }

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/3324279e/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/main/java/org/apache/eagle/alert/engine/evaluator/nodata/NoDataPolicyTimeBatchHandler.java
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/main/java/org/apache/eagle/alert/engine/evaluator/nodata/NoDataPolicyTimeBatchHandler.java b/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/main/java/org/apache/eagle/alert/engine/evaluator/nodata/NoDataPolicyTimeBatchHandler.java
index 741fce4..53a7af0 100644
--- a/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/main/java/org/apache/eagle/alert/engine/evaluator/nodata/NoDataPolicyTimeBatchHandler.java
+++ b/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/main/java/org/apache/eagle/alert/engine/evaluator/nodata/NoDataPolicyTimeBatchHandler.java
@@ -16,12 +16,6 @@
  */
 package org.apache.eagle.alert.engine.evaluator.nodata;
 
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-
 import org.apache.commons.collections.CollectionUtils;
 import org.apache.eagle.alert.engine.Collector;
 import org.apache.eagle.alert.engine.coordinator.PolicyDefinition;
@@ -36,6 +30,8 @@ import org.joda.time.Period;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import java.util.*;
+
 public class NoDataPolicyTimeBatchHandler implements PolicyStreamHandler {
 	
 	private static final Logger LOG = LoggerFactory.getLogger(NoDataPolicyTimeBatchHandler.class);
@@ -151,7 +147,7 @@ public class NoDataPolicyTimeBatchHandler implements PolicyStreamHandler {
 		event.setTimestamp(timestamp);
 		event.setData(triggerEvent);
 		event.setStreamId(policyDef.getOutputStreams().get(0));
-		event.setPolicy(context.getPolicyDefinition());
+		event.setPolicyId(context.getPolicyDefinition().getName());
 		if (this.context.getPolicyEvaluator() != null) {
 			event.setCreatedBy(context.getPolicyEvaluator().getName());
 		}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/3324279e/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/main/java/org/apache/eagle/alert/engine/utils/AlertStreamUtils.java
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/main/java/org/apache/eagle/alert/engine/utils/AlertStreamUtils.java b/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/main/java/org/apache/eagle/alert/engine/utils/AlertStreamUtils.java
index 0eaf065..fc9cc8a 100644
--- a/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/main/java/org/apache/eagle/alert/engine/utils/AlertStreamUtils.java
+++ b/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/main/java/org/apache/eagle/alert/engine/utils/AlertStreamUtils.java
@@ -41,7 +41,7 @@ public class AlertStreamUtils {
         alertStreamEvent.setTimestamp(event.getTimestamp());
         alertStreamEvent.setData(event.getData());
         alertStreamEvent.setStreamId(policyDef.getOutputStreams().get(0));
-        alertStreamEvent.setPolicy(policyDef);
+        alertStreamEvent.setPolicyId(policyDef.getName());
 
         if (context.getPolicyEvaluator() != null) {
             alertStreamEvent.setCreatedBy(context.getPolicyEvaluator().getName());

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/3324279e/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/test/java/org/apache/eagle/alert/engine/router/TestAlertPublisherBolt.java
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/test/java/org/apache/eagle/alert/engine/router/TestAlertPublisherBolt.java b/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/test/java/org/apache/eagle/alert/engine/router/TestAlertPublisherBolt.java
index 1854c41..1e52036 100644
--- a/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/test/java/org/apache/eagle/alert/engine/router/TestAlertPublisherBolt.java
+++ b/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/test/java/org/apache/eagle/alert/engine/router/TestAlertPublisherBolt.java
@@ -70,7 +70,7 @@ public class TestAlertPublisherBolt {
         AlertStreamEvent alert = new AlertStreamEvent();
         PolicyDefinition policy = new PolicyDefinition();
         policy.setName("policy1");
-        alert.setPolicy(policy);
+        alert.setPolicyId(policy.getName());
         alert.setCreatedTime(System.currentTimeMillis());
         alert.setData(new Object[]{"field_1", 2, "field_3"});
         alert.setStreamId(streamId);
@@ -169,7 +169,7 @@ public class TestAlertPublisherBolt {
         AlertStreamEvent alert = new AlertStreamEvent();
         PolicyDefinition policy = new PolicyDefinition();
         policy.setName("perfmon_cpu_host_check");
-        alert.setPolicy(policy);
+        alert.setPolicyId(policy.getName());
         alert.setCreatedTime(System.currentTimeMillis());
         alert.setData(new Object[]{appName, hostname});
         alert.setStreamId("testAlertStream");


[38/52] [abbrv] incubator-eagle git commit: Update spark history job feeder config & refactor the code

Posted by yo...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/3110c72e/eagle-jpm/eagle-jpm-spark-running/src/main/java/org/apache/eagle/jpm/spark/running/entities/SparkAppEntity.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-spark-running/src/main/java/org/apache/eagle/jpm/spark/running/entities/SparkAppEntity.java b/eagle-jpm/eagle-jpm-spark-running/src/main/java/org/apache/eagle/jpm/spark/running/entities/SparkAppEntity.java
deleted file mode 100644
index 7de1530..0000000
--- a/eagle-jpm/eagle-jpm-spark-running/src/main/java/org/apache/eagle/jpm/spark/running/entities/SparkAppEntity.java
+++ /dev/null
@@ -1,475 +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.eagle.jpm.spark.running.entities;
-
-import org.apache.eagle.jpm.util.Constants;
-import org.apache.eagle.jpm.util.resourcefetch.model.AppInfo;
-import org.apache.eagle.log.base.taggedlog.TaggedLogAPIEntity;
-import org.apache.eagle.log.entity.meta.*;
-import org.codehaus.jackson.annotate.JsonIgnoreProperties;
-
-@Table("eagleSparkRunningApps")
-@ColumnFamily("f")
-@Prefix("sparkApp")
-@Service(Constants.RUNNING_SPARK_APP_SERVICE_ENDPOINT_NAME)
-@JsonIgnoreProperties(ignoreUnknown = true)
-@TimeSeries(true)
-@Tags({"site","sparkAppId", "sparkAppAttemptId", "sparkAppName", "user", "queue"})
-@Partition({"site"})
-public class SparkAppEntity extends TaggedLogAPIEntity {
-    @Column("a")
-    private long  startTime;
-    @Column("b")
-    private long endTime;
-    @Column("c")
-    private String yarnState;
-    @Column("d")
-    private String yarnStatus;
-    @Column("e")
-    private JobConfig config;
-    @Column("f")
-    private int numJobs;
-    @Column("g")
-    private int totalStages;
-    @Column("h")
-    private int skippedStages;
-    @Column("i")
-    private int failedStages;
-    @Column("j")
-    private int totalTasks;
-    @Column("k")
-    private int skippedTasks;
-    @Column("l")
-    private int failedTasks;
-    @Column("m")
-    private int executors;
-    @Column("n")
-    private long inputBytes;
-    @Column("o")
-    private long inputRecords;
-    @Column("p")
-    private long outputBytes;
-    @Column("q")
-    private long outputRecords;
-    @Column("r")
-    private long shuffleReadBytes;
-    @Column("s")
-    private long shuffleReadRecords;
-    @Column("t")
-    private long shuffleWriteBytes;
-    @Column("u")
-    private long shuffleWriteRecords;
-    @Column("v")
-    private long executorDeserializeTime;
-    @Column("w")
-    private long executorRunTime;
-    @Column("x")
-    private long resultSize;
-    @Column("y")
-    private long jvmGcTime;
-    @Column("z")
-    private long resultSerializationTime;
-    @Column("ab")
-    private long memoryBytesSpilled;
-    @Column("ac")
-    private long diskBytesSpilled;
-    @Column("ad")
-    private long execMemoryBytes;
-    @Column("ae")
-    private long driveMemoryBytes;
-    @Column("af")
-    private int completeTasks;
-    @Column("ag")
-    private long totalExecutorTime;
-    @Column("ah")
-    private long executorMemoryOverhead;
-    @Column("ai")
-    private long driverMemoryOverhead;
-    @Column("aj")
-    private int executorCores;
-    @Column("ak")
-    private int driverCores;
-    @Column("al")
-    private AppInfo appInfo;
-    @Column("am")
-    private int activeStages;
-    @Column("an")
-    private int completeStages;
-    @Column("ba")
-    private int activeTasks;
-
-    public int getActiveTasks() {
-        return activeTasks;
-    }
-
-    public void setActiveTasks(int activeTasks) {
-        this.activeTasks = activeTasks;
-        valueChanged("activeTasks");
-    }
-
-    public int getCompleteStages() {
-        return completeStages;
-    }
-
-    public void setCompleteStages(int completeStages) {
-        this.completeStages = completeStages;
-        valueChanged("completeStages");
-    }
-
-    public int getActiveStages() {
-        return activeStages;
-    }
-
-    public void setActiveStages(int activeStages) {
-        this.activeStages = activeStages;
-        valueChanged("activeStages");
-    }
-
-    public AppInfo getAppInfo() {
-        return appInfo;
-    }
-
-    public void setAppInfo(AppInfo appInfo) {
-        this.appInfo = appInfo;
-        valueChanged("appInfo");
-    }
-
-    public long getStartTime() {
-        return startTime;
-    }
-
-    public long getEndTime() {
-        return endTime;
-    }
-
-    public String getYarnState() {
-        return yarnState;
-    }
-
-    public String getYarnStatus() {
-        return yarnStatus;
-    }
-
-    public int getNumJobs() {
-        return numJobs;
-    }
-
-    public int getTotalStages() {
-        return totalStages;
-    }
-
-    public int getSkippedStages() {
-        return skippedStages;
-    }
-
-    public int getFailedStages() {
-        return failedStages;
-    }
-
-    public int getTotalTasks() {
-        return totalTasks;
-    }
-
-    public int getSkippedTasks() {
-        return skippedTasks;
-    }
-
-    public int getFailedTasks() {
-        return failedTasks;
-    }
-
-    public int getExecutors() {
-        return executors;
-    }
-
-    public long getInputBytes() {
-        return inputBytes;
-    }
-
-    public long getInputRecords() {
-        return inputRecords;
-    }
-
-    public long getOutputBytes() {
-        return outputBytes;
-    }
-
-    public long getOutputRecords() {
-        return outputRecords;
-    }
-
-    public long getShuffleReadBytes() {
-        return shuffleReadBytes;
-    }
-
-    public long getShuffleReadRecords() {
-        return shuffleReadRecords;
-    }
-
-    public long getShuffleWriteBytes() {
-        return shuffleWriteBytes;
-    }
-
-    public long getShuffleWriteRecords() {
-        return shuffleWriteRecords;
-    }
-
-    public long getExecutorDeserializeTime() {
-        return executorDeserializeTime;
-    }
-
-    public long getExecutorRunTime() {
-        return executorRunTime;
-    }
-
-    public long getResultSize() {
-        return resultSize;
-    }
-
-    public long getJvmGcTime() {
-        return jvmGcTime;
-    }
-
-    public long getResultSerializationTime() {
-        return resultSerializationTime;
-    }
-
-    public long getMemoryBytesSpilled() {
-        return memoryBytesSpilled;
-    }
-
-    public long getDiskBytesSpilled() {
-        return diskBytesSpilled;
-    }
-
-    public long getExecMemoryBytes() {
-        return execMemoryBytes;
-    }
-
-    public long getDriveMemoryBytes() {
-        return driveMemoryBytes;
-    }
-
-    public int getCompleteTasks() {
-        return completeTasks;
-    }
-
-    public JobConfig getConfig() {
-        return config;
-    }
-
-    public void setStartTime(long startTime) {
-        this.startTime = startTime;
-        valueChanged("startTime");
-    }
-
-    public void setEndTime(long endTime) {
-        this.endTime = endTime;
-        valueChanged("endTime");
-    }
-
-    public void setYarnState(String yarnState) {
-        this.yarnState = yarnState;
-        valueChanged("yarnState");
-    }
-
-    public void setYarnStatus(String yarnStatus) {
-        this.yarnStatus = yarnStatus;
-        valueChanged("yarnStatus");
-    }
-
-    public void setConfig(JobConfig config) {
-        this.config = config;
-        valueChanged("config");
-    }
-
-    public void setNumJobs(int numJobs) {
-        this.numJobs = numJobs;
-        valueChanged("numJobs");
-    }
-
-    public void setTotalStages(int totalStages) {
-        this.totalStages = totalStages;
-        valueChanged("totalStages");
-    }
-
-    public void setSkippedStages(int skippedStages) {
-        this.skippedStages = skippedStages;
-        valueChanged("skippedStages");
-    }
-
-    public void setFailedStages(int failedStages) {
-        this.failedStages = failedStages;
-        valueChanged("failedStages");
-    }
-
-    public void setTotalTasks(int totalTasks) {
-        this.totalTasks = totalTasks;
-        valueChanged("totalTasks");
-    }
-
-    public void setSkippedTasks(int skippedTasks) {
-        this.skippedTasks = skippedTasks;
-        valueChanged("skippedTasks");
-    }
-
-    public void setFailedTasks(int failedTasks) {
-        this.failedTasks = failedTasks;
-        valueChanged("failedTasks");
-    }
-
-    public void setExecutors(int executors) {
-        this.executors = executors;
-        valueChanged("executors");
-    }
-
-    public void setInputBytes(long inputBytes) {
-        this.inputBytes = inputBytes;
-        valueChanged("inputBytes");
-    }
-
-    public void setInputRecords(long inputRecords) {
-        this.inputRecords = inputRecords;
-        valueChanged("inputRecords");
-    }
-
-    public void setOutputBytes(long outputBytes) {
-        this.outputBytes = outputBytes;
-        valueChanged("outputBytes");
-    }
-
-    public void setOutputRecords(long outputRecords) {
-        this.outputRecords = outputRecords;
-        valueChanged("outputRecords");
-    }
-
-    public void setShuffleReadBytes(long shuffleReadRemoteBytes) {
-        this.shuffleReadBytes = shuffleReadRemoteBytes;
-        valueChanged("shuffleReadBytes");
-    }
-
-    public void setShuffleReadRecords(long shuffleReadRecords) {
-        this.shuffleReadRecords = shuffleReadRecords;
-        valueChanged("shuffleReadRecords");
-    }
-
-    public void setShuffleWriteBytes(long shuffleWriteBytes) {
-        this.shuffleWriteBytes = shuffleWriteBytes;
-        valueChanged("shuffleWriteBytes");
-    }
-
-    public void setShuffleWriteRecords(long shuffleWriteRecords) {
-        this.shuffleWriteRecords = shuffleWriteRecords;
-        valueChanged("shuffleWriteRecords");
-    }
-
-    public void setExecutorDeserializeTime(long executorDeserializeTime) {
-        this.executorDeserializeTime = executorDeserializeTime;
-        valueChanged("executorDeserializeTime");
-    }
-
-    public void setExecutorRunTime(long executorRunTime) {
-        this.executorRunTime = executorRunTime;
-        valueChanged("executorRunTime");
-    }
-
-    public void setResultSize(long resultSize) {
-        this.resultSize = resultSize;
-        valueChanged("resultSize");
-    }
-
-    public void setJvmGcTime(long jvmGcTime) {
-        this.jvmGcTime = jvmGcTime;
-        valueChanged("jvmGcTime");
-    }
-
-    public void setResultSerializationTime(long resultSerializationTime) {
-        this.resultSerializationTime = resultSerializationTime;
-        valueChanged("resultSerializationTime");
-    }
-
-    public void setMemoryBytesSpilled(long memoryBytesSpilled) {
-        this.memoryBytesSpilled = memoryBytesSpilled;
-        valueChanged("memoryBytesSpilled");
-    }
-
-    public void setDiskBytesSpilled(long diskBytesSpilled) {
-        this.diskBytesSpilled = diskBytesSpilled;
-        valueChanged("diskBytesSpilled");
-    }
-
-    public void setExecMemoryBytes(long execMemoryBytes) {
-        this.execMemoryBytes = execMemoryBytes;
-        valueChanged("execMemoryBytes");
-    }
-
-    public void setDriveMemoryBytes(long driveMemoryBytes) {
-        this.driveMemoryBytes = driveMemoryBytes;
-        valueChanged("driveMemoryBytes");
-    }
-
-    public void setCompleteTasks(int completeTasks) {
-        this.completeTasks = completeTasks;
-        valueChanged("completeTasks");
-    }
-
-    public long getTotalExecutorTime() {
-        return totalExecutorTime;
-    }
-
-    public void setTotalExecutorTime(long totalExecutorTime) {
-        this.totalExecutorTime = totalExecutorTime;
-        valueChanged("totalExecutorTime");
-    }
-
-    public long getExecutorMemoryOverhead() {
-        return executorMemoryOverhead;
-    }
-
-    public void setExecutorMemoryOverhead(long executorMemoryOverhead) {
-        this.executorMemoryOverhead = executorMemoryOverhead;
-        valueChanged("executorMemoryOverhead");
-    }
-
-    public long getDriverMemoryOverhead() {
-        return driverMemoryOverhead;
-    }
-
-    public void setDriverMemoryOverhead(long driverMemoryOverhead) {
-        this.driverMemoryOverhead = driverMemoryOverhead;
-        valueChanged("driverMemoryOverhead");
-    }
-
-    public int getExecutorCores() {
-        return executorCores;
-    }
-
-    public void setExecutorCores(int executorCores) {
-        this.executorCores = executorCores;
-        valueChanged("executorCores");
-    }
-
-    public int getDriverCores() {
-        return driverCores;
-    }
-
-    public void setDriverCores(int driverCores) {
-        this.driverCores = driverCores;
-        valueChanged("driverCores");
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/3110c72e/eagle-jpm/eagle-jpm-spark-running/src/main/java/org/apache/eagle/jpm/spark/running/entities/SparkExecutorEntity.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-spark-running/src/main/java/org/apache/eagle/jpm/spark/running/entities/SparkExecutorEntity.java b/eagle-jpm/eagle-jpm-spark-running/src/main/java/org/apache/eagle/jpm/spark/running/entities/SparkExecutorEntity.java
deleted file mode 100644
index 89549ca..0000000
--- a/eagle-jpm/eagle-jpm-spark-running/src/main/java/org/apache/eagle/jpm/spark/running/entities/SparkExecutorEntity.java
+++ /dev/null
@@ -1,232 +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.eagle.jpm.spark.running.entities;
-
-import org.apache.eagle.jpm.util.Constants;
-import org.apache.eagle.log.base.taggedlog.TaggedLogAPIEntity;
-import org.apache.eagle.log.entity.meta.*;
-import org.codehaus.jackson.annotate.JsonIgnoreProperties;
-
-@Table("eagleSparkRunningExecutors")
-@ColumnFamily("f")
-@Prefix("sparkExecutor")
-@Service(Constants.RUNNING_SPARK_EXECUTOR_SERVICE_ENDPOINT_NAME)
-@JsonIgnoreProperties(ignoreUnknown = true)
-@TimeSeries(true)
-@Tags({"site","sparkAppId", "sparkAppAttemptId", "sparkAppName", "executorId","user", "queue"})
-@Partition({"site"})
-public class SparkExecutorEntity extends TaggedLogAPIEntity {
-    @Column("a")
-    private String hostPort;
-    @Column("b")
-    private int rddBlocks;
-    @Column("c")
-    private long memoryUsed;
-    @Column("d")
-    private long diskUsed;
-    @Column("e")
-    private int activeTasks = 0;
-    @Column("f")
-    private int failedTasks = 0;
-    @Column("g")
-    private int completedTasks = 0;
-    @Column("h")
-    private int totalTasks = 0;
-    @Column("i")
-    private long totalDuration = 0;
-    @Column("j")
-    private long totalInputBytes = 0;
-    @Column("k")
-    private long totalShuffleRead = 0;
-    @Column("l")
-    private long totalShuffleWrite = 0;
-    @Column("m")
-    private long maxMemory;
-    @Column("n")
-    private long startTime;
-    @Column("o")
-    private long endTime = 0;
-    @Column("p")
-    private long execMemoryBytes;
-    @Column("q")
-    private int cores;
-    @Column("r")
-    private long memoryOverhead;
-
-    public String getHostPort() {
-        return hostPort;
-    }
-
-    public void setHostPort(String hostPort) {
-        this.hostPort = hostPort;
-        this.valueChanged("hostPort");
-    }
-
-    public int getRddBlocks() {
-        return rddBlocks;
-    }
-
-    public void setRddBlocks(int rddBlocks) {
-        this.rddBlocks = rddBlocks;
-        this.valueChanged("rddBlocks");
-    }
-
-    public long getMemoryUsed() {
-        return memoryUsed;
-    }
-
-    public void setMemoryUsed(long memoryUsed) {
-        this.memoryUsed = memoryUsed;
-        this.valueChanged("memoryUsed");
-    }
-
-    public long getDiskUsed() {
-        return diskUsed;
-    }
-
-    public void setDiskUsed(long diskUsed) {
-        this.diskUsed = diskUsed;
-        this.valueChanged("diskUsed");
-    }
-
-    public int getActiveTasks() {
-        return activeTasks;
-    }
-
-    public void setActiveTasks(int activeTasks) {
-        this.activeTasks = activeTasks;
-        this.valueChanged("activeTasks");
-    }
-
-    public int getFailedTasks() {
-        return failedTasks;
-    }
-
-    public void setFailedTasks(int failedTasks) {
-        this.failedTasks = failedTasks;
-        this.valueChanged("failedTasks");
-    }
-
-    public int getCompletedTasks() {
-        return completedTasks;
-    }
-
-    public void setCompletedTasks(int completedTasks) {
-        this.completedTasks = completedTasks;
-        this.valueChanged("completedTasks");
-    }
-
-    public int getTotalTasks() {
-        return totalTasks;
-    }
-
-    public void setTotalTasks(int totalTasks) {
-        this.totalTasks = totalTasks;
-        this.valueChanged("totalTasks");
-    }
-
-    public long getTotalDuration() {
-        return totalDuration;
-    }
-
-    public void setTotalDuration(long totalDuration) {
-        this.totalDuration = totalDuration;
-        this.valueChanged("totalDuration");
-    }
-
-    public long getTotalInputBytes() {
-        return totalInputBytes;
-    }
-
-    public void setTotalInputBytes(long totalInputBytes) {
-        this.totalInputBytes = totalInputBytes;
-        this.valueChanged("totalInputBytes");
-    }
-
-    public long getTotalShuffleRead() {
-        return totalShuffleRead;
-    }
-
-    public void setTotalShuffleRead(long totalShuffleRead) {
-        this.totalShuffleRead = totalShuffleRead;
-        this.valueChanged("totalShuffleRead");
-    }
-
-    public long getTotalShuffleWrite() {
-        return totalShuffleWrite;
-    }
-
-    public void setTotalShuffleWrite(long totalShuffleWrite) {
-        this.totalShuffleWrite = totalShuffleWrite;
-        this.valueChanged("totalShuffleWrite");
-    }
-
-    public long getMaxMemory() {
-        return maxMemory;
-    }
-
-    public void setMaxMemory(long maxMemory) {
-        this.maxMemory = maxMemory;
-        this.valueChanged("maxMemory");
-    }
-
-    public long getStartTime() {
-        return startTime;
-    }
-
-    public void setStartTime(long startTime) {
-        this.startTime = startTime;
-        valueChanged("startTime");
-    }
-
-    public long getEndTime() {
-        return endTime;
-    }
-
-    public void setEndTime(long endTime) {
-        this.endTime = endTime;
-        this.valueChanged("endTime");
-    }
-
-    public long getExecMemoryBytes() {
-        return execMemoryBytes;
-    }
-
-    public void setExecMemoryBytes(long execMemoryBytes) {
-        this.execMemoryBytes = execMemoryBytes;
-        this.valueChanged("execMemoryBytes");
-    }
-
-    public int getCores() {
-        return cores;
-    }
-
-    public void setCores(int cores) {
-        this.cores = cores;
-        valueChanged("cores");
-    }
-
-    public long getMemoryOverhead() {
-        return memoryOverhead;
-    }
-
-    public void setMemoryOverhead(long memoryOverhead) {
-        this.memoryOverhead = memoryOverhead;
-        valueChanged("memoryOverhead");
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/3110c72e/eagle-jpm/eagle-jpm-spark-running/src/main/java/org/apache/eagle/jpm/spark/running/entities/SparkJobEntity.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-spark-running/src/main/java/org/apache/eagle/jpm/spark/running/entities/SparkJobEntity.java b/eagle-jpm/eagle-jpm-spark-running/src/main/java/org/apache/eagle/jpm/spark/running/entities/SparkJobEntity.java
deleted file mode 100644
index bb56b52..0000000
--- a/eagle-jpm/eagle-jpm-spark-running/src/main/java/org/apache/eagle/jpm/spark/running/entities/SparkJobEntity.java
+++ /dev/null
@@ -1,190 +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.eagle.jpm.spark.running.entities;
-
-import org.apache.eagle.jpm.util.Constants;
-import org.apache.eagle.log.base.taggedlog.TaggedLogAPIEntity;
-import org.apache.eagle.log.entity.meta.*;
-import org.codehaus.jackson.annotate.JsonIgnoreProperties;
-
-import java.util.List;
-
-@Table("eagleSparkRunningJobs")
-@ColumnFamily("f")
-@Prefix("sparkJob")
-@Service(Constants.RUNNING_SPARK_JOB_SERVICE_ENDPOINT_NAME)
-@JsonIgnoreProperties(ignoreUnknown = true)
-@TimeSeries(true)
-@Tags({"site","sparkAppId", "sparkAppAttemptId", "sparkAppName", "jobId","user", "queue"})
-@Partition({"site"})
-public class SparkJobEntity extends TaggedLogAPIEntity {
-    @Column("a")
-    private long  submissionTime;
-    @Column("b")
-    private long completionTime;
-    @Column("c")
-    private int numStages = 0;
-    @Column("d")
-    private String status;
-    @Column("e")
-    private int numTask = 0;
-    @Column("f")
-    private int numActiveTasks = 0;
-    @Column("g")
-    private int numCompletedTasks = 0;
-    @Column("h")
-    private int numSkippedTasks = 0;
-    @Column("i")
-    private int numFailedTasks = 0;
-    @Column("j")
-    private int numActiveStages = 0;
-    @Column("k")
-    private int numCompletedStages = 0;
-    @Column("l")
-    private int numSkippedStages = 0;
-    @Column("m")
-    private int numFailedStages = 0;
-    @Column("n")
-    private List<Integer> stages;
-
-    public List<Integer> getStages() {
-        return stages;
-    }
-
-    public void setStages(List<Integer> stages) {
-        this.stages = stages;
-        this.valueChanged("stages");
-    }
-
-    public long getSubmissionTime() {
-        return submissionTime;
-    }
-
-    public long getCompletionTime() {
-        return completionTime;
-    }
-
-    public int getNumStages() {
-        return numStages;
-    }
-
-    public String getStatus() {
-        return status;
-    }
-
-    public int getNumTask() {
-        return numTask;
-    }
-
-    public int getNumActiveTasks() {
-        return numActiveTasks;
-    }
-
-    public int getNumCompletedTasks() {
-        return numCompletedTasks;
-    }
-
-    public int getNumSkippedTasks() {
-        return numSkippedTasks;
-    }
-
-    public int getNumFailedTasks() {
-        return numFailedTasks;
-    }
-
-    public int getNumActiveStages() {
-        return numActiveStages;
-    }
-
-    public int getNumCompletedStages() {
-        return numCompletedStages;
-    }
-
-    public int getNumSkippedStages() {
-        return numSkippedStages;
-    }
-
-    public int getNumFailedStages() {
-        return numFailedStages;
-    }
-
-    public void setSubmissionTime(long submissionTime) {
-        this.submissionTime = submissionTime;
-        this.valueChanged("submissionTime");
-    }
-
-    public void setCompletionTime(long completionTime) {
-        this.completionTime = completionTime;
-        this.valueChanged("completionTime");
-    }
-
-    public void setNumStages(int numStages) {
-        this.numStages = numStages;
-        this.valueChanged("numStages");
-    }
-
-    public void setStatus(String status) {
-        this.status = status;
-        this.valueChanged("status");
-    }
-
-    public void setNumTask(int numTask) {
-        this.numTask = numTask;
-        this.valueChanged("numTask");
-    }
-
-    public void setNumActiveTasks(int numActiveTasks) {
-        this.numActiveTasks = numActiveTasks;
-        this.valueChanged("numActiveTasks");
-    }
-
-    public void setNumCompletedTasks(int numCompletedTasks) {
-        this.numCompletedTasks = numCompletedTasks;
-        this.valueChanged("numCompletedTasks");
-    }
-
-    public void setNumSkippedTasks(int numSkippedTasks) {
-        this.numSkippedTasks = numSkippedTasks;
-        this.valueChanged("numSkippedTasks");
-    }
-
-    public void setNumFailedTasks(int numFailedTasks) {
-        this.numFailedTasks = numFailedTasks;
-        this.valueChanged("numFailedTasks");
-    }
-
-    public void setNumActiveStages(int numActiveStages) {
-        this.numActiveStages = numActiveStages;
-        this.valueChanged("numActiveStages");
-    }
-
-    public void setNumCompletedStages(int numCompletedStages) {
-        this.numCompletedStages = numCompletedStages;
-        this.valueChanged("numCompletedStages");
-    }
-
-    public void setNumSkippedStages(int numSkippedStages) {
-        this.numSkippedStages = numSkippedStages;
-        this.valueChanged("numSkippedStages");
-    }
-
-    public void setNumFailedStages(int numFailedStages) {
-        this.numFailedStages = numFailedStages;
-        this.valueChanged("numFailedStages");
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/3110c72e/eagle-jpm/eagle-jpm-spark-running/src/main/java/org/apache/eagle/jpm/spark/running/entities/SparkStageEntity.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-spark-running/src/main/java/org/apache/eagle/jpm/spark/running/entities/SparkStageEntity.java b/eagle-jpm/eagle-jpm-spark-running/src/main/java/org/apache/eagle/jpm/spark/running/entities/SparkStageEntity.java
deleted file mode 100644
index be0ffd0..0000000
--- a/eagle-jpm/eagle-jpm-spark-running/src/main/java/org/apache/eagle/jpm/spark/running/entities/SparkStageEntity.java
+++ /dev/null
@@ -1,298 +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.eagle.jpm.spark.running.entities;
-
-import org.apache.eagle.jpm.util.Constants;
-import org.apache.eagle.log.base.taggedlog.TaggedLogAPIEntity;
-import org.apache.eagle.log.entity.meta.*;
-import org.codehaus.jackson.annotate.JsonIgnoreProperties;
-
-@Table("eagleSparkRunningStages")
-@ColumnFamily("f")
-@Prefix("sparkStage")
-@Service(Constants.RUNNING_SPARK_STAGE_SERVICE_ENDPOINT_NAME)
-@JsonIgnoreProperties(ignoreUnknown = true)
-@TimeSeries(true)
-@Tags({"site","sparkAppId", "sparkAppAttemptId", "sparkAppName", "jobId", "stageId","stageAttemptId","user", "queue"})
-@Partition({"site"})
-public class SparkStageEntity extends TaggedLogAPIEntity {
-    @Column("a")
-    private String status;
-    @Column("b")
-    private int numActiveTasks = 0;
-    @Column("c")
-    private int numCompletedTasks = 0;
-    @Column("d")
-    private int numFailedTasks = 0;
-    @Column("e")
-    private long executorRunTime = 0L;
-    @Column("f")
-    private long inputBytes = 0L;
-    @Column("g")
-    private long inputRecords = 0L;
-    @Column("h")
-    private long outputBytes = 0L;
-    @Column("i")
-    private long outputRecords = 0L;
-    @Column("j")
-    private long shuffleReadBytes = 0L;
-    @Column("k")
-    private long shuffleReadRecords = 0L;
-    @Column("l")
-    private long shuffleWriteBytes = 0L;
-    @Column("m")
-    private long shuffleWriteRecords = 0L;
-    @Column("n")
-    private long memoryBytesSpilled = 0L;
-    @Column("o")
-    private long diskBytesSpilled = 0L;
-    @Column("p")
-    private String name;
-    @Column("q")
-    private String schedulingPool;
-    @Column("r")
-    private long submitTime;
-    @Column("s")
-    private long completeTime;
-    @Column("t")
-    private int numTasks;
-    @Column("u")
-    private long executorDeserializeTime;
-    @Column("v")
-    private long resultSize;
-    @Column("w")
-    private long jvmGcTime;
-    @Column("x")
-    private long resultSerializationTime;
-
-    public String getStatus() {
-        return status;
-    }
-
-    public int getNumActiveTasks() {
-        return numActiveTasks;
-    }
-
-    public int getNumCompletedTasks() {
-        return numCompletedTasks;
-    }
-
-    public int getNumFailedTasks() {
-        return numFailedTasks;
-    }
-
-    public long getExecutorRunTime() {
-        return executorRunTime;
-    }
-
-    public long getInputBytes() {
-        return inputBytes;
-    }
-
-    public long getInputRecords() {
-        return inputRecords;
-    }
-
-    public long getOutputBytes() {
-        return outputBytes;
-    }
-
-    public long getOutputRecords() {
-        return outputRecords;
-    }
-
-    public long getShuffleReadBytes() {
-        return shuffleReadBytes;
-    }
-
-    public long getShuffleReadRecords() {
-        return shuffleReadRecords;
-    }
-
-    public long getShuffleWriteBytes() {
-        return shuffleWriteBytes;
-    }
-
-    public long getShuffleWriteRecords() {
-        return shuffleWriteRecords;
-    }
-
-    public long getMemoryBytesSpilled() {
-        return memoryBytesSpilled;
-    }
-
-    public long getDiskBytesSpilled() {
-        return diskBytesSpilled;
-    }
-
-    public String getName() {
-        return name;
-    }
-
-    public String getSchedulingPool() {
-        return schedulingPool;
-    }
-
-    public long getSubmitTime() {
-        return submitTime;
-    }
-
-    public long getCompleteTime() {
-        return completeTime;
-    }
-
-    public int getNumTasks() {
-        return numTasks;
-    }
-
-    public long getExecutorDeserializeTime() {
-        return executorDeserializeTime;
-    }
-
-    public long getResultSize() {
-        return resultSize;
-    }
-
-    public long getJvmGcTime() {
-        return jvmGcTime;
-    }
-
-    public long getResultSerializationTime() {
-        return resultSerializationTime;
-    }
-
-    public void setStatus(String status) {
-        this.status = status;
-        this.valueChanged("status");
-    }
-
-    public void setNumActiveTasks(int numActiveTasks) {
-        this.numActiveTasks = numActiveTasks;
-        this.valueChanged("numActiveTasks");
-    }
-
-    public void setNumCompletedTasks(int numCompletedTasks) {
-        this.numCompletedTasks = numCompletedTasks;
-        this.valueChanged("numCompletedTasks");
-    }
-
-    public void setNumFailedTasks(int numFailedTasks) {
-        this.numFailedTasks = numFailedTasks;
-        this.valueChanged("numFailedTasks");
-    }
-
-    public void setExecutorRunTime(long executorRunTime) {
-        this.executorRunTime = executorRunTime;
-        this.valueChanged("executorRunTime");
-    }
-
-    public void setInputBytes(long inputBytes) {
-        this.inputBytes = inputBytes;
-        this.valueChanged("inputBytes");
-    }
-
-    public void setInputRecords(long inputRecords) {
-        this.inputRecords = inputRecords;
-        this.valueChanged("inputRecords");
-    }
-
-    public void setOutputBytes(long outputBytes) {
-        this.outputBytes = outputBytes;
-        this.valueChanged("outputBytes");
-    }
-
-    public void setOutputRecords(long outputRecords) {
-        this.outputRecords = outputRecords;
-        this.valueChanged("outputRecords");
-    }
-
-    public void setShuffleReadBytes(long shuffleReadBytes) {
-        this.shuffleReadBytes = shuffleReadBytes;
-        this.valueChanged("shuffleReadBytes");
-    }
-
-    public void setShuffleReadRecords(long shuffleReadRecords) {
-        this.shuffleReadRecords = shuffleReadRecords;
-        this.valueChanged("shuffleReadRecords");
-    }
-
-    public void setShuffleWriteBytes(long shuffleWriteBytes) {
-        this.shuffleWriteBytes = shuffleWriteBytes;
-        this.valueChanged("shuffleWriteBytes");
-    }
-
-    public void setShuffleWriteRecords(long shuffleWriteRecords) {
-        this.shuffleWriteRecords = shuffleWriteRecords;
-        this.valueChanged("shuffleWriteRecords");
-    }
-
-    public void setMemoryBytesSpilled(long memoryBytesSpilled) {
-        this.memoryBytesSpilled = memoryBytesSpilled;
-        this.valueChanged("memoryBytesSpilled");
-    }
-
-    public void setDiskBytesSpilled(long diskBytesSpilled) {
-        this.diskBytesSpilled = diskBytesSpilled;
-        this.valueChanged("diskBytesSpilled");
-    }
-
-    public void setName(String name) {
-        this.name = name;
-        this.valueChanged("name");
-    }
-
-    public void setSchedulingPool(String schedulingPool) {
-        this.schedulingPool = schedulingPool;
-        this.valueChanged("schedulingPool");
-    }
-
-    public void setSubmitTime(long submitTime) {
-        this.submitTime = submitTime;
-        this.valueChanged("submitTime");
-    }
-
-    public void setCompleteTime(long completeTime) {
-        this.completeTime = completeTime;
-        this.valueChanged("completeTime");
-    }
-
-    public void setNumTasks(int numTasks) {
-        this.numTasks = numTasks;
-        valueChanged("numTasks");
-    }
-
-    public void setExecutorDeserializeTime(long executorDeserializeTime) {
-        this.executorDeserializeTime = executorDeserializeTime;
-        valueChanged("executorDeserializeTime");
-    }
-
-    public void setResultSize(long resultSize) {
-        this.resultSize = resultSize;
-        valueChanged("resultSize");
-    }
-
-    public void setJvmGcTime(long jvmGcTime) {
-        this.jvmGcTime = jvmGcTime;
-        valueChanged("jvmGcTime");
-    }
-
-    public void setResultSerializationTime(long resultSerializationTime) {
-        this.resultSerializationTime = resultSerializationTime;
-        valueChanged("resultSerializationTime");
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/3110c72e/eagle-jpm/eagle-jpm-spark-running/src/main/java/org/apache/eagle/jpm/spark/running/entities/SparkTaskEntity.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-spark-running/src/main/java/org/apache/eagle/jpm/spark/running/entities/SparkTaskEntity.java b/eagle-jpm/eagle-jpm-spark-running/src/main/java/org/apache/eagle/jpm/spark/running/entities/SparkTaskEntity.java
deleted file mode 100644
index e531806..0000000
--- a/eagle-jpm/eagle-jpm-spark-running/src/main/java/org/apache/eagle/jpm/spark/running/entities/SparkTaskEntity.java
+++ /dev/null
@@ -1,289 +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.eagle.jpm.spark.running.entities;
-
-import org.apache.eagle.jpm.util.Constants;
-import org.apache.eagle.log.base.taggedlog.TaggedLogAPIEntity;
-import org.apache.eagle.log.entity.meta.*;
-import org.codehaus.jackson.annotate.JsonIgnoreProperties;
-
-@Table("eagleSparkRunningTasks")
-@ColumnFamily("f")
-@Prefix("sparkTask")
-@Service(Constants.RUNNING_SPARK_TASK_SERVICE_ENDPOINT_NAME)
-@JsonIgnoreProperties(ignoreUnknown = true)
-@TimeSeries(true)
-@Tags({"site","sparkAppId", "sparkAppAttemptId", "sparkAppName", "jobId", "jobName", "stageId","stageAttemptId","taskIndex","taskAttemptId","user", "queue"})
-@Partition({"site"})
-public class SparkTaskEntity extends TaggedLogAPIEntity {
-    @Column("a")
-    private int taskId;
-    @Column("b")
-    private long launchTime;
-    @Column("c")
-    private String executorId;
-    @Column("d")
-    private String host;
-    @Column("e")
-    private String taskLocality;
-    @Column("f")
-    private boolean speculative;
-    @Column("g")
-    private long executorDeserializeTime;
-    @Column("h")
-    private long executorRunTime;
-    @Column("i")
-    private long resultSize;
-    @Column("j")
-    private long jvmGcTime;
-    @Column("k")
-    private long resultSerializationTime;
-    @Column("l")
-    private long memoryBytesSpilled;
-    @Column("m")
-    private long diskBytesSpilled;
-    @Column("n")
-    private long inputBytes;
-    @Column("o")
-    private long inputRecords;
-    @Column("p")
-    private long outputBytes;
-    @Column("q")
-    private long outputRecords;
-    @Column("r")
-    private long shuffleReadRemoteBytes;
-    @Column("x")
-    private long shuffleReadLocalBytes;
-    @Column("s")
-    private long shuffleReadRecords;
-    @Column("t")
-    private long shuffleWriteBytes;
-    @Column("u")
-    private long shuffleWriteRecords;
-    @Column("v")
-    private boolean failed;
-
-    public int getTaskId() {
-        return taskId;
-    }
-
-    public long getLaunchTime() {
-        return launchTime;
-    }
-
-    public String getExecutorId() {
-        return executorId;
-    }
-
-    public String getHost() {
-        return host;
-    }
-
-    public String getTaskLocality() {
-        return taskLocality;
-    }
-
-    public boolean isSpeculative() {
-        return speculative;
-    }
-
-    public long getExecutorDeserializeTime() {
-        return executorDeserializeTime;
-    }
-
-    public long getExecutorRunTime() {
-        return executorRunTime;
-    }
-
-    public long getResultSize() {
-        return resultSize;
-    }
-
-    public long getJvmGcTime() {
-        return jvmGcTime;
-    }
-
-    public long getResultSerializationTime() {
-        return resultSerializationTime;
-    }
-
-    public long getMemoryBytesSpilled() {
-        return memoryBytesSpilled;
-    }
-
-    public long getDiskBytesSpilled() {
-        return diskBytesSpilled;
-    }
-
-    public long getInputBytes() {
-        return inputBytes;
-    }
-
-    public long getInputRecords() {
-        return inputRecords;
-    }
-
-    public long getOutputBytes() {
-        return outputBytes;
-    }
-
-    public long getOutputRecords() {
-        return outputRecords;
-    }
-
-    public long getShuffleReadRecords() {
-        return shuffleReadRecords;
-    }
-
-    public long getShuffleWriteBytes() {
-        return shuffleWriteBytes;
-    }
-
-    public long getShuffleWriteRecords() {
-        return shuffleWriteRecords;
-    }
-
-    public boolean isFailed() {
-        return failed;
-    }
-
-    public long getShuffleReadRemoteBytes() {
-        return shuffleReadRemoteBytes;
-    }
-
-    public long getShuffleReadLocalBytes() {
-        return shuffleReadLocalBytes;
-    }
-
-    public void setFailed(boolean failed) {
-        this.failed = failed;
-        valueChanged("failed");
-    }
-
-    public void setTaskId(int taskId) {
-        this.taskId = taskId;
-        valueChanged("taskId");
-    }
-
-    public void setLaunchTime(long launchTime) {
-        this.launchTime = launchTime;
-        valueChanged("launchTime");
-    }
-
-    public void setExecutorId(String executorId) {
-        this.executorId = executorId;
-        valueChanged("executorId");
-    }
-
-    public void setHost(String host) {
-        this.host = host;
-        this.valueChanged("host");
-    }
-
-    public void setTaskLocality(String taskLocality) {
-        this.taskLocality = taskLocality;
-        this.valueChanged("taskLocality");
-    }
-
-    public void setSpeculative(boolean speculative) {
-        this.speculative = speculative;
-        this.valueChanged("speculative");
-    }
-
-    public void setExecutorDeserializeTime(long executorDeserializeTime) {
-        this.executorDeserializeTime = executorDeserializeTime;
-        this.valueChanged("executorDeserializeTime");
-    }
-
-    public void setExecutorRunTime(long executorRunTime) {
-        this.executorRunTime = executorRunTime;
-        this.valueChanged("executorRunTime");
-    }
-
-    public void setResultSize(long resultSize) {
-        this.resultSize = resultSize;
-        this.valueChanged("resultSize");
-    }
-
-    public void setJvmGcTime(long jvmGcTime) {
-        this.jvmGcTime = jvmGcTime;
-        this.valueChanged("jvmGcTime");
-    }
-
-    public void setResultSerializationTime(long resultSerializationTime) {
-        this.resultSerializationTime = resultSerializationTime;
-        this.valueChanged("resultSerializationTime");
-    }
-
-    public void setMemoryBytesSpilled(long memoryBytesSpilled) {
-        this.memoryBytesSpilled = memoryBytesSpilled;
-        this.valueChanged("memoryBytesSpilled");
-    }
-
-    public void setDiskBytesSpilled(long diskBytesSpilled) {
-        this.diskBytesSpilled = diskBytesSpilled;
-        this.valueChanged("diskBytesSpilled");
-    }
-
-    public void setInputBytes(long inputBytes) {
-        this.inputBytes = inputBytes;
-        this.valueChanged("inputBytes");
-    }
-
-    public void setInputRecords(long inputRecords) {
-        this.inputRecords = inputRecords;
-        this.valueChanged("inputRecords");
-    }
-
-    public void setOutputBytes(long outputBytes) {
-        this.outputBytes = outputBytes;
-        this.valueChanged("outputBytes");
-    }
-
-    public void setOutputRecords(long outputRecords) {
-        this.outputRecords = outputRecords;
-        this.valueChanged("outputRecords");
-    }
-
-
-
-    public void setShuffleReadRecords(long shuffleReadRecords) {
-        this.shuffleReadRecords = shuffleReadRecords;
-        this.valueChanged("shuffleReadRecords");
-    }
-
-    public void setShuffleWriteBytes(long shuffleWriteBytes) {
-        this.shuffleWriteBytes = shuffleWriteBytes;
-        this.valueChanged("shuffleWriteBytes");
-    }
-
-    public void setShuffleWriteRecords(long shuffleWriteRecords) {
-        this.shuffleWriteRecords = shuffleWriteRecords;
-        this.valueChanged("shuffleWriteRecords");
-    }
-
-    public void setShuffleReadRemoteBytes(long shuffleReadRemoteBytes) {
-        this.shuffleReadRemoteBytes = shuffleReadRemoteBytes;
-        this.valueChanged("shuffleReadRemoteBytes");
-    }
-
-    public void setShuffleReadLocalBytes(long shuffleReadLocalBytes) {
-        this.shuffleReadLocalBytes = shuffleReadLocalBytes;
-        this.valueChanged("shuffleReadLocalBytes");
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/3110c72e/eagle-jpm/eagle-jpm-spark-running/src/main/java/org/apache/eagle/jpm/spark/running/parser/SparkApplicationParser.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-spark-running/src/main/java/org/apache/eagle/jpm/spark/running/parser/SparkApplicationParser.java b/eagle-jpm/eagle-jpm-spark-running/src/main/java/org/apache/eagle/jpm/spark/running/parser/SparkApplicationParser.java
index 6411018..3719325 100644
--- a/eagle-jpm/eagle-jpm-spark-running/src/main/java/org/apache/eagle/jpm/spark/running/parser/SparkApplicationParser.java
+++ b/eagle-jpm/eagle-jpm-spark-running/src/main/java/org/apache/eagle/jpm/spark/running/parser/SparkApplicationParser.java
@@ -18,14 +18,10 @@
 
 package org.apache.eagle.jpm.spark.running.parser;
 
-import org.apache.eagle.jpm.spark.crawl.EventType;
 import org.apache.eagle.jpm.spark.running.SparkRunningJobAppConfig;
 import org.apache.eagle.jpm.spark.running.entities.*;
 import org.apache.eagle.jpm.spark.running.recover.SparkRunningJobManager;
-import org.apache.eagle.jpm.util.Constants;
-import org.apache.eagle.jpm.util.HDFSUtil;
-import org.apache.eagle.jpm.util.SparkJobTagName;
-import org.apache.eagle.jpm.util.Utils;
+import org.apache.eagle.jpm.util.*;
 import org.apache.eagle.jpm.util.resourcefetch.ResourceFetcher;
 import org.apache.eagle.jpm.util.resourcefetch.connection.InputStreamUtils;
 import org.apache.eagle.jpm.util.resourcefetch.model.*;
@@ -219,7 +215,7 @@ public class SparkApplicationParser implements Runnable {
                     if (eventObj != null) {
                         String eventType = (String) eventObj.get("Event");
                         LOG.info("Event type: " + eventType);
-                        if (eventType.equalsIgnoreCase(EventType.SparkListenerEnvironmentUpdate.toString())) {
+                        if (eventType.equalsIgnoreCase(SparkEventType.SparkListenerEnvironmentUpdate.toString())) {
                             stop = true;
                             JSONObject sparkProps = (JSONObject) eventObj.get("Spark Properties");
                             for (Object key : sparkProps.keySet()) {

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/3110c72e/eagle-jpm/eagle-jpm-spark-running/src/main/resources/application.conf
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-spark-running/src/main/resources/application.conf b/eagle-jpm/eagle-jpm-spark-running/src/main/resources/application.conf
index 9d9f622..4d07b38 100644
--- a/eagle-jpm/eagle-jpm-spark-running/src/main/resources/application.conf
+++ b/eagle-jpm/eagle-jpm-spark-running/src/main/resources/application.conf
@@ -14,6 +14,9 @@
 # limitations under the License.
 
 {
+  "appId":"sparkRunningJob",
+  "mode":"LOCAL",
+  "workers" : 3,
   "envContextConfig" : {
     "stormConfigFile" : "storm.yaml",
     "parallelismConfig" : {
@@ -24,7 +27,6 @@
       "sparkRunningJobFetchSpout" : 1,
       "sparkRunningJobParseBolt" : 4
     },
-    "workers" : 2
   },
   "jobExtractorConfig" : {
     "site" : "sandbox",
@@ -48,8 +50,6 @@
     "zkRetryTimes" : 3,
     "zkRetryInterval" : 20000
   },
-  "appId":"sparkRunningJob",
-  "mode":"LOCAL",
   "eagleProps" : {
     "mailHost" : "abc.com",
     "mailDebug" : "true",

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/3110c72e/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/SparkEventType.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/SparkEventType.java b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/SparkEventType.java
new file mode 100644
index 0000000..7c9f625
--- /dev/null
+++ b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/SparkEventType.java
@@ -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.
+ */
+
+package org.apache.eagle.jpm.util;
+
+public enum SparkEventType {
+    SparkListenerBlockManagerAdded, SparkListenerEnvironmentUpdate, SparkListenerApplicationStart,
+    SparkListenerExecutorAdded, SparkListenerJobStart,SparkListenerStageSubmitted, SparkListenerTaskStart,SparkListenerBlockManagerRemoved,
+    SparkListenerTaskEnd, SparkListenerStageCompleted, SparkListenerJobEnd, SparkListenerApplicationEnd,SparkListenerExecutorRemoved
+}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/3110c72e/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/RMResourceFetcher.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/RMResourceFetcher.java b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/RMResourceFetcher.java
index b1881ef..61b2fee 100644
--- a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/RMResourceFetcher.java
+++ b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/RMResourceFetcher.java
@@ -107,7 +107,7 @@ public class RMResourceFetcher implements ResourceFetcher<AppInfo> {
 
     private String getMRFinishedJobURL(String lastFinishedTime) {
         String url = URLUtil.removeTrailingSlash(selector.getSelectedUrl());
-        return url + "/" + "Constants.V2_APPS_URL"
+        return url + "/" + Constants.V2_APPS_URL
                 + "?applicationTypes=MAPREDUCE&state=FINISHED&finishedTimeBegin="
                 + lastFinishedTime + "&" + Constants.ANONYMOUS_PARAMETER;
     }


[47/52] [abbrv] incubator-eagle git commit: [EAGLE-520] Fix and decouple co-processor from eagle aggreation query service

Posted by yo...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/21187b55/eagle-core/eagle-query/eagle-storage-hbase/src/main/java/org/apache/eagle/storage/hbase/query/coprocessor/impl/AggregateClientImpl.java
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-query/eagle-storage-hbase/src/main/java/org/apache/eagle/storage/hbase/query/coprocessor/impl/AggregateClientImpl.java b/eagle-core/eagle-query/eagle-storage-hbase/src/main/java/org/apache/eagle/storage/hbase/query/coprocessor/impl/AggregateClientImpl.java
index 5835738..0e92c64 100755
--- a/eagle-core/eagle-query/eagle-storage-hbase/src/main/java/org/apache/eagle/storage/hbase/query/coprocessor/impl/AggregateClientImpl.java
+++ b/eagle-core/eagle-query/eagle-storage-hbase/src/main/java/org/apache/eagle/storage/hbase/query/coprocessor/impl/AggregateClientImpl.java
@@ -18,8 +18,8 @@ package org.apache.eagle.storage.hbase.query.coprocessor.impl;
 
 import org.apache.eagle.log.entity.meta.EntityDefinition;
 import org.apache.eagle.query.aggregate.AggregateFunctionType;
-import org.apache.eagle.storage.hbase.query.coprocessor.generated.AggregateProtos;
 import org.apache.eagle.storage.hbase.query.coprocessor.*;
+import org.apache.eagle.storage.hbase.query.coprocessor.generated.AggregateProtos;
 import org.apache.hadoop.hbase.client.HTableInterface;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.client.coprocessor.Batch;
@@ -32,115 +32,111 @@ import java.util.ArrayList;
 import java.util.List;
 
 /**
- * Not thread safe
- *
- * @since : 11/2/14,2014
+ * Not thread safe.
  */
 public class AggregateClientImpl implements AggregateClient {
-	private final static Logger LOG = LoggerFactory.getLogger(AggregateClient.class);
-	private AggregateResultCallback callback;
+    private static final Logger LOG = LoggerFactory.getLogger(AggregateClient.class);
+    private AggregateResultCallback callback;
 
-	private void checkNotNull(Object obj,String name) {
-		if(obj==null) throw new NullPointerException(name+" is null");
-	}
+    private void checkNotNull(Object obj, String name) {
+        if (obj == null) {
+            throw new NullPointerException(name + " is null");
+        }
+    }
 
-	@Override
-	public AggregateResult aggregate(final HTableInterface table,
-	                                       final EntityDefinition entityDefinition,
-	                                       final Scan scan,
-	                                       final List<String> groupbyFields,
-	                                       final List<AggregateFunctionType> aggregateFuncTypes,
-	                                       final List<String> aggregatedFields,
-	                                       final boolean timeSeries,
-	                                       final long startTime,
-	                                       final long endTime,
-	                                       final long intervalMin) throws IOException {
-		checkNotNull(entityDefinition,"entityDefinition");
-		final List<AggregateFunctionType> _aggregateFuncTypes = convertToCoprocessorAggregateFunc(aggregateFuncTypes);
-		final List<byte[]> _aggregateFuncTypesBytes = AggregateFunctionType.toBytesList(_aggregateFuncTypes);
-//		if(timeSeries) TimeSeriesAggregator.validateTimeRange(startTime,endTime,intervalMin);
-		callback = new AggregateResultCallbackImpl(aggregateFuncTypes);
-		try{
-			if(!LOG.isDebugEnabled()){
-				LOG.info("Going to exec coprocessor: "+AggregateProtocol.class.getSimpleName());
-			}else{
-				LOG.debug("Going to exec coprocessor: "+AggregateProtocol.class.getName());
-			}
+    @Override
+    public AggregateResult aggregate(final HTableInterface table,
+                                     final EntityDefinition entityDefinition,
+                                     final Scan scan,
+                                     final List<String> groupbyFields,
+                                     final List<AggregateFunctionType> aggregateFuncTypes,
+                                     final List<String> aggregatedFields,
+                                     final boolean timeSeries,
+                                     final long startTime,
+                                     final long endTime,
+                                     final long intervalMin) throws IOException {
+        checkNotNull(entityDefinition, "entityDefinition");
+        final List<AggregateFunctionType> _aggregateFuncTypes = convertToCoprocessorAggregateFunc(aggregateFuncTypes);
+        final List<byte[]> _aggregateFuncTypesBytes = AggregateFunctionType.toBytesList(_aggregateFuncTypes);
+        // if(timeSeries) TimeSeriesAggregator.validateTimeRange(startTime,endTime,intervalMin);
+        callback = new AggregateResultCallbackImpl(aggregateFuncTypes);
+        try {
+            if (!LOG.isDebugEnabled()) {
+                LOG.info("Going to exec coprocessor: " + AggregateProtocol.class.getSimpleName());
+            } else {
+                LOG.debug("Going to exec coprocessor: " + AggregateProtocol.class.getName());
+            }
 
-//			table.coprocessorExec(AggregateProtocol.class,scan.getStartRow(),scan.getStopRow(),new Batch.Call<AggregateProtocol, AggregateResult>(){
-//				@Override
-//				public AggregateResult call(AggregateProtocol instance) throws IOException {
-//					if(timeSeries){
-//						return instance.aggregate(entityDefinition, scan, groupbyFields, _aggregateFuncTypesBytes, aggregatedFields,startTime,endTime,intervalMin);
-//					}else{
-//						return instance.aggregate(entityDefinition, scan, groupbyFields, _aggregateFuncTypesBytes, aggregatedFields);
-//					}
-//				}
-//			},callback);
+            //  table.coprocessorExec(AggregateProtocol.class,scan.getStartRow(),scan.getStopRow(),new Batch.Call<AggregateProtocol, AggregateResult>(){
+            //  @Override
+            //  public AggregateResult call(AggregateProtocol instance) throws IOException {
+            //      if(timeSeries){
+            //   return instance.aggregate(entityDefinition, scan, groupbyFields, _aggregateFuncTypesBytes, aggregatedFields,startTime,endTime,intervalMin);
+            //      }else{
+            //   return instance.aggregate(entityDefinition, scan, groupbyFields, _aggregateFuncTypesBytes, aggregatedFields);
+            //      }
+            //  }
+            //  },callback);
 
-          table.coprocessorService(AggregateProtos.AggregateProtocol.class, scan.getStartRow(), scan.getStopRow(), new Batch.Call<AggregateProtos.AggregateProtocol, AggregateProtos.AggregateResult>() {
-              @Override
-              public AggregateProtos.AggregateResult call(AggregateProtos.AggregateProtocol instance) throws IOException {
-                  BlockingRpcCallback<AggregateProtos.AggregateResult> rpcCallback = new BlockingRpcCallback<AggregateProtos.AggregateResult>();
-                  if(timeSeries){
-                      AggregateProtos.TimeSeriesAggregateRequest timeSeriesAggregateRequest = ProtoBufConverter
-                              .toPBTimeSeriesRequest(
-                                      entityDefinition,
-                                      scan,
-                                      groupbyFields,
-                                      _aggregateFuncTypesBytes,
-                                      aggregatedFields,
-                                      startTime,
-                                      endTime,
-                                      intervalMin);
-                      instance.timeseriesAggregate(null, timeSeriesAggregateRequest, rpcCallback);
-                      return rpcCallback.get();
-					}else{
-                      AggregateProtos.AggregateRequest aggregateRequest = ProtoBufConverter.toPBRequest(
-                                      entityDefinition, scan, groupbyFields, _aggregateFuncTypesBytes, aggregatedFields);
-                      instance.aggregate(null, aggregateRequest, rpcCallback);
-                      return rpcCallback.get();
-					}
-              }
-          }, callback);
-		} catch (Throwable t){
-			LOG.error(t.getMessage(),t);
-			throw new IOException(t);
-		}
-		return callback.result();
-	}
-	
-//	@Override
-//	public void result(final GroupbyKeyValueCreationListener[] listeners) {
-//		callback.asyncRead(Arrays.asList(listeners));
-//	}
+            table.coprocessorService(AggregateProtos.AggregateProtocol.class,
+                scan.getStartRow(), scan.getStopRow(), new Batch.Call<AggregateProtos.AggregateProtocol, AggregateProtos.AggregateResult>() {
+                    @Override
+                    public AggregateProtos.AggregateResult call(AggregateProtos.AggregateProtocol instance) throws IOException {
+                        BlockingRpcCallback<AggregateProtos.AggregateResult> rpcCallback = new BlockingRpcCallback<>();
+                        if (timeSeries) {
+                            AggregateProtos.TimeSeriesAggregateRequest timeSeriesAggregateRequest = ProtoBufConverter
+                                    .toPBTimeSeriesRequest(
+                                            entityDefinition,
+                                            scan,
+                                            groupbyFields,
+                                            _aggregateFuncTypesBytes,
+                                            aggregatedFields,
+                                            startTime,
+                                            endTime,
+                                            intervalMin);
+                            instance.timeseriesAggregate(null, timeSeriesAggregateRequest, rpcCallback);
+                            return rpcCallback.get();
+                        } else {
+                            AggregateProtos.AggregateRequest aggregateRequest = ProtoBufConverter.toPBRequest(
+                                    entityDefinition, scan, groupbyFields, _aggregateFuncTypesBytes, aggregatedFields);
+                            instance.aggregate(null, aggregateRequest, rpcCallback);
+                            return rpcCallback.get();
+                        }
+                    }
+                }, callback);
+        } catch (Throwable t) {
+            LOG.error(t.getMessage(), t);
+            throw new IOException(t);
+        }
+        return callback.result();
+    }
 
-	@Override
-	public AggregateResult  aggregate(HTableInterface table, EntityDefinition entityDefinition, Scan scan, List<String> groupbyFields, List<AggregateFunctionType> aggregateFuncTypes, List<String> aggregatedFields) throws IOException {
-		return this.aggregate(table,entityDefinition,scan,groupbyFields,aggregateFuncTypes,aggregatedFields,false,0,0,0);
-	}
+    @Override
+    public AggregateResult aggregate(HTableInterface table, EntityDefinition entityDefinition, Scan scan,
+                                     List<String> groupbyFields, List<AggregateFunctionType> aggregateFuncTypes, List<String> aggregatedFields) throws IOException {
+        return this.aggregate(table, entityDefinition, scan, groupbyFields, aggregateFuncTypes, aggregatedFields, false, 0, 0, 0);
+    }
 
-	/**
-	 *
-	 * <h4>
-	 *   Convert client side funcs to server side funcs, especially for <b>avg</b>
-	 * </h4>
-	 * <ul>
-	 *  <li><b>avg</b>:
-	 *    Coprocessor[ <b>&lt;sum,count&gt;</b>] => Callback[(sum<SUB>1</SUB>+sum<SUB>2</SUB>+...+sum<SUB>n</SUB>)/(count<SUB>1</SUB>+count<SUB>2</SUB>+...+count<SUB>n</SUB>)]
-	 * </li>
-	 * </ul>
-	 * @param funcs List&lt;AggregateFunctionType&gt;
-	 * @return
-	 */
-	private List<AggregateFunctionType> convertToCoprocessorAggregateFunc(List<AggregateFunctionType> funcs){
-		List<AggregateFunctionType> copy = new ArrayList<AggregateFunctionType>(funcs);
-		for(int i=0;i<funcs.size();i++){
-			AggregateFunctionType func = copy.get(i);
-			if(AggregateFunctionType.avg.equals(func)){
-				copy.set(i,AggregateFunctionType.sum);
-			}
-		}
-		return copy;
-	}
+    /**
+     * <h4>
+     * Convert client side funcs to server side funcs, especially for <b>avg</b>
+     * </h4>
+     * <ul>
+     * <li><b>avg</b>:
+     * Coprocessor[ <b>&lt;sum,count&gt;</b>] => Callback[(sum<SUB>1</SUB>+sum<SUB>2</SUB>+...+sum<SUB>n</SUB>)/(count<SUB>1</SUB>+count<SUB>2</SUB>+...+count<SUB>n</SUB>)]
+     * </li>
+     * </ul>
+     *
+     * @param funcs List&lt;AggregateFunctionType&gt;
+     */
+    private List<AggregateFunctionType> convertToCoprocessorAggregateFunc(List<AggregateFunctionType> funcs) {
+        List<AggregateFunctionType> copy = new ArrayList<AggregateFunctionType>(funcs);
+        for (int i = 0; i < funcs.size(); i++) {
+            AggregateFunctionType func = copy.get(i);
+            if (AggregateFunctionType.avg.equals(func)) {
+                copy.set(i, AggregateFunctionType.sum);
+            }
+        }
+        return copy;
+    }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/21187b55/eagle-core/eagle-query/eagle-storage-hbase/src/main/java/org/apache/eagle/storage/hbase/query/coprocessor/impl/AggregateResultCallbackImpl.java
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-query/eagle-storage-hbase/src/main/java/org/apache/eagle/storage/hbase/query/coprocessor/impl/AggregateResultCallbackImpl.java b/eagle-core/eagle-query/eagle-storage-hbase/src/main/java/org/apache/eagle/storage/hbase/query/coprocessor/impl/AggregateResultCallbackImpl.java
index 2e0248f..5d7011f 100755
--- a/eagle-core/eagle-query/eagle-storage-hbase/src/main/java/org/apache/eagle/storage/hbase/query/coprocessor/impl/AggregateResultCallbackImpl.java
+++ b/eagle-core/eagle-query/eagle-storage-hbase/src/main/java/org/apache/eagle/storage/hbase/query/coprocessor/impl/AggregateResultCallbackImpl.java
@@ -16,13 +16,13 @@
  */
 package org.apache.eagle.storage.hbase.query.coprocessor.impl;
 
+import org.apache.eagle.common.ByteUtil;
 import org.apache.eagle.query.aggregate.AggregateFunctionType;
+import org.apache.eagle.query.aggregate.raw.*;
 import org.apache.eagle.storage.hbase.query.coprocessor.AggregateResult;
 import org.apache.eagle.storage.hbase.query.coprocessor.AggregateResultCallback;
 import org.apache.eagle.storage.hbase.query.coprocessor.ProtoBufConverter;
 import org.apache.eagle.storage.hbase.query.coprocessor.generated.AggregateProtos;
-import org.apache.eagle.common.ByteUtil;
-import org.apache.eagle.query.aggregate.raw.*;
 import org.apache.hadoop.hbase.coprocessor.CoprocessorException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -33,111 +33,113 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 
-/**
- * @since : 11/3/14,2014
- */
 public class AggregateResultCallbackImpl implements AggregateResultCallback {
-	private final static Logger LOG = LoggerFactory.getLogger(AggregateResultCallback.class);
-	private Map<GroupbyKey,List<Function>> groupedFuncs = new HashMap<GroupbyKey, List<Function>>();
-	private List<FunctionFactory> functionFactories = new ArrayList<FunctionFactory>();
-	private int numFuncs = 0;
-	private long kvCounter = 0;
-	private int regionCounter = 0;
-	private long startTimestamp;
-	private long stopTimestamp;
-	
-	// Start RPC call time (i.e constructor initialized time)
-	private final long _start;
-	
-	public AggregateResultCallbackImpl(List<AggregateFunctionType> aggregateFunctionTypes){
-		this.numFuncs = aggregateFunctionTypes.size();
-		for(AggregateFunctionType type: aggregateFunctionTypes){
-			 functionFactories.add(FunctionFactory.locateFunctionFactory(type));
-		}
-		this._start = System.currentTimeMillis();
-	}
+    private static final Logger LOG = LoggerFactory.getLogger(AggregateResultCallback.class);
+    private Map<GroupbyKey, List<Function>> groupedFuncs = new HashMap<GroupbyKey, List<Function>>();
+    private List<FunctionFactory> functionFactories = new ArrayList<FunctionFactory>();
+    private int numFuncs = 0;
+    private long kvCounter = 0;
+    private int regionCounter = 0;
+    private long startTimestamp;
+    private long stopTimestamp;
 
-//	@Override
-	public void update(byte[] region, byte[] row, AggregateResult result) {
-		AggregateResult _result = result;
-		regionCounter ++;
-		kvCounter += _result.getKeyValues().size();
-		if(this.startTimestamp == 0 || this.startTimestamp > _result.getStartTimestamp()){
-			this.startTimestamp = _result.getStartTimestamp();
-		}
-		if(this.stopTimestamp == 0 || this.stopTimestamp < _result.getStopTimestamp()){
-			this.stopTimestamp = _result.getStopTimestamp();
-		}
-		for(GroupbyKeyValue keyValue:_result.getKeyValues()){
-			update(keyValue);
-		}
-	}
+    // Start RPC call time (i.e constructor initialized time)
+    private final long _start;
 
-	public void update(GroupbyKeyValue keyValue) {
-		// Incr kvCounter if call #update(GroupbyKeyValue) directly
-		// instead of #update(byte[] region, byte[] row, AggregateResult result)
-		if(this.getKVCounter() == 0) this.kvCounter ++;
-		// Accumulate key value for GroubyKey mapped Functions
-		GroupbyKey groupedKey = keyValue.getKey();
-		List<Function> funcs = groupedFuncs.get(groupedKey);
-		if(funcs==null){
-			funcs = new ArrayList<Function>();
-			for(FunctionFactory functionFactory:this.functionFactories){
-				funcs.add(functionFactory.createFunction());
-			}
-			groupedFuncs.put(groupedKey, funcs);
-		}
-		for(int i=0;i<this.numFuncs;i++){
-			int intCount = 1;
-			byte[] count = keyValue.getValue().getMeta(i).getBytes();
-			if(count != null){
-				intCount = ByteUtil.bytesToInt(count);
-			}
-			funcs.get(i).run(keyValue.getValue().get(i).get(), intCount);
-		}
-	}
+    public AggregateResultCallbackImpl(List<AggregateFunctionType> aggregateFunctionTypes) {
+        this.numFuncs = aggregateFunctionTypes.size();
+        for (AggregateFunctionType type : aggregateFunctionTypes) {
+            functionFactories.add(FunctionFactory.locateFunctionFactory(type));
+        }
+        this._start = System.currentTimeMillis();
+    }
 
-	public long getKVCounter(){
-		return this.kvCounter;
-	}
+    public long getKVCounter() {
+        return this.kvCounter;
+    }
 
-	public long getRegionCounter(){
-		return this.regionCounter;
-	}
+    public long getRegionCounter() {
+        return this.regionCounter;
+    }
 
-	public AggregateResult result(){
-		List<GroupbyKeyValue> mergedKeyValues = new ArrayList<GroupbyKeyValue>();
-		for(Map.Entry<GroupbyKey,List<Function>> entry:this.groupedFuncs.entrySet()){
-			GroupbyValue value = new GroupbyValue(this.numFuncs);
-			for(Function func:entry.getValue()){
-				double _result = func.result();
-				int _count = func.count();
-				value.add(_result);
-				value.addMeta(_count);
-			}
-			mergedKeyValues.add(new GroupbyKeyValue(entry.getKey(),value));
-		}
-		
-		final long _stop = System.currentTimeMillis();
-		if(this.getRegionCounter() > 0) {
-			LOG.info(String.format("result = %d rows, startTime = %d, endTime = %d, source = %d rows, regions = %d, , spend = %d ms", mergedKeyValues.size(),this.startTimestamp,this.stopTimestamp, this.getKVCounter(), this.getRegionCounter(),(_stop - _start)));
-		}else{
-			LOG.info(String.format("result = %d rows, startTime = %d, endTime = %d, source = %d rows, spend = %d ms", mergedKeyValues.size(),this.startTimestamp,this.stopTimestamp,this.getKVCounter(), (_stop - _start)));
-		}
-		AggregateResult result = new AggregateResult();
-		result.setKeyValues(mergedKeyValues);
-		result.setStartTimestamp(this.startTimestamp);
-		result.setStopTimestamp(this.stopTimestamp);
-		return result;
-	}
+    public AggregateResult result() {
+        List<GroupbyKeyValue> mergedKeyValues = new ArrayList<GroupbyKeyValue>();
+        for (Map.Entry<GroupbyKey, List<Function>> entry : this.groupedFuncs.entrySet()) {
+            GroupbyValue value = new GroupbyValue(this.numFuncs);
+            for (Function func : entry.getValue()) {
+                double _result = func.result();
+                int _count = func.count();
+                value.add(_result);
+                value.addMeta(_count);
+            }
+            mergedKeyValues.add(new GroupbyKeyValue(entry.getKey(), value));
+        }
+
+        final long _stop = System.currentTimeMillis();
+        if (this.getRegionCounter() > 0) {
+            LOG.info(String.format("result = %d rows, startTime = %d, endTime = %d, source = %d rows, regions = %d, , spend = %d ms",
+                mergedKeyValues.size(), this.startTimestamp, this.stopTimestamp, this.getKVCounter(), this.getRegionCounter(), (_stop - _start)));
+        } else {
+            LOG.info(String.format("result = %d rows, startTime = %d, endTime = %d, source = %d rows, spend = %d ms",
+                mergedKeyValues.size(), this.startTimestamp, this.stopTimestamp, this.getKVCounter(), (_stop - _start)));
+        }
+        AggregateResult result = new AggregateResult();
+        result.setKeyValues(mergedKeyValues);
+        result.setStartTimestamp(this.startTimestamp);
+        result.setStopTimestamp(this.stopTimestamp);
+        return result;
+    }
 
     @Override
     public void update(byte[] region, byte[] row, AggregateProtos.AggregateResult result) {
         try {
-            if(result == null) throw new IllegalStateException(new CoprocessorException("result is null"));
-            this.update(region,row, ProtoBufConverter.fromPBResult(result));
+            if (result == null) {
+                throw new IllegalStateException(new CoprocessorException("result is null"));
+            }
+            this.update(region, row, ProtoBufConverter.fromPBResult(result));
         } catch (IOException e) {
-            LOG.error("Failed to convert PB-Based message",e);
+            LOG.error("Failed to convert PB-Based message", e);
+        }
+    }
+
+    public void update(GroupbyKeyValue keyValue) {
+        // Incr kvCounter if call #update(GroupbyKeyValue) directly
+        // instead of #update(byte[] region, byte[] row, AggregateResult result)
+        if (this.getKVCounter() == 0) {
+            this.kvCounter++;
+        }
+        // Accumulate key value for GroubyKey mapped Functions
+        GroupbyKey groupedKey = keyValue.getKey();
+        List<Function> funcs = groupedFuncs.get(groupedKey);
+        if (funcs == null) {
+            funcs = new ArrayList<Function>();
+            for (FunctionFactory functionFactory : this.functionFactories) {
+                funcs.add(functionFactory.createFunction());
+            }
+            groupedFuncs.put(groupedKey, funcs);
+        }
+        for (int i = 0; i < this.numFuncs; i++) {
+            int intCount = 1;
+            byte[] count = keyValue.getValue().getMeta(i).getBytes();
+            if (count != null) {
+                intCount = ByteUtil.bytesToInt(count);
+            }
+            funcs.get(i).run(keyValue.getValue().get(i).get(), intCount);
+        }
+    }
+
+    public void update(byte[] region, byte[] row, AggregateResult result) {
+        AggregateResult _result = result;
+        regionCounter++;
+        kvCounter += _result.getKeyValues().size();
+        if (this.startTimestamp == 0 || this.startTimestamp > _result.getStartTimestamp()) {
+            this.startTimestamp = _result.getStartTimestamp();
+        }
+        if (this.stopTimestamp == 0 || this.stopTimestamp < _result.getStopTimestamp()) {
+            this.stopTimestamp = _result.getStopTimestamp();
+        }
+        for (GroupbyKeyValue keyValue : _result.getKeyValues()) {
+            update(keyValue);
         }
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/21187b55/eagle-core/eagle-query/eagle-storage-hbase/src/main/java/org/apache/eagle/storage/hbase/query/coprocessor/package-info.java
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-query/eagle-storage-hbase/src/main/java/org/apache/eagle/storage/hbase/query/coprocessor/package-info.java b/eagle-core/eagle-query/eagle-storage-hbase/src/main/java/org/apache/eagle/storage/hbase/query/coprocessor/package-info.java
index 7d81872..09cf983 100755
--- a/eagle-core/eagle-query/eagle-storage-hbase/src/main/java/org/apache/eagle/storage/hbase/query/coprocessor/package-info.java
+++ b/eagle-core/eagle-query/eagle-storage-hbase/src/main/java/org/apache/eagle/storage/hbase/query/coprocessor/package-info.java
@@ -14,35 +14,35 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+
 /**
- *
  * <h1>Eagle Aggregation Coprocessor</h1>
- *
+ * <p>
  * <h2>Deployment and Usage</h2>
  * <ol>
- * 	<li>
- *  Firstly deploy jar files to cluster on local file system or HDFS.<br/>
- * 	</li>
- * 	<li>
- * 	Secondly configure in <code>hbase-site.xml</code> as following:
- * 	<pre>&lt;property&gt;
+ * <li>
+ * Firstly deploy jar files to cluster on local file system or HDFS.<br/>
+ * </li>
+ * <li>
+ * Secondly configure in <code>hbase-site.xml</code> as following:
+ * <pre>&lt;property&gt;
  *   &lt;name>hbase.coprocessor.region.classes&lt;/name&gt;
  *   &lt;value>AggregateProtocolEndPoint&lt;/value&gt;
  * &lt;/property&gt;
- * 	</pre>
- * 	Or register on related hbase tables
- * 	<pre> hbase(main):005:0>  alter 't1', METHOD => 'table_att', 'coprocessor'=>'hdfs:///foo.jar|AggregateProtocolEndPoint|1001|'</pre>
- * 	</li>
- * 	<li>
+ * </pre>
+ * Or register on related hbase tables
+ * <pre> hbase(main):005:0>  alter 't1', METHOD => 'table_att', 'coprocessor'=>'hdfs:///foo.jar|AggregateProtocolEndPoint|1001|'</pre>
+ * </li>
+ * <li>
  * <code>
  * AggregateClient client = new AggregateClientImpl();
  * client.aggregate
  * AggregateResult result = client.result
- * 
+ *
  * </code>
  * </li>
  * </ol>
- * 
+ *
  * <h2>Performance</h2>
  *
  * <b>NOTE:</b>
@@ -56,56 +56,53 @@
  * <b>A simple benchmark report for reference</b>
  * <br/>
  * <table border="1">
- *     <thead>
- *         <tr>
- *             <th>Region Servers</th> <th>Record Count</th>
- *             <th>Coprocessor</th><th>No-Coprocessor</th><th>Aggregation</th>
- *         </tr>
- *     </thead>
- *     <tbody>
- *         <tr>
- *             <td rowspan="10">1</td><td rowspan="10">1000,000</td>
- *             <td>10193 ms</td><td>21988 ms</t...@datacenter>{count}</td>
- *         </tr>
- *         <tr>
- *             <td>10010 ms</td><td>22547 ms</t...@datacenter>{sum(numTotalMaps)}</td>
- *         </tr>
- *         <tr>
- *             <td>10334 ms</td><td>23433 ms</t...@datacenter>{avg(numTotalMaps)}</td>
- *         </tr>
- *         <tr>
- *             <td>10045 ms</td><td>22690 ms</t...@datacenter>{max(numTotalMaps)}</td>
- *         </tr>
- *         <tr>
- *             <td>10190 ms</td><td>21902 ms</t...@datacenter>{min(numTotalMaps)}</td>
- *         </tr>
- *     </tbody>
+ * <thead>
+ * <tr>
+ * <th>Region Servers</th> <th>Record Count</th>
+ * <th>Coprocessor</th><th>No-Coprocessor</th><th>Aggregation</th>
+ * </tr>
+ * </thead>
+ * <tbody>
+ * <tr>
+ * <td rowspan="10">1</td><td rowspan="10">1000,000</td>
+ * <td>10193 ms</td><td>21988 ms</t...@datacenter>{count}</td>
+ * </tr>
+ * <tr>
+ * <td>10010 ms</td><td>22547 ms</t...@datacenter>{sum(numTotalMaps)}</td>
+ * </tr>
+ * <tr>
+ * <td>10334 ms</td><td>23433 ms</t...@datacenter>{avg(numTotalMaps)}</td>
+ * </tr>
+ * <tr>
+ * <td>10045 ms</td><td>22690 ms</t...@datacenter>{max(numTotalMaps)}</td>
+ * </tr>
+ * <tr>
+ * <td>10190 ms</td><td>21902 ms</t...@datacenter>{min(numTotalMaps)}</td>
+ * </tr>
+ * </tbody>
  * </table>
  * <h2>Reference</h2>
  * <a href="https://blogs.apache.org/hbase/entry/coprocessor_introduction">
- * 	Coprocessor Introduction 
+ * Coprocessor Introduction
  * </a>
  * (Trend Micro Hadoop Group: Mingjie Lai, Eugene Koontz, Andrew Purtell)
- * 
+ *
  * <h2>TO-DO</h2>
  * <ol>
  * <li>
- *   TODO: Pass writable self-described entity definition into HBase coprocessor instead of serviceName in String
+ * TODO: Pass writable self-described entity definition into HBase coprocessor instead of serviceName in String
  *
- *   Because using serviceName to get entity definition will reply on entity API code under eagle-app, so that
- *   when modifying or creating new entities, we have to update coprocessor jar in HBase side
- *   (hchen9@xyz.com)
+ * Because using serviceName to get entity definition will reply on entity API code under eagle-app, so that
+ * when modifying or creating new entities, we have to update coprocessor jar in HBase side
+ * (hchen9@xyz.com)
  * </li>
  * <li>
- * 	 TODO: Using String.format instead substrings addition for long log to avoid recreating string objects
+ * TODO: Using String.format instead substrings addition for long log to avoid recreating string objects
  * </li>
  * </ol>
  *
  * </table>
- * @see eagle.query.aggregate.coprocessor.AggregateClient
- * @see eagle.query.aggregate.coprocessor.AggregateResult
- * @see eagle.query.aggregate.coprocessor.AggregateProtocol
- * 
- * @since   : 11/10/14,2014
+ *
+ * @since : 11/10/14,2014
  */
 package org.apache.eagle.storage.hbase.query.coprocessor;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/21187b55/eagle-core/eagle-query/eagle-storage-hbase/src/main/java/org/apache/eagle/storage/hbase/spi/HBaseStorageServiceProvider.java
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-query/eagle-storage-hbase/src/main/java/org/apache/eagle/storage/hbase/spi/HBaseStorageServiceProvider.java b/eagle-core/eagle-query/eagle-storage-hbase/src/main/java/org/apache/eagle/storage/hbase/spi/HBaseStorageServiceProvider.java
index 1ee1c52..3f3f831 100644
--- a/eagle-core/eagle-query/eagle-storage-hbase/src/main/java/org/apache/eagle/storage/hbase/spi/HBaseStorageServiceProvider.java
+++ b/eagle-core/eagle-query/eagle-storage-hbase/src/main/java/org/apache/eagle/storage/hbase/spi/HBaseStorageServiceProvider.java
@@ -20,11 +20,8 @@ import org.apache.eagle.storage.DataStorage;
 import org.apache.eagle.storage.hbase.HBaseStorage;
 import org.apache.eagle.storage.spi.DataStorageServiceProvider;
 
-/**
- * @since 3/20/15
- */
 public final class HBaseStorageServiceProvider implements DataStorageServiceProvider {
-    private final static String HBASE = "hbase";
+    private static final String HBASE = "hbase";
 
     @Override
     public String getType() {

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/21187b55/eagle-core/eagle-query/eagle-storage-hbase/src/main/java/org/apache/eagle/storage/hbase/tools/CoprocessorTool.java
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-query/eagle-storage-hbase/src/main/java/org/apache/eagle/storage/hbase/tools/CoprocessorTool.java b/eagle-core/eagle-query/eagle-storage-hbase/src/main/java/org/apache/eagle/storage/hbase/tools/CoprocessorTool.java
new file mode 100644
index 0000000..c5aecab
--- /dev/null
+++ b/eagle-core/eagle-query/eagle-storage-hbase/src/main/java/org/apache/eagle/storage/hbase/tools/CoprocessorTool.java
@@ -0,0 +1,159 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.eagle.storage.hbase.tools;
+
+import org.apache.eagle.storage.hbase.query.coprocessor.AggregateProtocolEndPoint;
+
+import org.apache.commons.cli.*;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.conf.Configured;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.Coprocessor;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.hadoop.util.Tool;
+import org.apache.hadoop.util.ToolRunner;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.HashMap;
+
+/**
+ * Coprocessor CLI Tool.
+ */
+public class CoprocessorTool extends Configured implements Tool {
+    private static final Logger LOGGER = LoggerFactory.getLogger(CoprocessorTool.class);
+
+    public static void main(String[] args) throws Exception {
+        System.exit(ToolRunner.run(new CoprocessorTool(), args));
+    }
+
+    private void unregisterCoprocessor(String tableName) throws IOException {
+        Configuration configuration = getConf();
+        TableName table = TableName.valueOf(tableName);
+        try (HBaseAdmin admin = new HBaseAdmin(configuration)) {
+            HTableDescriptor tableDescriptor = admin.getTableDescriptor(table);
+            LOGGER.info("Table {} found", tableName);
+            if (tableDescriptor.hasCoprocessor(AggregateProtocolEndPoint.class.getName())) {
+                LOGGER.warn("No coprocessor was registered on table '{}'", tableName);
+                throw new IOException("No coprocessor was registered on table " + tableName);
+            } else {
+                tableDescriptor.removeCoprocessor(AggregateProtocolEndPoint.class.getName());
+                admin.modifyTable(table, tableDescriptor);
+                LOGGER.info("Succeed to remove coprocessor from table " + tableName);
+            }
+        }
+    }
+
+    private void registerCoprocessor(String jarPath, String tableName, String localJarPath) throws IOException {
+        Configuration configuration = getConf();
+        try (FileSystem fs = FileSystem.get(configuration); HBaseAdmin admin = new HBaseAdmin(configuration)) {
+            Path path = new Path(fs.getUri() + Path.SEPARATOR + jarPath);
+            LOGGER.info("Checking path {} ... ", path.toString());
+            if (!fs.exists(path)) {
+                LOGGER.info("Path: {} not exist, uploading jar ...", path.toString());
+                if (localJarPath == null) {
+                    throw new IOException("local jar path is not given, please manually upload coprocessor jar onto hdfs at " + jarPath
+                            + " and retry, or provide local coprocessor jar path through CLI argument and upload automatically");
+                }
+                LOGGER.info("Copying from local {} to {}", localJarPath, jarPath);
+                fs.copyFromLocalFile(new Path(localJarPath), path);
+                LOGGER.info("Succeed to copied coprocessor jar to {}", path.toString());
+            } else {
+                LOGGER.info("Path {} already exists", path.toString());
+            }
+            LOGGER.info("Checking hbase table {}", tableName);
+            TableName table = TableName.valueOf(tableName);
+            HTableDescriptor tableDescriptor = admin.getTableDescriptor(table);
+            LOGGER.info("Table {} found", tableName);
+            if (tableDescriptor.hasCoprocessor(AggregateProtocolEndPoint.class.getName())) {
+                LOGGER.warn("Table '" + tableName + "' already registered coprocessor: " + AggregateProtocolEndPoint.class.getName() + ", removing firstly");
+                tableDescriptor.removeCoprocessor(AggregateProtocolEndPoint.class.getName());
+                admin.modifyTable(table, tableDescriptor);
+                tableDescriptor = admin.getTableDescriptor(table);
+            }
+            tableDescriptor.addCoprocessor(AggregateProtocolEndPoint.class.getName(),
+                    path, Coprocessor.PRIORITY_USER, new HashMap<>());
+            admin.modifyTable(table, tableDescriptor);
+            LOGGER.info("Succeed to enable coprocessor on table " + tableName);
+        }
+    }
+
+    private void printHelpMessage(Options cmdOptions) {
+        HelpFormatter formatter = new HelpFormatter();
+        formatter.printHelp("java " + CoprocessorTool.class.getName() + " [--register/--unregister] [OPTIONS]", cmdOptions);
+    }
+
+    @Override
+    public int run(String[] args) throws Exception {
+        Options cmdOptions = new Options();
+        cmdOptions.addOption(new Option("register", false, "Register coprocessor"));
+        cmdOptions.addOption(new Option("unregister", false, "Unregister coprocessor"));
+
+        cmdOptions.addOption("table", true, "HBase table name, separated with comma, for example, table1,table2,..");
+        cmdOptions.addOption("jar", true, "Coprocessor target jar path");
+        cmdOptions.addOption("localJar", true, "Coprocessor local source jar path");
+        cmdOptions.addOption("config", true, "Configuration file");
+
+        cmdOptions.getOption("table").setType(String.class);
+        cmdOptions.getOption("table").setRequired(true);
+        cmdOptions.getOption("jar").setType(String.class);
+        cmdOptions.getOption("jar").setRequired(false);
+        cmdOptions.getOption("localJar").setType(String.class);
+        cmdOptions.getOption("localJar").setRequired(false);
+        cmdOptions.getOption("config").setType(String.class);
+        cmdOptions.getOption("config").setRequired(false);
+
+        GnuParser parser = new GnuParser();
+        CommandLine cmdCli = parser.parse(cmdOptions, args);
+        String tableName = cmdCli.getOptionValue("table");
+        String configFile = cmdCli.getOptionValue("config");
+
+        if (configFile != null) {
+            Configuration.addDefaultResource(configFile);
+        }
+
+        if (cmdCli.hasOption("register")) {
+            if (args.length < 3) {
+                System.err.println("Error: coprocessor jar path is missing");
+                System.err.println("Usage: java " + CoprocessorTool.class.getName() + " enable " + tableName + " [jarOnHdfs] [jarOnLocal]");
+                return 1;
+            }
+            String jarPath = cmdCli.getOptionValue("jar");
+            LOGGER.info("Table name: {}", tableName);
+            LOGGER.info("Coprocessor jar on hdfs: {}", jarPath);
+            String localJarPath = cmdCli.getOptionValue("localJar");
+            LOGGER.info("Coprocessor jar on local: {}", localJarPath);
+
+            String[] tableNames = tableName.split(",\\s*");
+            for (String table : tableNames) {
+                LOGGER.info("Registering coprocessor for table {}", table);
+                registerCoprocessor(jarPath, table, localJarPath);
+            }
+        } else if (cmdCli.hasOption("unregister")) {
+            unregisterCoprocessor(tableName);
+        } else {
+            System.err.println("command is required, --register/--unregister");
+            printHelpMessage(cmdOptions);
+        }
+        return 0;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/21187b55/eagle-core/eagle-query/eagle-storage-hbase/src/main/protobuf/AggregateProtocol.proto
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-query/eagle-storage-hbase/src/main/protobuf/AggregateProtocol.proto b/eagle-core/eagle-query/eagle-storage-hbase/src/main/protobuf/AggregateProtocol.proto
index c3385a1..da5846c 100644
--- a/eagle-core/eagle-query/eagle-storage-hbase/src/main/protobuf/AggregateProtocol.proto
+++ b/eagle-core/eagle-query/eagle-storage-hbase/src/main/protobuf/AggregateProtocol.proto
@@ -27,7 +27,7 @@ option optimize_for = SPEED;
  */
 import "Client.proto";
 
-//
+
 //message ScanWrapper{
 //    required bytes byte_array = 1;
 //}
@@ -43,7 +43,7 @@ message AggregateResult {
 message AggregateRequest {
     required EntityDefinition entity_definition = 1;
     required Scan scan = 2;
-    repeated string groupby_fields= 3;
+    repeated string groupby_fields = 3;
     repeated bytes aggregate_func_types = 4;
     repeated string aggregated_fields = 5;
 }
@@ -51,7 +51,7 @@ message AggregateRequest {
 message TimeSeriesAggregateRequest {
     required EntityDefinition entity_definition = 1;
     required Scan scan = 2;
-    repeated string groupby_fields= 3;
+    repeated string groupby_fields = 3;
     repeated bytes aggregate_func_types = 4;
     repeated string aggregated_fields = 5;
     required int64 start_time = 6;
@@ -60,6 +60,6 @@ message TimeSeriesAggregateRequest {
 }
 
 service AggregateProtocol {
-    rpc aggregate(AggregateRequest) returns (AggregateResult);
-    rpc timeseriesAggregate(TimeSeriesAggregateRequest) returns (AggregateResult);
+    rpc aggregate (AggregateRequest) returns (AggregateResult);
+    rpc timeseriesAggregate (TimeSeriesAggregateRequest) returns (AggregateResult);
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/21187b55/eagle-core/eagle-query/eagle-storage-hbase/src/test/java/org/apache/eagle/storage/hbase/TestHBaseStatement.java
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-query/eagle-storage-hbase/src/test/java/org/apache/eagle/storage/hbase/TestHBaseStatement.java b/eagle-core/eagle-query/eagle-storage-hbase/src/test/java/org/apache/eagle/storage/hbase/TestHBaseStatement.java
index 079cf12..6dcbc78 100644
--- a/eagle-core/eagle-query/eagle-storage-hbase/src/test/java/org/apache/eagle/storage/hbase/TestHBaseStatement.java
+++ b/eagle-core/eagle-query/eagle-storage-hbase/src/test/java/org/apache/eagle/storage/hbase/TestHBaseStatement.java
@@ -67,18 +67,18 @@ public class TestHBaseStatement extends TestHBaseBase {
             {
                 put("cluster", "test");
                 put("datacenter", "test");
-                put("name","unit.test.name");
+                put("name", "unit.test.name");
             }
         });
 
         entities.add(entity);
 
-        CreateStatement createStatement = new CreateStatement(entities,"TestTimeSeriesAPIEntity");
+        CreateStatement createStatement = new CreateStatement(entities, "TestTimeSeriesAPIEntity");
         ModifyResult resultSet = createStatement.execute(DataStorageManager.newDataStorage("hbase"));
 
         Assert.assertEquals(1, resultSet.getIdentifiers().size());
 
-        createStatement = new CreateStatement(entities,"TestTimeSeriesAPIEntity");
+        createStatement = new CreateStatement(entities, "TestTimeSeriesAPIEntity");
         resultSet = createStatement.execute(DataStorageManager.newDataStorage("hbase"));
 
         Assert.assertEquals(1, resultSet.getIdentifiers().size());

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/21187b55/eagle-core/eagle-query/eagle-storage-hbase/src/test/java/org/apache/eagle/storage/hbase/TestHBaseStorage.java
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-query/eagle-storage-hbase/src/test/java/org/apache/eagle/storage/hbase/TestHBaseStorage.java b/eagle-core/eagle-query/eagle-storage-hbase/src/test/java/org/apache/eagle/storage/hbase/TestHBaseStorage.java
deleted file mode 100644
index 76cc507..0000000
--- a/eagle-core/eagle-query/eagle-storage-hbase/src/test/java/org/apache/eagle/storage/hbase/TestHBaseStorage.java
+++ /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.
- */
-package org.apache.eagle.storage.hbase;
-
-import org.junit.Test;
-
-/**
- * @since 3/23/15
- */
-public class TestHBaseStorage {
-    @Test
-    public void testCreate(){
-
-    }
-
-    @Test
-    public void testQuery(){
-
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/21187b55/eagle-core/eagle-query/eagle-storage-hbase/src/test/java/org/apache/eagle/storage/hbase/aggregate/coprocessor/TestAggregateResultCallback.java
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-query/eagle-storage-hbase/src/test/java/org/apache/eagle/storage/hbase/aggregate/coprocessor/TestAggregateResultCallback.java b/eagle-core/eagle-query/eagle-storage-hbase/src/test/java/org/apache/eagle/storage/hbase/aggregate/coprocessor/TestAggregateResultCallback.java
index ba1b781..df5dcd9 100755
--- a/eagle-core/eagle-query/eagle-storage-hbase/src/test/java/org/apache/eagle/storage/hbase/aggregate/coprocessor/TestAggregateResultCallback.java
+++ b/eagle-core/eagle-query/eagle-storage-hbase/src/test/java/org/apache/eagle/storage/hbase/aggregate/coprocessor/TestAggregateResultCallback.java
@@ -16,26 +16,24 @@
  */
 package org.apache.eagle.storage.hbase.aggregate.coprocessor;
 
+import org.apache.eagle.common.ByteUtil;
 import org.apache.eagle.query.aggregate.AggregateFunctionType;
-import org.apache.eagle.storage.hbase.query.coprocessor.AggregateResult;
-import org.apache.eagle.storage.hbase.query.coprocessor.AggregateResultCallback;
-import org.apache.eagle.storage.hbase.query.coprocessor.impl.AggregateResultCallbackImpl;
 import org.apache.eagle.query.aggregate.raw.GroupbyKey;
 import org.apache.eagle.query.aggregate.raw.GroupbyKeyValue;
 import org.apache.eagle.query.aggregate.raw.GroupbyValue;
-import org.apache.eagle.common.ByteUtil;
+import org.apache.eagle.storage.hbase.query.coprocessor.AggregateResult;
+import org.apache.eagle.storage.hbase.query.coprocessor.AggregateResultCallback;
+import org.apache.eagle.storage.hbase.query.coprocessor.impl.AggregateResultCallbackImpl;
 import org.junit.Assert;
-import org.junit.Ignore;
 import org.junit.Test;
 
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.List;
 
-@Ignore
 public class TestAggregateResultCallback {
     @Test
-    public void testUpdate(){
+    public void testUpdate() {
         // -----------------------------------------------------------------------------
         // key      |       max      min        count       avg         sum      | count
         // -----------------------------------------------------------------------------
@@ -49,89 +47,88 @@ public class TestAggregateResultCallback {
         // -----------------------------------------------------------------------------
 
         AggregateResultCallback callback = new AggregateResultCallbackImpl(Arrays.asList(
-                        AggregateFunctionType.max,
-                        AggregateFunctionType.min,
-                        AggregateFunctionType.count,
-                        AggregateFunctionType.avg,
-                        AggregateFunctionType.sum));
+                AggregateFunctionType.max,
+                AggregateFunctionType.min,
+                AggregateFunctionType.count,
+                AggregateFunctionType.avg,
+                AggregateFunctionType.sum));
         AggregateResult result1 = AggregateResult.build(
                 Arrays.asList(
-                    new String[]{"a","b"},
-                    new String[]{"a","b"},
-                    new String[]{"a","b","c"},
-                    new String[]{"a","b","c"}
+                        new String[]{"a", "b"},
+                        new String[]{"a", "b"},
+                        new String[]{"a", "b", "c"},
+                        new String[]{"a", "b", "c"}
                 ),
                 Arrays.asList(
-                    new double[]{1.0,2.0,3.0,4.0,5.0},
-                    new double[]{2.0,3.0,6.0,5.0,6.0},
-                    new double[]{3.0,3.0,5.0,5.0,6.0},
-                    new double[]{4.0,5.0,5.0,5.0,7.0}
+                        new double[]{1.0, 2.0, 3.0, 4.0, 5.0},
+                        new double[]{2.0, 3.0, 6.0, 5.0, 6.0},
+                        new double[]{3.0, 3.0, 5.0, 5.0, 6.0},
+                        new double[]{4.0, 5.0, 5.0, 5.0, 7.0}
                 ),
-                Arrays.asList(3,6,5,5),
+                Arrays.asList(3, 6, 5, 5),
                 System.currentTimeMillis(),
                 System.currentTimeMillis()
         );
-        callback.update(null,null,result1);
+        callback.update(null, null, result1);
         AggregateResult callbackResult = callback.result();
-        Assert.assertEquals(2,callbackResult.getKeyValues().size());
+        Assert.assertEquals(2, callbackResult.getKeyValues().size());
 
         // == ROW-#0 ==
         // Should be:
         // key      |       max      min        count       avg         sum      | count
         // -----------------------------------------------------------------------------
-        // a,b,c    |       4        3          10          1           13       | 10
+        // a,b      |       2        2          9           1           11       | 9
         GroupbyKeyValue row0 = callbackResult.getKeyValues().get(0);
-//        Assert.assertEquals("a",new String(row0.getKey().getValue().get(0).copyBytes()));
-//        Assert.assertEquals("b",new String(row0.getKey().getValue().get(1).copyBytes()));
-        Assert.assertEquals(new GroupbyKey(Arrays.asList("a".getBytes(),"b".getBytes(),"c".getBytes())),row0.getKey());
-        Assert.assertEquals(4.0,row0.getValue().get(0).get(), 0.00001);
-        Assert.assertEquals(10, ByteUtil.bytesToInt(row0.getValue().getMeta(0).getBytes()));
-        Assert.assertEquals(3.0, row0.getValue().get(1).get(), 0.00001);
-        Assert.assertEquals(10, ByteUtil.bytesToInt(row0.getValue().getMeta(1).getBytes()));
-        Assert.assertEquals(10.0,row0.getValue().get(2).get(), 0.00001);
-        Assert.assertEquals(10, ByteUtil.bytesToInt(row0.getValue().getMeta(2).getBytes()));
-        Assert.assertEquals(1.0,row0.getValue().get(3).get(), 0.00001);
-        Assert.assertEquals(10, ByteUtil.bytesToInt(row0.getValue().getMeta(3).getBytes()));
-        Assert.assertEquals(13.0,row0.getValue().get(4).get(), 0.00001);
-        Assert.assertEquals(10, ByteUtil.bytesToInt(row0.getValue().getMeta(4).getBytes()));
+        Assert.assertEquals(new GroupbyKey(Arrays.asList("a".getBytes(), "b".getBytes())), row0.getKey());
+        Assert.assertEquals(2.0, row0.getValue().get(0).get(), 0.00001);
+        Assert.assertEquals(9, ByteUtil.bytesToInt(row0.getValue().getMeta(4).getBytes()));
+        Assert.assertEquals(2.0, row0.getValue().get(1).get(), 0.00001);
+        Assert.assertEquals(9, ByteUtil.bytesToInt(row0.getValue().getMeta(4).getBytes()));
+        Assert.assertEquals(9.0, row0.getValue().get(2).get(), 0.00001);
+        Assert.assertEquals(9, ByteUtil.bytesToInt(row0.getValue().getMeta(4).getBytes()));
+        Assert.assertEquals(1.0, row0.getValue().get(3).get(), 0.00001);
+        Assert.assertEquals(9, ByteUtil.bytesToInt(row0.getValue().getMeta(4).getBytes()));
+        Assert.assertEquals(11.0, row0.getValue().get(4).get(), 0.00001);
+        Assert.assertEquals(9, ByteUtil.bytesToInt(row0.getValue().getMeta(4).getBytes()));
 
         // == ROW-#1 ==
         // Should be:
         // key      |       max      min        count       avg         sum      | count
         // -----------------------------------------------------------------------------
-        // a,b      |       2        2          9           1           11       | 9
+        // a,b,c    |       4        3          10          1           13       | 10
         GroupbyKeyValue row1 = callbackResult.getKeyValues().get(1);
-        Assert.assertEquals(new GroupbyKey(Arrays.asList("a".getBytes(),"b".getBytes())),row1.getKey());
-        Assert.assertEquals(2.0,row1.getValue().get(0).get(), 0.00001);
-        Assert.assertEquals(9, ByteUtil.bytesToInt(row1.getValue().getMeta(4).getBytes()));
-        Assert.assertEquals(2.0, row1.getValue().get(1).get(), 0.00001);
-        Assert.assertEquals(9, ByteUtil.bytesToInt(row1.getValue().getMeta(4).getBytes()));
-        Assert.assertEquals(9.0,row1.getValue().get(2).get(), 0.00001);
-        Assert.assertEquals(9, ByteUtil.bytesToInt(row1.getValue().getMeta(4).getBytes()));
-        Assert.assertEquals(1.0,row1.getValue().get(3).get(), 0.00001);
-        Assert.assertEquals(9, ByteUtil.bytesToInt(row1.getValue().getMeta(4).getBytes()));
-        Assert.assertEquals(11.0,row1.getValue().get(4).get(), 0.00001);
-        Assert.assertEquals(9, ByteUtil.bytesToInt(row1.getValue().getMeta(4).getBytes()));
+        Assert.assertEquals(new GroupbyKey(Arrays.asList("a".getBytes(), "b".getBytes(), "c".getBytes())), row1.getKey());
+        Assert.assertEquals(4.0, row1.getValue().get(0).get(), 0.00001);
+        Assert.assertEquals(10, ByteUtil.bytesToInt(row1.getValue().getMeta(0).getBytes()));
+        Assert.assertEquals(3.0, row1.getValue().get(1).get(), 0.00001);
+        Assert.assertEquals(10, ByteUtil.bytesToInt(row1.getValue().getMeta(1).getBytes()));
+        Assert.assertEquals(10.0, row1.getValue().get(2).get(), 0.00001);
+        Assert.assertEquals(10, ByteUtil.bytesToInt(row1.getValue().getMeta(2).getBytes()));
+        Assert.assertEquals(1.0, row1.getValue().get(3).get(), 0.00001);
+        Assert.assertEquals(10, ByteUtil.bytesToInt(row1.getValue().getMeta(3).getBytes()));
+        Assert.assertEquals(13.0, row1.getValue().get(4).get(), 0.00001);
+        Assert.assertEquals(10, ByteUtil.bytesToInt(row1.getValue().getMeta(4).getBytes()));
+
     }
 
     @Test
-    public void testAggregateResultTimestamp(){
+    public void testAggregateResultTimestamp() {
         AggregateResult result1 = new AggregateResult();
-        result1.setStartTimestamp(2l);
-        result1.setStopTimestamp(4l);
+        result1.setStartTimestamp(2L);
+        result1.setStopTimestamp(4L);
         AggregateResult result2 = new AggregateResult();
-        result2.setStartTimestamp(1l);
-        result2.setStopTimestamp(3l);
-        AggregateResultCallback  callback = new AggregateResultCallbackImpl(new ArrayList<AggregateFunctionType>());
-        callback.update(null,null,result1);
-        callback.update(null,null,result2);
+        result2.setStartTimestamp(1L);
+        result2.setStopTimestamp(3L);
+        AggregateResultCallback callback = new AggregateResultCallbackImpl(new ArrayList<AggregateFunctionType>());
+        callback.update(null, null, result1);
+        callback.update(null, null, result2);
         AggregateResult result3 = callback.result();
-        Assert.assertEquals(1l,result3.getStartTimestamp());
-        Assert.assertEquals(4l,result3.getStopTimestamp());
+        Assert.assertEquals(1L, result3.getStartTimestamp());
+        Assert.assertEquals(4L, result3.getStopTimestamp());
     }
 
     @Test
-    public void testUpdatePerformance(){
+    public void testUpdatePerformance() {
         AggregateResultCallback callback = new AggregateResultCallbackImpl(
                 Arrays.asList(
                         AggregateFunctionType.max,
@@ -139,10 +136,10 @@ public class TestAggregateResultCallback {
                         AggregateFunctionType.count,
                         AggregateFunctionType.avg));
 
-        for(int i=0;i<1000000;i++) {
+        for (int i = 0; i < 1000000; i++) {
             AggregateResult result1 = new AggregateResult();
             result1.setStartTimestamp(System.currentTimeMillis());
-            List<GroupbyKeyValue> keyValues = new ArrayList<GroupbyKeyValue>();
+            final List<GroupbyKeyValue> keyValues = new ArrayList<GroupbyKeyValue>();
 
             // <a,b> - <1*3, 2*3, 3*3, 4*3>
             GroupbyKey key = new GroupbyKey();

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/21187b55/eagle-core/eagle-query/eagle-storage-hbase/src/test/java/org/apache/eagle/storage/hbase/aggregate/coprocessor/TestGroupAggregateClient.java
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-query/eagle-storage-hbase/src/test/java/org/apache/eagle/storage/hbase/aggregate/coprocessor/TestGroupAggregateClient.java b/eagle-core/eagle-query/eagle-storage-hbase/src/test/java/org/apache/eagle/storage/hbase/aggregate/coprocessor/TestGroupAggregateClient.java
index 2a9ed0d..86c08fd 100755
--- a/eagle-core/eagle-query/eagle-storage-hbase/src/test/java/org/apache/eagle/storage/hbase/aggregate/coprocessor/TestGroupAggregateClient.java
+++ b/eagle-core/eagle-query/eagle-storage-hbase/src/test/java/org/apache/eagle/storage/hbase/aggregate/coprocessor/TestGroupAggregateClient.java
@@ -31,10 +31,7 @@ import org.apache.hadoop.hbase.client.HTableInterface;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.io.BytesWritable;
 import org.apache.hadoop.io.DoubleWritable;
-import org.junit.After;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Ignore;
+import org.junit.*;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -55,201 +52,201 @@ import org.apache.eagle.service.hbase.TestHBaseBase;
  */
 @Ignore
 public class TestGroupAggregateClient extends TestHBaseBase {
-	HTableInterface table;
-	long startTime;
-	long endTime;
-	List<String> rowkeys;
-	AggregateClient client;
-	Scan scan;
-	int num = 200;
-
-	private final static Logger LOG = LoggerFactory.getLogger(TestGroupAggregateClient.class);
-
-	@Before
-	public void setUp(){
-		hbase.createTable("unittest", "f");
-		startTime = System.currentTimeMillis();
-		try {
-			rowkeys = prepareData(num);
-		} catch (Exception e) {
-			e.printStackTrace();
-			Assert.fail(e.getMessage());
-		}
-		endTime = System.currentTimeMillis();
-		table = EagleConfigFactory.load().getHTable("unittest");
-		client = new AggregateClientImpl();
-		scan = new Scan();
-		scan.setCaching(200);
-		
-		ListQueryCompiler compiler = null;
-		try {
-			compiler = new ListQueryCompiler("TestLogAPIEntity[@cluster=\"test4UT\" and @datacenter=\"dc1\"]{@field1,@field2}");
-		} catch (Exception e) {
-			Assert.fail(e.getMessage());
-		}
-		scan.setFilter(compiler.filter());
-	}
-	
-	@After
-	public void shutdown(){
-		try {
-			hbase.deleteTable("unittest");
-			new HTableFactory().releaseHTableInterface(table);
-		} catch (IOException e) {
-			LOG.error(e.getMessage(),e);
-		}
-	}
-	
-	private List<String> prepareData(int count) throws Exception {
-		List<TaggedLogAPIEntity> list = new ArrayList<TaggedLogAPIEntity>();
-		EntityDefinition ed = EntityDefinitionManager.getEntityDefinitionByEntityClass(TestLogAPIEntity.class);
-
-		if (ed == null) {
-			EntityDefinitionManager.registerEntity(TestLogAPIEntity.class);
-			ed = EntityDefinitionManager.getEntityDefinitionByEntityClass(TestLogAPIEntity.class);
-		}
-		ed.setTimeSeries(true);
-		for(int i=0;i<count;i++){
-			TestLogAPIEntity e = new TestLogAPIEntity();
-			e.setTimestamp(System.currentTimeMillis());
-			e.setField1(1);
-			e.setField2(2);
-			e.setField3(3);
-			e.setField4(4L);
-			e.setField5(5.0);
-			e.setField6(5.0);
-			e.setField7("7");
-			e.setTags(new HashMap<String, String>());
-			e.getTags().put("cluster", "test4UT");
-			e.getTags().put("datacenter", "dc1");
-			e.getTags().put("index", ""+i);
-			e.getTags().put("jobId", "job_"+System.currentTimeMillis());
-			list.add(e);
-
-		}
-		GenericEntityWriter writer = new GenericEntityWriter(ed.getService());
-		LOG.info("Writing "+list.size()+" TestLogAPIEntity entities");
-		List<String> result = writer.write(list);
-		LOG.info("Finish writing test entities");
-		return result;
-	}
-
-	//@Test
-	public void testGroupAggregateCountClient(){
-		try {
-			EntityDefinition ed = EntityDefinitionManager.getEntityByServiceName("TestLogAPIEntity");
-			List<GroupbyKeyValue> result = client.aggregate(table,ed,scan, Arrays.asList("cluster","datacenter"),Arrays.asList(AggregateFunctionType.count),Arrays.asList("field2")).getKeyValues();
-			if(LOG.isDebugEnabled()) LOG.debug("COUNT");
-			logGroupbyKeyValue(result);
-			Assert.assertNotNull(result);
-			Assert.assertTrue(result.size()>0);
-		} catch (Exception e) {
-			e.printStackTrace();
-			Assert.fail(e.getMessage());
-		}
-	}
-
-	//@Test
-	public void testGroupAggregateAvgClient(){
-		try {
-			EntityDefinition ed = EntityDefinitionManager.getEntityByServiceName("TestLogAPIEntity");
-			List<GroupbyKeyValue> result = client.aggregate(table,ed,scan, Arrays.asList("cluster","datacenter"),Arrays.asList(AggregateFunctionType.avg),Arrays.asList("field2")).getKeyValues();
-			if(LOG.isDebugEnabled()) LOG.debug("AVG");
-			logGroupbyKeyValue(result);
-			Assert.assertNotNull(result);
-			Assert.assertTrue(result.size()>0);
-		} catch (Exception e) {
-			e.printStackTrace();
-			Assert.fail(e.getMessage());
-		}
-	}
-
-	//@Test
-	public void testGroupAggregateMaxClient(){
-		try {
-			EntityDefinition ed = EntityDefinitionManager.getEntityByServiceName("TestLogAPIEntity");
-			List<GroupbyKeyValue> result = client.aggregate(table,ed,scan, Arrays.asList("cluster","datacenter"),Arrays.asList(AggregateFunctionType.max),Arrays.asList("field1")).getKeyValues();
-			if(LOG.isDebugEnabled()) LOG.debug("MAX");
-			logGroupbyKeyValue(result);
-			Assert.assertNotNull(result);
-			Assert.assertTrue(result.size()>0);
-		} catch (Exception e) {
-			e.printStackTrace();
-			Assert.fail(e.getMessage());
-		}
-	}
-
-	//@Test
-	public void testGroupAggregateSumClient(){
-		try {
-			EntityDefinition ed = EntityDefinitionManager.getEntityByServiceName("TestLogAPIEntity");
-			List<GroupbyKeyValue> result = client.aggregate(table,ed,scan, Arrays.asList("cluster","datacenter"),Arrays.asList(AggregateFunctionType.sum),Arrays.asList("field2")).getKeyValues();
-			if(LOG.isDebugEnabled()) LOG.debug("MAX");
-			logGroupbyKeyValue(result);
-			Assert.assertNotNull(result);
-			Assert.assertTrue(result.size()>0);
-		} catch (Exception e) {
-			e.printStackTrace();
-			Assert.fail(e.getMessage());
-		}
-	}
-
-	//@Test
-	public void testGroupAggregateMinClient(){
-
-		try {
-			EntityDefinition ed = EntityDefinitionManager.getEntityByServiceName("TestLogAPIEntity");
-			List<GroupbyKeyValue> result = client.aggregate(table,ed,scan, Arrays.asList("cluster","datacenter"),Arrays.asList(AggregateFunctionType.min),Arrays.asList("field2")).getKeyValues();
-			if(LOG.isDebugEnabled()) LOG.debug("MIN");
-			logGroupbyKeyValue(result);
-			Assert.assertNotNull(result);
-			Assert.assertTrue(result.size()>0);
-		} catch (Exception e) {
-			e.printStackTrace();
-			Assert.fail(e.getMessage());
-		}
-	}
-
-	//@Test
-	public void testGroupAggregateMultipleClient(){
-		try {
-			EntityDefinition ed = EntityDefinitionManager.getEntityByServiceName("TestLogAPIEntity");
-			List<GroupbyKeyValue> result = client.aggregate(table,ed,scan, Arrays.asList("cluster","datacenter"),
-					Arrays.asList(AggregateFunctionType.min,
-							AggregateFunctionType.max,
-							AggregateFunctionType.avg,
-							AggregateFunctionType.count,
-							AggregateFunctionType.sum),
-					Arrays.asList("field2","field2","field2","field2","field2")).getKeyValues();
-			logGroupbyKeyValue(result);
-			Assert.assertNotNull(result);
-			Assert.assertTrue(result.size() > 0);
-			Assert.assertEquals("test4UT", new String(result.get(0).getKey().getValue().get(0).copyBytes()));
-			Assert.assertEquals("dc1", new String(result.get(0).getKey().getValue().get(1).copyBytes()));
-			Assert.assertEquals(2.0, result.get(0).getValue().get(0).get(), 0.00001);
-			Assert.assertEquals(2.0, result.get(0).getValue().get(1).get(), 0.00001);
-			Assert.assertEquals(2.0, result.get(0).getValue().get(2).get(), 0.00001);
-			Assert.assertTrue(num <= result.get(0).getValue().get(3).get());
-			Assert.assertTrue(2.0 * num <= result.get(0).getValue().get(4).get());
-		} catch (Exception e) {
-			e.printStackTrace();
-			Assert.fail(e.getMessage());
-		}
-	}
-
-	private void logGroupbyKeyValue(List<GroupbyKeyValue> keyValues){
-		for(GroupbyKeyValue keyValue:keyValues){
-			GroupbyKey key = keyValue.getKey();
-			List<String> keys = new ArrayList<String>();
-			for(BytesWritable bytes:key.getValue()){
-				keys.add(new String(bytes.copyBytes()));
-			}
-			List<Double> vals = new ArrayList<Double>();
-			GroupbyValue val = keyValue.getValue();
-			for(DoubleWritable dw:val.getValue()){
-				vals.add(dw.get());
-			}
-			if(LOG.isDebugEnabled()) LOG.debug("KEY: "+keys+", VALUE: "+vals);
-		}
-	}
+    HTableInterface table;
+    long startTime;
+    long endTime;
+    List<String> rowkeys;
+    AggregateClient client;
+    Scan scan;
+    int num = 200;
+
+    private final static Logger LOG = LoggerFactory.getLogger(TestGroupAggregateClient.class);
+
+    @Before
+    public void setUp() {
+        hbase.createTable("unittest", "f");
+        startTime = System.currentTimeMillis();
+        try {
+            rowkeys = prepareData(num);
+        } catch (Exception e) {
+            e.printStackTrace();
+            Assert.fail(e.getMessage());
+        }
+        endTime = System.currentTimeMillis();
+        table = EagleConfigFactory.load().getHTable("unittest");
+        client = new AggregateClientImpl();
+        scan = new Scan();
+        scan.setCaching(200);
+
+        ListQueryCompiler compiler = null;
+        try {
+            compiler = new ListQueryCompiler("TestLogAPIEntity[@cluster=\"test4UT\" and @datacenter=\"dc1\"]{@field1,@field2}");
+        } catch (Exception e) {
+            Assert.fail(e.getMessage());
+        }
+        scan.setFilter(compiler.filter());
+    }
+
+    @After
+    public void shutdown() {
+        try {
+            hbase.deleteTable("unittest");
+            new HTableFactory().releaseHTableInterface(table);
+        } catch (IOException e) {
+            LOG.error(e.getMessage(), e);
+        }
+    }
+
+    private List<String> prepareData(int count) throws Exception {
+        List<TaggedLogAPIEntity> list = new ArrayList<TaggedLogAPIEntity>();
+        EntityDefinition ed = EntityDefinitionManager.getEntityDefinitionByEntityClass(TestLogAPIEntity.class);
+
+        if (ed == null) {
+            EntityDefinitionManager.registerEntity(TestLogAPIEntity.class);
+            ed = EntityDefinitionManager.getEntityDefinitionByEntityClass(TestLogAPIEntity.class);
+        }
+        ed.setTimeSeries(true);
+        for (int i = 0; i < count; i++) {
+            TestLogAPIEntity e = new TestLogAPIEntity();
+            e.setTimestamp(System.currentTimeMillis());
+            e.setField1(1);
+            e.setField2(2);
+            e.setField3(3);
+            e.setField4(4L);
+            e.setField5(5.0);
+            e.setField6(5.0);
+            e.setField7("7");
+            e.setTags(new HashMap<String, String>());
+            e.getTags().put("cluster", "test4UT");
+            e.getTags().put("datacenter", "dc1");
+            e.getTags().put("index", "" + i);
+            e.getTags().put("jobId", "job_" + System.currentTimeMillis());
+            list.add(e);
+
+        }
+        GenericEntityWriter writer = new GenericEntityWriter(ed.getService());
+        LOG.info("Writing " + list.size() + " TestLogAPIEntity entities");
+        List<String> result = writer.write(list);
+        LOG.info("Finish writing test entities");
+        return result;
+    }
+
+    @Test
+    public void testGroupAggregateCountClient() {
+        try {
+            EntityDefinition ed = EntityDefinitionManager.getEntityByServiceName("TestLogAPIEntity");
+            List<GroupbyKeyValue> result = client.aggregate(table, ed, scan, Arrays.asList("cluster", "datacenter"), Arrays.asList(AggregateFunctionType.count), Arrays.asList("field2")).getKeyValues();
+            if (LOG.isDebugEnabled()) LOG.debug("COUNT");
+            logGroupbyKeyValue(result);
+            Assert.assertNotNull(result);
+            Assert.assertTrue(result.size() > 0);
+        } catch (Exception e) {
+            e.printStackTrace();
+            Assert.fail(e.getMessage());
+        }
+    }
+
+    @Test
+    public void testGroupAggregateAvgClient() {
+        try {
+            EntityDefinition ed = EntityDefinitionManager.getEntityByServiceName("TestLogAPIEntity");
+            List<GroupbyKeyValue> result = client.aggregate(table, ed, scan, Arrays.asList("cluster", "datacenter"), Arrays.asList(AggregateFunctionType.avg), Arrays.asList("field2")).getKeyValues();
+            if (LOG.isDebugEnabled()) LOG.debug("AVG");
+            logGroupbyKeyValue(result);
+            Assert.assertNotNull(result);
+            Assert.assertTrue(result.size() > 0);
+        } catch (Exception e) {
+            e.printStackTrace();
+            Assert.fail(e.getMessage());
+        }
+    }
+
+    @Test
+    public void testGroupAggregateMaxClient() {
+        try {
+            EntityDefinition ed = EntityDefinitionManager.getEntityByServiceName("TestLogAPIEntity");
+            List<GroupbyKeyValue> result = client.aggregate(table, ed, scan, Arrays.asList("cluster", "datacenter"), Arrays.asList(AggregateFunctionType.max), Arrays.asList("field1")).getKeyValues();
+            if (LOG.isDebugEnabled()) LOG.debug("MAX");
+            logGroupbyKeyValue(result);
+            Assert.assertNotNull(result);
+            Assert.assertTrue(result.size() > 0);
+        } catch (Exception e) {
+            e.printStackTrace();
+            Assert.fail(e.getMessage());
+        }
+    }
+
+    @Test
+    public void testGroupAggregateSumClient() {
+        try {
+            EntityDefinition ed = EntityDefinitionManager.getEntityByServiceName("TestLogAPIEntity");
+            List<GroupbyKeyValue> result = client.aggregate(table, ed, scan, Arrays.asList("cluster", "datacenter"), Arrays.asList(AggregateFunctionType.sum), Arrays.asList("field2")).getKeyValues();
+            if (LOG.isDebugEnabled()) LOG.debug("MAX");
+            logGroupbyKeyValue(result);
+            Assert.assertNotNull(result);
+            Assert.assertTrue(result.size() > 0);
+        } catch (Exception e) {
+            e.printStackTrace();
+            Assert.fail(e.getMessage());
+        }
+    }
+
+    @Test
+    public void testGroupAggregateMinClient() {
+
+        try {
+            EntityDefinition ed = EntityDefinitionManager.getEntityByServiceName("TestLogAPIEntity");
+            List<GroupbyKeyValue> result = client.aggregate(table, ed, scan, Arrays.asList("cluster", "datacenter"), Arrays.asList(AggregateFunctionType.min), Arrays.asList("field2")).getKeyValues();
+            if (LOG.isDebugEnabled()) LOG.debug("MIN");
+            logGroupbyKeyValue(result);
+            Assert.assertNotNull(result);
+            Assert.assertTrue(result.size() > 0);
+        } catch (Exception e) {
+            e.printStackTrace();
+            Assert.fail(e.getMessage());
+        }
+    }
+
+    @Test
+    public void testGroupAggregateMultipleClient() {
+        try {
+            EntityDefinition ed = EntityDefinitionManager.getEntityByServiceName("TestLogAPIEntity");
+            List<GroupbyKeyValue> result = client.aggregate(table, ed, scan, Arrays.asList("cluster", "datacenter"),
+                    Arrays.asList(AggregateFunctionType.min,
+                            AggregateFunctionType.max,
+                            AggregateFunctionType.avg,
+                            AggregateFunctionType.count,
+                            AggregateFunctionType.sum),
+                    Arrays.asList("field2", "field2", "field2", "field2", "field2")).getKeyValues();
+            logGroupbyKeyValue(result);
+            Assert.assertNotNull(result);
+            Assert.assertTrue(result.size() > 0);
+            Assert.assertEquals("test4UT", new String(result.get(0).getKey().getValue().get(0).copyBytes()));
+            Assert.assertEquals("dc1", new String(result.get(0).getKey().getValue().get(1).copyBytes()));
+            Assert.assertEquals(2.0, result.get(0).getValue().get(0).get(), 0.00001);
+            Assert.assertEquals(2.0, result.get(0).getValue().get(1).get(), 0.00001);
+            Assert.assertEquals(2.0, result.get(0).getValue().get(2).get(), 0.00001);
+            Assert.assertTrue(num <= result.get(0).getValue().get(3).get());
+            Assert.assertTrue(2.0 * num <= result.get(0).getValue().get(4).get());
+        } catch (Exception e) {
+            e.printStackTrace();
+            Assert.fail(e.getMessage());
+        }
+    }
+
+    private void logGroupbyKeyValue(List<GroupbyKeyValue> keyValues) {
+        for (GroupbyKeyValue keyValue : keyValues) {
+            GroupbyKey key = keyValue.getKey();
+            List<String> keys = new ArrayList<String>();
+            for (BytesWritable bytes : key.getValue()) {
+                keys.add(new String(bytes.copyBytes()));
+            }
+            List<Double> vals = new ArrayList<Double>();
+            GroupbyValue val = keyValue.getValue();
+            for (DoubleWritable dw : val.getValue()) {
+                vals.add(dw.get());
+            }
+            if (LOG.isDebugEnabled()) LOG.debug("KEY: " + keys + ", VALUE: " + vals);
+        }
+    }
 }
\ No newline at end of file


[36/52] [abbrv] incubator-eagle git commit: [EAGLE-489] loosen javadoc restriction to get rid of javadoc failures of deploying artifacts

Posted by yo...@apache.org.
[EAGLE-489] loosen javadoc restriction to get rid of javadoc failures of deploying artifacts

Author: anyway1021 <mw...@apache.org>

Closes #405 from anyway1021/EAGLE-489.


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

Branch: refs/heads/master
Commit: b66e27b909156644b68f4ec12e28565ce3dc8ee4
Parents: 4a15e18
Author: anyway1021 <mw...@apache.org>
Authored: Mon Sep 5 16:08:10 2016 +0800
Committer: anyway1021 <mw...@apache.org>
Committed: Mon Sep 5 16:08:10 2016 +0800

----------------------------------------------------------------------
 pom.xml | 7 +++++++
 1 file changed, 7 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/b66e27b9/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 1d4f64f..2a5b88a 100755
--- a/pom.xml
+++ b/pom.xml
@@ -1238,6 +1238,13 @@
                     </execution>
                 </executions>
             </plugin>
+            <plugin>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-javadoc-plugin</artifactId>
+                <configuration>
+                    <additionalparam>-Xdoclint:none</additionalparam>
+                </configuration>
+            </plugin>
         </plugins>
     </build>
     <repositories>


[15/52] [abbrv] incubator-eagle git commit: [minor] Change resourceFetch to resourcefetch

Posted by yo...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/b5fd782c/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/MRTaskAttempts.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/MRTaskAttempts.java b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/MRTaskAttempts.java
deleted file mode 100644
index 03af0d7..0000000
--- a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/MRTaskAttempts.java
+++ /dev/null
@@ -1,39 +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.eagle.jpm.util.resourcefetch.model;
-
-import org.codehaus.jackson.annotate.JsonIgnoreProperties;
-import org.codehaus.jackson.map.annotate.JsonSerialize;
-
-import java.util.List;
-
-@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
-@JsonIgnoreProperties(ignoreUnknown = true)
-public class MRTaskAttempts {
-    public List<MRTaskAttempt> getTaskAttempt() {
-        return taskAttempt;
-    }
-
-    public void setTaskAttempt(List<MRTaskAttempt> taskAttempt) {
-        this.taskAttempt = taskAttempt;
-    }
-
-    private List<MRTaskAttempt> taskAttempt;
-}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/b5fd782c/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/MRTasks.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/MRTasks.java b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/MRTasks.java
deleted file mode 100644
index b23b23d..0000000
--- a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/MRTasks.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.eagle.jpm.util.resourcefetch.model;
-
-import org.codehaus.jackson.annotate.JsonIgnoreProperties;
-import org.codehaus.jackson.map.annotate.JsonSerialize;
-
-import java.util.List;
-
-@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
-@JsonIgnoreProperties(ignoreUnknown = true)
-public class MRTasks {
-    public List<MRTask> getTask() {
-        return task;
-    }
-
-    public void setTask(List<MRTask> task) {
-        this.task = task;
-    }
-
-    private List<MRTask> task;
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/b5fd782c/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/MRTasksWrapper.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/MRTasksWrapper.java b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/MRTasksWrapper.java
deleted file mode 100644
index f92234a..0000000
--- a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/MRTasksWrapper.java
+++ /dev/null
@@ -1,37 +0,0 @@
-/*
- *
- *  * Licensed to the Apache Software Foundation (ASF) under one or more
- *  * contributor license agreements.  See the NOTICE file distributed with
- *  * this work for additional information regarding copyright ownership.
- *  * The ASF licenses this file to You under the Apache License, Version 2.0
- *  * (the "License"); you may not use this file except in compliance with
- *  * the License.  You may obtain a copy of the License at
- *  *
- *  *    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.eagle.jpm.util.resourcefetch.model;
-
-import org.codehaus.jackson.annotate.JsonIgnoreProperties;
-import org.codehaus.jackson.map.annotate.JsonSerialize;
-
-@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
-@JsonIgnoreProperties(ignoreUnknown = true)
-public class MRTasksWrapper {
-    public MRTasks getTasks() {
-        return tasks;
-    }
-
-    public void setTasks(MRTasks tasks) {
-        this.tasks = tasks;
-    }
-
-    private MRTasks tasks;
-}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/b5fd782c/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/MrJobs.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/MrJobs.java b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/MrJobs.java
deleted file mode 100644
index 1d4871c..0000000
--- a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/MrJobs.java
+++ /dev/null
@@ -1,39 +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.eagle.jpm.util.resourcefetch.model;
-
-import org.codehaus.jackson.annotate.JsonIgnoreProperties;
-import org.codehaus.jackson.map.annotate.JsonSerialize;
-
-import java.util.List;
-
-@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
-@JsonIgnoreProperties(ignoreUnknown = true)
-public class MrJobs {
-    public List<MRJob> getJob() {
-        return job;
-    }
-
-    public void setJobs(List<MRJob> job) {
-        this.job = job;
-    }
-
-    private List<MRJob> job;
-}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/b5fd782c/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/SparkApplication.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/SparkApplication.java b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/SparkApplication.java
deleted file mode 100644
index 299934e..0000000
--- a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/SparkApplication.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.eagle.jpm.util.resourcefetch.model;
-
-import org.codehaus.jackson.annotate.JsonIgnoreProperties;
-import org.codehaus.jackson.map.annotate.JsonSerialize;
-
-import java.util.List;
-
-@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
-@JsonIgnoreProperties(ignoreUnknown = true)
-public class SparkApplication {
-    String id;
-    String name;
-    List<SparkApplicationAttempt> attempts;
-
-    public String getId() {
-        return id;
-    }
-
-    public void setId(String id) {
-        this.id = id;
-    }
-
-    public String getName() {
-        return name;
-    }
-
-    public void setName(String name) {
-        this.name = name;
-    }
-
-    public List<SparkApplicationAttempt> getAttempts() {
-        return attempts;
-    }
-
-    public void setAttempts(List<SparkApplicationAttempt> attempts) {
-        this.attempts = attempts;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/b5fd782c/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/SparkApplicationAttempt.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/SparkApplicationAttempt.java b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/SparkApplicationAttempt.java
deleted file mode 100644
index b437cc2..0000000
--- a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/SparkApplicationAttempt.java
+++ /dev/null
@@ -1,73 +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.eagle.jpm.util.resourcefetch.model;
-
-import org.codehaus.jackson.annotate.JsonIgnoreProperties;
-import org.codehaus.jackson.map.annotate.JsonSerialize;
-
-@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
-@JsonIgnoreProperties(ignoreUnknown = true)
-public class SparkApplicationAttempt {
-    String attemptId;
-    String sparkUser;
-    String startTime;
-    String endTime;
-    boolean completed;
-
-    public String getAttemptId() {
-        return attemptId;
-    }
-
-    public void setAttemptId(String attemptId) {
-        this.attemptId = attemptId;
-    }
-
-    public String getSparkUser() {
-        return sparkUser;
-    }
-
-    public void setSparkUser(String sparkUser) {
-        this.sparkUser = sparkUser;
-    }
-
-    public String getStartTime() {
-        return startTime;
-    }
-
-    public void setStartTime(String startTime) {
-        this.startTime = startTime;
-    }
-
-    public String getEndTime() {
-        return endTime;
-    }
-
-    public void setEndTime(String endTime) {
-        this.endTime = endTime;
-    }
-
-    public boolean isCompleted() {
-        return completed;
-    }
-
-    public void setCompleted(boolean completed) {
-        this.completed = completed;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/b5fd782c/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/SparkApplicationWrapper.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/SparkApplicationWrapper.java b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/SparkApplicationWrapper.java
deleted file mode 100644
index 0b04f31..0000000
--- a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/SparkApplicationWrapper.java
+++ /dev/null
@@ -1,38 +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.eagle.jpm.util.resourcefetch.model;
-
-import org.codehaus.jackson.annotate.JsonIgnoreProperties;
-import org.codehaus.jackson.map.annotate.JsonSerialize;
-
-@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
-@JsonIgnoreProperties(ignoreUnknown = true)
-public class SparkApplicationWrapper {
-
-    SparkApplication app;
-
-    public SparkApplication getApp() {
-        return app;
-    }
-
-    public void setApp(SparkApplication app) {
-        this.app = app;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/b5fd782c/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/SparkExecutor.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/SparkExecutor.java b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/SparkExecutor.java
deleted file mode 100644
index c2a2115..0000000
--- a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/SparkExecutor.java
+++ /dev/null
@@ -1,155 +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.eagle.jpm.util.resourcefetch.model;
-
-import org.codehaus.jackson.annotate.JsonIgnoreProperties;
-import org.codehaus.jackson.map.annotate.JsonSerialize;
-
-@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
-@JsonIgnoreProperties(ignoreUnknown = true)
-public class SparkExecutor {
-    private String id;
-    private String hostPort;
-    private int rddBlocks;
-    private long memoryUsed;
-    private long diskUsed;
-    private int activeTasks;
-    private int failedTasks;
-    private int completedTasks;
-    private int totalTasks;
-    private long totalDuration;
-    private long totalInputBytes;
-    private long totalShuffleRead;
-    private long totalShuffleWrite;
-    private long maxMemory;
-
-    public String getId() {
-        return id;
-    }
-
-    public void setId(String id) {
-        this.id = id;
-    }
-
-    public String getHostPort() {
-        return hostPort;
-    }
-
-    public void setHostPort(String hostPort) {
-        this.hostPort = hostPort;
-    }
-
-    public int getRddBlocks() {
-        return rddBlocks;
-    }
-
-    public void setRddBlocks(int rddBlocks) {
-        this.rddBlocks = rddBlocks;
-    }
-
-    public long getMemoryUsed() {
-        return memoryUsed;
-    }
-
-    public void setMemoryUsed(long memoryUsed) {
-        this.memoryUsed = memoryUsed;
-    }
-
-    public long getDiskUsed() {
-        return diskUsed;
-    }
-
-    public void setDiskUsed(long diskUsed) {
-        this.diskUsed = diskUsed;
-    }
-
-    public int getActiveTasks() {
-        return activeTasks;
-    }
-
-    public void setActiveTasks(int activeTasks) {
-        this.activeTasks = activeTasks;
-    }
-
-    public int getFailedTasks() {
-        return failedTasks;
-    }
-
-    public void setFailedTasks(int failedTasks) {
-        this.failedTasks = failedTasks;
-    }
-
-    public int getCompletedTasks() {
-        return completedTasks;
-    }
-
-    public void setCompletedTasks(int completedTasks) {
-        this.completedTasks = completedTasks;
-    }
-
-    public int getTotalTasks() {
-        return totalTasks;
-    }
-
-    public void setTotalTasks(int totalTasks) {
-        this.totalTasks = totalTasks;
-    }
-
-    public long getTotalDuration() {
-        return totalDuration;
-    }
-
-    public void setTotalDuration(long totalDuration) {
-        this.totalDuration = totalDuration;
-    }
-
-    public long getTotalInputBytes() {
-        return totalInputBytes;
-    }
-
-    public void setTotalInputBytes(long totalInputBytes) {
-        this.totalInputBytes = totalInputBytes;
-    }
-
-    public long getTotalShuffleRead() {
-        return totalShuffleRead;
-    }
-
-    public void setTotalShuffleRead(long totalShuffleRead) {
-        this.totalShuffleRead = totalShuffleRead;
-    }
-
-    public long getTotalShuffleWrite() {
-        return totalShuffleWrite;
-    }
-
-    public void setTotalShuffleWrite(long totalShuffleWrite) {
-        this.totalShuffleWrite = totalShuffleWrite;
-    }
-
-    public long getMaxMemory() {
-        return maxMemory;
-    }
-
-    public void setMaxMemory(long maxMemory) {
-        this.maxMemory = maxMemory;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/b5fd782c/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/SparkJob.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/SparkJob.java b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/SparkJob.java
deleted file mode 100644
index 280cf67..0000000
--- a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/SparkJob.java
+++ /dev/null
@@ -1,165 +0,0 @@
-/*
- *
- *  * Licensed to the Apache Software Foundation (ASF) under one or more
- *  * contributor license agreements.  See the NOTICE file distributed with
- *  * this work for additional information regarding copyright ownership.
- *  * The ASF licenses this file to You under the Apache License, Version 2.0
- *  * (the "License"); you may not use this file except in compliance with
- *  * the License.  You may obtain a copy of the License at
- *  *
- *  *    http://www.apache.org/licenses/LICENSE-2.0
- *  *
- *  * Unless required by applicable law or agreed to in writing, software
- *  * distributed under the License is distributed on an "AS IS" BASIS,
- *  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- *  * See the License for the specific language governing permissions and
- *  * limitations under the License.
- *
- */
-
-package org.apache.eagle.jpm.util.resourcefetch.model;
-
-import org.codehaus.jackson.annotate.JsonIgnoreProperties;
-import org.codehaus.jackson.map.annotate.JsonSerialize;
-
-import java.util.List;
-
-@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
-@JsonIgnoreProperties(ignoreUnknown = true)
-public class SparkJob {
-    private int jobId;
-    private String name;
-    private String submissionTime;
-    private String completionTime;
-    private List<Integer> stageIds;
-    private String status;
-    private int numTasks;
-    private int numActiveTasks;
-    private int numCompletedTasks;
-    private int numSkippedTasks;
-    private int numFailedTasks;
-    private int numActiveStages;
-    private int numCompletedStages;
-    private int numSkippedStages;
-    private int numFailedStages;
-
-    public int getJobId() {
-        return jobId;
-    }
-
-    public void setJobId(int jobId) {
-        this.jobId = jobId;
-    }
-
-    public String getName() {
-        return name;
-    }
-
-    public void setName(String name) {
-        this.name = name;
-    }
-
-    public String getSubmissionTime() {
-        return submissionTime;
-    }
-
-    public void setSubmissionTime(String submissionTime) {
-        this.submissionTime = submissionTime;
-    }
-
-    public String getCompletionTime() {
-        return completionTime;
-    }
-
-    public void setCompletionTime(String completionTime) {
-        this.completionTime = completionTime;
-    }
-
-    public List<Integer> getStageIds() {
-        return stageIds;
-    }
-
-    public void setStageIds(List<Integer> stageIds) {
-        this.stageIds = stageIds;
-    }
-
-    public String getStatus() {
-        return status;
-    }
-
-    public void setStatus(String status) {
-        this.status = status;
-    }
-
-    public int getNumTasks() {
-        return numTasks;
-    }
-
-    public void setNumTasks(int numTasks) {
-        this.numTasks = numTasks;
-    }
-
-    public int getNumActiveTasks() {
-        return numActiveTasks;
-    }
-
-    public void setNumActiveTasks(int numActiveTasks) {
-        this.numActiveTasks = numActiveTasks;
-    }
-
-    public int getNumCompletedTasks() {
-        return numCompletedTasks;
-    }
-
-    public void setNumCompletedTasks(int numCompletedTasks) {
-        this.numCompletedTasks = numCompletedTasks;
-    }
-
-    public int getNumSkippedTasks() {
-        return numSkippedTasks;
-    }
-
-    public void setNumSkippedTasks(int numSkippedTasks) {
-        this.numSkippedTasks = numSkippedTasks;
-    }
-
-    public int getNumFailedTasks() {
-        return numFailedTasks;
-    }
-
-    public void setNumFailedTasks(int numFailedTasks) {
-        this.numFailedTasks = numFailedTasks;
-    }
-
-    public int getNumActiveStages() {
-        return numActiveStages;
-    }
-
-    public void setNumActiveStages(int numActiveStages) {
-        this.numActiveStages = numActiveStages;
-    }
-
-    public int getNumCompletedStages() {
-        return numCompletedStages;
-    }
-
-    public void setNumCompletedStages(int numCompletedStages) {
-        this.numCompletedStages = numCompletedStages;
-    }
-
-    public int getNumSkippedStages() {
-        return numSkippedStages;
-    }
-
-    public void setNumSkippedStages(int numSkippedStages) {
-        this.numSkippedStages = numSkippedStages;
-    }
-
-    public int getNumFailedStages() {
-        return numFailedStages;
-    }
-
-    public void setNumFailedStages(int numFailedStages) {
-        this.numFailedStages = numFailedStages;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/b5fd782c/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/SparkStage.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/SparkStage.java b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/SparkStage.java
deleted file mode 100644
index 14edeee..0000000
--- a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/SparkStage.java
+++ /dev/null
@@ -1,211 +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.eagle.jpm.util.resourcefetch.model;
-
-import org.codehaus.jackson.annotate.JsonIgnoreProperties;
-import org.codehaus.jackson.map.annotate.JsonSerialize;
-
-import java.util.Map;
-
-@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
-@JsonIgnoreProperties(ignoreUnknown = true)
-public class SparkStage {
-    private String status;
-    private int stageId;
-    private int attemptId;
-    private int numActiveTasks;
-    private int numCompleteTasks;
-    private int numFailedTasks;
-    private long executorRunTime;
-    private long inputBytes;
-    private long inputRecords;
-    private long outputBytes;
-    private long outputRecords;
-    private long shuffleReadBytes;
-    private long shuffleReadRecords;
-    private long shuffleWriteBytes;
-    private long shuffleWriteRecords;
-    private long memoryBytesSpilled;
-    private long diskBytesSpilled;
-    private String name;
-    private String schedulingPool;
-    //taskID
-    private Map<String, SparkTask> tasks;
-
-    public String getStatus() {
-        return status;
-    }
-
-    public void setStatus(String status) {
-        this.status = status;
-    }
-
-    public int getStageId() {
-        return stageId;
-    }
-
-    public void setStageId(int stageId) {
-        this.stageId = stageId;
-    }
-
-    public int getAttemptId() {
-        return attemptId;
-    }
-
-    public void setAttemptId(int attemptId) {
-        this.attemptId = attemptId;
-    }
-
-    public int getNumActiveTasks() {
-        return numActiveTasks;
-    }
-
-    public void setNumActiveTasks(int numActiveTasks) {
-        this.numActiveTasks = numActiveTasks;
-    }
-
-    public int getNumCompleteTasks() {
-        return numCompleteTasks;
-    }
-
-    public void setNumCompleteTasks(int numCompleteTasks) {
-        this.numCompleteTasks = numCompleteTasks;
-    }
-
-    public int getNumFailedTasks() {
-        return numFailedTasks;
-    }
-
-    public void setNumFailedTasks(int numFailedTasks) {
-        this.numFailedTasks = numFailedTasks;
-    }
-
-    public long getExecutorRunTime() {
-        return executorRunTime;
-    }
-
-    public void setExecutorRunTime(long executorRunTime) {
-        this.executorRunTime = executorRunTime;
-    }
-
-    public long getInputBytes() {
-        return inputBytes;
-    }
-
-    public void setInputBytes(long inputBytes) {
-        this.inputBytes = inputBytes;
-    }
-
-    public long getInputRecords() {
-        return inputRecords;
-    }
-
-    public void setInputRecords(long inputRecords) {
-        this.inputRecords = inputRecords;
-    }
-
-    public long getOutputBytes() {
-        return outputBytes;
-    }
-
-    public void setOutputBytes(long outputBytes) {
-        this.outputBytes = outputBytes;
-    }
-
-    public long getOutputRecords() {
-        return outputRecords;
-    }
-
-    public void setOutputRecords(long outputRecords) {
-        this.outputRecords = outputRecords;
-    }
-
-    public long getShuffleReadBytes() {
-        return shuffleReadBytes;
-    }
-
-    public void setShuffleReadBytes(long shuffleReadBytes) {
-        this.shuffleReadBytes = shuffleReadBytes;
-    }
-
-    public long getShuffleReadRecords() {
-        return shuffleReadRecords;
-    }
-
-    public void setShuffleReadRecords(long shuffleReadRecords) {
-        this.shuffleReadRecords = shuffleReadRecords;
-    }
-
-    public long getShuffleWriteBytes() {
-        return shuffleWriteBytes;
-    }
-
-    public void setShuffleWriteBytes(long shuffleWriteBytes) {
-        this.shuffleWriteBytes = shuffleWriteBytes;
-    }
-
-    public long getShuffleWriteRecords() {
-        return shuffleWriteRecords;
-    }
-
-    public void setShuffleWriteRecords(long shuffleWriteRecords) {
-        this.shuffleWriteRecords = shuffleWriteRecords;
-    }
-
-    public long getMemoryBytesSpilled() {
-        return memoryBytesSpilled;
-    }
-
-    public void setMemoryBytesSpilled(long memoryBytesSpilled) {
-        this.memoryBytesSpilled = memoryBytesSpilled;
-    }
-
-    public long getDiskBytesSpilled() {
-        return diskBytesSpilled;
-    }
-
-    public void setDiskBytesSpilled(long diskBytesSpilled) {
-        this.diskBytesSpilled = diskBytesSpilled;
-    }
-
-    public String getName() {
-        return name;
-    }
-
-    public void setName(String name) {
-        this.name = name;
-    }
-
-    public String getSchedulingPool() {
-        return schedulingPool;
-    }
-
-    public void setSchedulingPool(String schedulingPool) {
-        this.schedulingPool = schedulingPool;
-    }
-
-    public Map<String, SparkTask> getTasks() {
-        return tasks;
-    }
-
-    public void setTasks(Map<String, SparkTask> tasks) {
-        this.tasks = tasks;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/b5fd782c/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/SparkTask.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/SparkTask.java b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/SparkTask.java
deleted file mode 100644
index a5dd08f..0000000
--- a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/SparkTask.java
+++ /dev/null
@@ -1,111 +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.eagle.jpm.util.resourcefetch.model;
-
-
-import org.codehaus.jackson.annotate.JsonIgnoreProperties;
-import org.codehaus.jackson.map.annotate.JsonSerialize;
-
-@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
-@JsonIgnoreProperties(ignoreUnknown = true)
-public class SparkTask {
-    private int taskId;
-    private int index;
-    private int attempt;
-    private String launchTime;
-    private String executorId;
-    private String host;
-    private String taskLocality;
-    private boolean speculative;
-    private SparkTaskMetrics taskMetrics;
-
-    public int getTaskId() {
-        return taskId;
-    }
-
-    public void setTaskId(int taskId) {
-        this.taskId = taskId;
-    }
-
-    public int getIndex() {
-        return index;
-    }
-
-    public void setIndex(int index) {
-        this.index = index;
-    }
-
-    public int getAttempt() {
-        return attempt;
-    }
-
-    public void setAttempt(int attempt) {
-        this.attempt = attempt;
-    }
-
-    public String getLaunchTime() {
-        return launchTime;
-    }
-
-    public void setLaunchTime(String launchTime) {
-        this.launchTime = launchTime;
-    }
-
-    public String getExecutorId() {
-        return executorId;
-    }
-
-    public void setExecutorId(String executorId) {
-        this.executorId = executorId;
-    }
-
-    public String getHost() {
-        return host;
-    }
-
-    public void setHost(String host) {
-        this.host = host;
-    }
-
-    public String getTaskLocality() {
-        return taskLocality;
-    }
-
-    public void setTaskLocality(String taskLocality) {
-        this.taskLocality = taskLocality;
-    }
-
-    public boolean isSpeculative() {
-        return speculative;
-    }
-
-    public void setSpeculative(boolean speculative) {
-        this.speculative = speculative;
-    }
-
-    public SparkTaskMetrics getTaskMetrics() {
-        return taskMetrics;
-    }
-
-    public void setTaskMetrics(SparkTaskMetrics taskMetrics) {
-        this.taskMetrics = taskMetrics;
-    }
-}
-

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/b5fd782c/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/SparkTaskInputMetrics.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/SparkTaskInputMetrics.java b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/SparkTaskInputMetrics.java
deleted file mode 100644
index d079694..0000000
--- a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/SparkTaskInputMetrics.java
+++ /dev/null
@@ -1,46 +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.eagle.jpm.util.resourcefetch.model;
-
-import org.codehaus.jackson.annotate.JsonIgnoreProperties;
-import org.codehaus.jackson.map.annotate.JsonSerialize;
-
-@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
-@JsonIgnoreProperties(ignoreUnknown = true)
-public class SparkTaskInputMetrics {
-    private long bytesRead;
-    private long recordsRead;
-
-    public long getBytesRead() {
-        return bytesRead;
-    }
-
-    public void setBytesRead(long bytesRead) {
-        this.bytesRead = bytesRead;
-    }
-
-    public long getRecordsRead() {
-        return recordsRead;
-    }
-
-    public void setRecordsRead(long recordsRead) {
-        this.recordsRead = recordsRead;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/b5fd782c/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/SparkTaskMetrics.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/SparkTaskMetrics.java b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/SparkTaskMetrics.java
deleted file mode 100644
index cdb913d..0000000
--- a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/SparkTaskMetrics.java
+++ /dev/null
@@ -1,118 +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.eagle.jpm.util.resourcefetch.model;
-
-import org.codehaus.jackson.annotate.JsonIgnoreProperties;
-import org.codehaus.jackson.map.annotate.JsonSerialize;
-
-@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
-@JsonIgnoreProperties(ignoreUnknown = true)
-public class SparkTaskMetrics {
-    private long executorDeserializeTime;
-    private long executorRunTime;
-    private long resultSize;
-    private long jvmGcTime;
-    private long resultSerializationTime;
-    private long memoryBytesSpilled;
-    private long diskBytesSpilled;
-    private SparkTaskInputMetrics inputMetrics;
-    private SparkTaskShuffleWriteMetrics shuffleWriteMetrics;
-    private SparkTaskShuffleReadMetrics shuffleReadMetrics;
-
-    public long getExecutorDeserializeTime() {
-        return executorDeserializeTime;
-    }
-
-    public void setExecutorDeserializeTime(long executorDeserializeTime) {
-        this.executorDeserializeTime = executorDeserializeTime;
-    }
-
-    public long getExecutorRunTime() {
-        return executorRunTime;
-    }
-
-    public void setExecutorRunTime(long executorRunTime) {
-        this.executorRunTime = executorRunTime;
-    }
-
-    public long getResultSize() {
-        return resultSize;
-    }
-
-    public void setResultSize(long resultSize) {
-        this.resultSize = resultSize;
-    }
-
-    public long getJvmGcTime() {
-        return jvmGcTime;
-    }
-
-    public void setJvmGcTime(long jvmGcTime) {
-        this.jvmGcTime = jvmGcTime;
-    }
-
-    public long getResultSerializationTime() {
-        return resultSerializationTime;
-    }
-
-    public void setResultSerializationTime(long resultSerializationTime) {
-        this.resultSerializationTime = resultSerializationTime;
-    }
-
-    public long getMemoryBytesSpilled() {
-        return memoryBytesSpilled;
-    }
-
-    public void setMemoryBytesSpilled(long memoryBytesSpilled) {
-        this.memoryBytesSpilled = memoryBytesSpilled;
-    }
-
-    public long getDiskBytesSpilled() {
-        return diskBytesSpilled;
-    }
-
-    public void setDiskBytesSpilled(long diskBytesSpilled) {
-        this.diskBytesSpilled = diskBytesSpilled;
-    }
-
-    public SparkTaskInputMetrics getInputMetrics() {
-        return inputMetrics;
-    }
-
-    public void setInputMetrics(SparkTaskInputMetrics inputMetrics) {
-        this.inputMetrics = inputMetrics;
-    }
-
-    public SparkTaskShuffleWriteMetrics getShuffleWriteMetrics() {
-        return shuffleWriteMetrics;
-    }
-
-    public void setShuffleWriteMetrics(SparkTaskShuffleWriteMetrics shuffleWriteMetrics) {
-        this.shuffleWriteMetrics = shuffleWriteMetrics;
-    }
-
-    public SparkTaskShuffleReadMetrics getShuffleReadMetrics() {
-        return shuffleReadMetrics;
-    }
-
-    public void setShuffleReadMetrics(SparkTaskShuffleReadMetrics shuffleReadMetrics) {
-        this.shuffleReadMetrics = shuffleReadMetrics;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/b5fd782c/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/SparkTaskShuffleReadMetrics.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/SparkTaskShuffleReadMetrics.java b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/SparkTaskShuffleReadMetrics.java
deleted file mode 100644
index 3aa94bb..0000000
--- a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/SparkTaskShuffleReadMetrics.java
+++ /dev/null
@@ -1,82 +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.eagle.jpm.util.resourcefetch.model;
-
-import org.codehaus.jackson.annotate.JsonIgnoreProperties;
-import org.codehaus.jackson.map.annotate.JsonSerialize;
-
-@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
-@JsonIgnoreProperties(ignoreUnknown = true)
-public class SparkTaskShuffleReadMetrics {
-    private long remoteBlocksFetched;
-    private long localBlocksFetched;
-    private long fetchWaitTime;
-    private long remoteBytesRead;
-    private long totalBlocksFetched;
-    private long recordsRead;
-
-    public long getRemoteBlocksFetched() {
-        return remoteBlocksFetched;
-    }
-
-    public void setRemoteBlocksFetched(long remoteBlocksFetched) {
-        this.remoteBlocksFetched = remoteBlocksFetched;
-    }
-
-    public long getLocalBlocksFetched() {
-        return localBlocksFetched;
-    }
-
-    public void setLocalBlocksFetched(long localBlocksFetched) {
-        this.localBlocksFetched = localBlocksFetched;
-    }
-
-    public long getFetchWaitTime() {
-        return fetchWaitTime;
-    }
-
-    public void setFetchWaitTime(long fetchWaitTime) {
-        this.fetchWaitTime = fetchWaitTime;
-    }
-
-    public long getRemoteBytesRead() {
-        return remoteBytesRead;
-    }
-
-    public void setRemoteBytesRead(long remoteBytesRead) {
-        this.remoteBytesRead = remoteBytesRead;
-    }
-
-    public long getTotalBlocksFetched() {
-        return totalBlocksFetched;
-    }
-
-    public void setTotalBlocksFetched(long totalBlocksFetched) {
-        this.totalBlocksFetched = totalBlocksFetched;
-    }
-
-    public long getRecordsRead() {
-        return recordsRead;
-    }
-
-    public void setRecordsRead(long recordsRead) {
-        this.recordsRead = recordsRead;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/b5fd782c/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/SparkTaskShuffleWriteMetrics.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/SparkTaskShuffleWriteMetrics.java b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/SparkTaskShuffleWriteMetrics.java
deleted file mode 100644
index 3d8f025..0000000
--- a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/SparkTaskShuffleWriteMetrics.java
+++ /dev/null
@@ -1,55 +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.eagle.jpm.util.resourcefetch.model;
-
-import org.codehaus.jackson.annotate.JsonIgnoreProperties;
-import org.codehaus.jackson.map.annotate.JsonSerialize;
-
-@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
-@JsonIgnoreProperties(ignoreUnknown = true)
-public class SparkTaskShuffleWriteMetrics {
-    private long bytesWritten;
-    private long writeTime;
-    private long recordsWritten;
-
-    public long getBytesWritten() {
-        return bytesWritten;
-    }
-
-    public void setBytesWritten(long bytesWritten) {
-        this.bytesWritten = bytesWritten;
-    }
-
-    public long getWriteTime() {
-        return writeTime;
-    }
-
-    public void setWriteTime(long writeTime) {
-        this.writeTime = writeTime;
-    }
-
-    public long getRecordsWritten() {
-        return recordsWritten;
-    }
-
-    public void setRecordsWritten(long recordsWritten) {
-        this.recordsWritten = recordsWritten;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/b5fd782c/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/TaskCounterGroup.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/TaskCounterGroup.java b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/TaskCounterGroup.java
deleted file mode 100644
index 9803d11..0000000
--- a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/TaskCounterGroup.java
+++ /dev/null
@@ -1,46 +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.eagle.jpm.util.resourcefetch.model;
-
-import org.codehaus.jackson.annotate.JsonIgnoreProperties;
-import org.codehaus.jackson.map.annotate.JsonSerialize;
-
-import java.util.List;
-
-@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
-@JsonIgnoreProperties(ignoreUnknown = true)
-public class TaskCounterGroup {
-    public String getCounterGroupName() {
-        return counterGroupName;
-    }
-
-    public void setCounterGroupName(String counterGroupName) {
-        this.counterGroupName = counterGroupName;
-    }
-
-    public List<TaskCounterItem> getCounter() {
-        return counter;
-    }
-
-    public void setCounter(List<TaskCounterItem> counter) {
-        this.counter = counter;
-    }
-
-    private String counterGroupName;
-    private List<TaskCounterItem> counter;
-}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/b5fd782c/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/TaskCounterItem.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/TaskCounterItem.java b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/TaskCounterItem.java
deleted file mode 100644
index f8ea22c..0000000
--- a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/TaskCounterItem.java
+++ /dev/null
@@ -1,44 +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.eagle.jpm.util.resourcefetch.model;
-
-import org.codehaus.jackson.annotate.JsonIgnoreProperties;
-import org.codehaus.jackson.map.annotate.JsonSerialize;
-
-@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
-@JsonIgnoreProperties(ignoreUnknown = true)
-public class TaskCounterItem {
-    public String getName() {
-        return name;
-    }
-
-    public void setName(String name) {
-        this.name = name;
-    }
-
-    public long getValue() {
-        return value;
-    }
-
-    public void setValue(long value) {
-        this.value = value;
-    }
-
-    private String name;
-    private long value;
-}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/b5fd782c/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/TaskCounters.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/TaskCounters.java b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/TaskCounters.java
deleted file mode 100644
index ef99547..0000000
--- a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/TaskCounters.java
+++ /dev/null
@@ -1,46 +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.eagle.jpm.util.resourcefetch.model;
-
-import org.codehaus.jackson.annotate.JsonIgnoreProperties;
-import org.codehaus.jackson.map.annotate.JsonSerialize;
-
-import java.util.List;
-
-@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
-@JsonIgnoreProperties(ignoreUnknown = true)
-public class TaskCounters {
-    public String getId() {
-        return id;
-    }
-
-    public void setId(String id) {
-        this.id = id;
-    }
-
-    public List<TaskCounterGroup> getTaskCounterGroup() {
-        return taskCounterGroup;
-    }
-
-    public void setTaskCounterGroup(List<TaskCounterGroup> taskCounterGroup) {
-        this.taskCounterGroup = taskCounterGroup;
-    }
-
-    private String id;
-    private List<TaskCounterGroup> taskCounterGroup;
-}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/b5fd782c/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/TaskCountersWrapper.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/TaskCountersWrapper.java b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/TaskCountersWrapper.java
deleted file mode 100644
index 80808a3..0000000
--- a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/TaskCountersWrapper.java
+++ /dev/null
@@ -1,35 +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.eagle.jpm.util.resourcefetch.model;
-
-import org.codehaus.jackson.annotate.JsonIgnoreProperties;
-import org.codehaus.jackson.map.annotate.JsonSerialize;
-
-@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
-@JsonIgnoreProperties(ignoreUnknown = true)
-public class TaskCountersWrapper {
-    public TaskCounters getJobTaskCounters() {
-        return jobTaskCounters;
-    }
-
-    public void setJobTaskCounters(TaskCounters jobTaskCounters) {
-        this.jobTaskCounters = jobTaskCounters;
-    }
-
-    private TaskCounters jobTaskCounters;
-}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/b5fd782c/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/url/JobListServiceURLBuilderImpl.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/url/JobListServiceURLBuilderImpl.java b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/url/JobListServiceURLBuilderImpl.java
deleted file mode 100644
index 5513771..0000000
--- a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/url/JobListServiceURLBuilderImpl.java
+++ /dev/null
@@ -1,52 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.eagle.jpm.util.resourcefetch.url;
-
-import org.apache.eagle.jpm.util.Constants;
-
-public class JobListServiceURLBuilderImpl implements ServiceURLBuilder {
-
-    public String build(String... parameters) {
-        /**
-         * {rmUrl}/ws/v1/cluster/apps?state=RUNNING.
-         * We need to remove tailing slashes to avoid "url//ws/v1"
-         * because it would not be found and would be redirected to
-         * history server ui.
-         */
-        String rmUrl = URLUtil.removeTrailingSlash(parameters[0]);
-
-        String restApi = null;
-        String jobState = parameters[1];
-
-        if (jobState.equals(Constants.JobState.RUNNING.name())) {
-            restApi = Constants.V2_APPS_RUNNING_URL;
-        } else if (jobState.equals(Constants.JobState.FINISHED.name())) {
-            restApi = Constants.V2_APPS_COMPLETED_URL;
-        } else if (jobState.equals(Constants.JobState.ALL.name())) {
-            restApi = Constants.V2_APPS_URL;
-        }
-        if (restApi == null) {
-            return null;
-        }
-        // "/ws/v1/cluster/apps?state=RUNNING"
-        StringBuilder sb = new StringBuilder();
-        sb.append(rmUrl).append("/").append(restApi);
-        sb.append("&").append(Constants.ANONYMOUS_PARAMETER);
-
-        return sb.toString();
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/b5fd782c/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/url/ServiceURLBuilder.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/url/ServiceURLBuilder.java b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/url/ServiceURLBuilder.java
deleted file mode 100644
index 09fea2f..0000000
--- a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/url/ServiceURLBuilder.java
+++ /dev/null
@@ -1,21 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.eagle.jpm.util.resourcefetch.url;
-
-public interface ServiceURLBuilder {
-    String build(String... parameters);
-}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/b5fd782c/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/url/SparkCompleteJobServiceURLBuilderImpl.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/url/SparkCompleteJobServiceURLBuilderImpl.java b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/url/SparkCompleteJobServiceURLBuilderImpl.java
deleted file mode 100644
index ca6e938..0000000
--- a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/url/SparkCompleteJobServiceURLBuilderImpl.java
+++ /dev/null
@@ -1,33 +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.eagle.jpm.util.resourcefetch.url;
-
-import org.apache.eagle.jpm.util.Constants;
-
-public class SparkCompleteJobServiceURLBuilderImpl implements ServiceURLBuilder {
-
-    public String build(String... parameters) {
-        String url = URLUtil.removeTrailingSlash(parameters[0]);
-
-        return url + "/" + Constants.V2_APPS_URL
-                + "?applicationTypes=SPARK&state=FINISHED&finishedTimeBegin="
-                + parameters[1] + "&" + Constants.ANONYMOUS_PARAMETER;
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/b5fd782c/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/url/SparkJobServiceURLBuilderImpl.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/url/SparkJobServiceURLBuilderImpl.java b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/url/SparkJobServiceURLBuilderImpl.java
deleted file mode 100644
index c5ec67a..0000000
--- a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/url/SparkJobServiceURLBuilderImpl.java
+++ /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.
- *
- */
-
-package org.apache.eagle.jpm.util.resourcefetch.url;
-
-import org.apache.eagle.jpm.util.Constants;
-
-public class SparkJobServiceURLBuilderImpl implements ServiceURLBuilder {
-
-    public String build(String... parameters) {
-        String serverAddress = URLUtil.removeTrailingSlash(parameters[0]);
-
-        return serverAddress + Constants.SPARK_APPS_URL + parameters[1];
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/b5fd782c/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/url/URLUtil.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/url/URLUtil.java b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/url/URLUtil.java
deleted file mode 100644
index 11cde80..0000000
--- a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/url/URLUtil.java
+++ /dev/null
@@ -1,33 +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.eagle.jpm.util.resourcefetch.url;
-
-/**
- * URL utils.
- */
-public class URLUtil {
-    public static String removeTrailingSlash(String url) {
-        int i = url.length() - 1;
-        // Skip all slashes from the end.
-        while (i >= 0 && url.charAt(i) == '/') {
-            i--;
-        }
-
-        return url.substring(0, i + 1);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/b5fd782c/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/RMResourceFetcher.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/RMResourceFetcher.java b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/RMResourceFetcher.java
new file mode 100644
index 0000000..b1881ef
--- /dev/null
+++ b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/RMResourceFetcher.java
@@ -0,0 +1,189 @@
+/*
+ * 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.eagle.jpm.util.resourcefetch;
+
+import org.apache.eagle.jpm.util.Constants;
+import org.apache.eagle.jpm.util.resourcefetch.connection.InputStreamUtils;
+import org.apache.eagle.jpm.util.resourcefetch.ha.HAURLSelector;
+import org.apache.eagle.jpm.util.resourcefetch.ha.HAURLSelectorImpl;
+import org.apache.eagle.jpm.util.resourcefetch.model.AppInfo;
+import org.apache.eagle.jpm.util.resourcefetch.model.AppsWrapper;
+import org.apache.eagle.jpm.util.resourcefetch.model.ClusterInfo;
+import org.apache.eagle.jpm.util.resourcefetch.model.ClusterInfoWrapper;
+import org.apache.eagle.jpm.util.resourcefetch.url.JobListServiceURLBuilderImpl;
+import org.apache.eagle.jpm.util.resourcefetch.url.ServiceURLBuilder;
+import org.apache.eagle.jpm.util.resourcefetch.url.SparkCompleteJobServiceURLBuilderImpl;
+import org.apache.eagle.jpm.util.resourcefetch.url.URLUtil;
+import org.codehaus.jackson.JsonParser;
+import org.codehaus.jackson.map.ObjectMapper;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.List;
+
+public class RMResourceFetcher implements ResourceFetcher<AppInfo> {
+
+    private static final Logger LOG = LoggerFactory.getLogger(RMResourceFetcher.class);
+    private final HAURLSelector selector;
+    private final ServiceURLBuilder jobListServiceURLBuilder;
+    private final ServiceURLBuilder sparkCompleteJobServiceURLBuilder;
+    private static final ObjectMapper OBJ_MAPPER = new ObjectMapper();
+
+    static {
+        OBJ_MAPPER.configure(JsonParser.Feature.ALLOW_NON_NUMERIC_NUMBERS, true);
+    }
+
+    public RMResourceFetcher(String[] rmBasePaths) {
+        this.jobListServiceURLBuilder = new JobListServiceURLBuilderImpl();
+        this.sparkCompleteJobServiceURLBuilder = new SparkCompleteJobServiceURLBuilderImpl();
+
+        this.selector = new HAURLSelectorImpl(rmBasePaths, jobListServiceURLBuilder, Constants.CompressionType.GZIP);
+    }
+
+    private void checkUrl() throws IOException {
+        if (!selector.checkUrl(jobListServiceURLBuilder.build(selector.getSelectedUrl(), Constants.JobState.RUNNING.name()))) {
+            selector.reSelectUrl();
+        }
+    }
+
+    private List<AppInfo> doFetchFinishApplicationsList(String urlString, Constants.CompressionType compressionType) throws Exception {
+        List<AppInfo> result;
+        InputStream is = null;
+        try {
+            checkUrl();
+            LOG.info("Going to call yarn api to fetch finished application list: " + urlString);
+            is = InputStreamUtils.getInputStream(urlString, null, compressionType);
+            final AppsWrapper appWrapper = OBJ_MAPPER.readValue(is, AppsWrapper.class);
+            if (appWrapper != null && appWrapper.getApps() != null
+                && appWrapper.getApps().getApp() != null) {
+                result = appWrapper.getApps().getApp();
+                return result;
+            }
+            return null;
+        } finally {
+            if (is != null) {
+                try {
+                    is.close();
+                } catch (Exception e) {
+                    LOG.warn("{}", e);
+                }
+            }
+        }
+    }
+
+    private String getSparkRunningJobURL() {
+        return selector.getSelectedUrl()
+                + "/"
+                + Constants.V2_APPS_URL
+                + "?applicationTypes=SPARK&state=RUNNING&"
+                + Constants.ANONYMOUS_PARAMETER;
+    }
+
+    private String getMRRunningJobURL() {
+        return String.format("%s/%s?applicationTypes=MAPREDUCE&state=RUNNING&%s",
+            selector.getSelectedUrl(),
+            Constants.V2_APPS_URL,
+            Constants.ANONYMOUS_PARAMETER);
+    }
+
+    private String getMRFinishedJobURL(String lastFinishedTime) {
+        String url = URLUtil.removeTrailingSlash(selector.getSelectedUrl());
+        return url + "/" + "Constants.V2_APPS_URL"
+                + "?applicationTypes=MAPREDUCE&state=FINISHED&finishedTimeBegin="
+                + lastFinishedTime + "&" + Constants.ANONYMOUS_PARAMETER;
+    }
+
+    private List<AppInfo> doFetchRunningApplicationsList(String urlString, Constants.CompressionType compressionType) throws Exception {
+        List<AppInfo> result;
+        InputStream is = null;
+        try {
+            checkUrl();
+            LOG.info("Going to call yarn api to fetch running application list: " + urlString);
+            is = InputStreamUtils.getInputStream(urlString, null, compressionType);
+            final AppsWrapper appWrapper = OBJ_MAPPER.readValue(is, AppsWrapper.class);
+            if (appWrapper != null && appWrapper.getApps() != null && appWrapper.getApps().getApp() != null) {
+                result = appWrapper.getApps().getApp();
+                return result;
+            }
+            return null;
+        } finally {
+            if (is != null) {
+                try {
+                    is.close();
+                } catch (Exception e) {
+                    LOG.warn("{}", e);
+                }
+            }
+        }
+    }
+
+    private List<AppInfo> getResource(Constants.ResourceType resourceType, Constants.CompressionType compressionType, Object... parameter) throws Exception {
+        switch (resourceType) {
+            case COMPLETE_SPARK_JOB:
+                final String urlString = sparkCompleteJobServiceURLBuilder.build(selector.getSelectedUrl(), (String) parameter[0]);
+                return doFetchFinishApplicationsList(urlString, compressionType);
+            case RUNNING_SPARK_JOB:
+                return doFetchRunningApplicationsList(getSparkRunningJobURL(), compressionType);
+            case RUNNING_MR_JOB:
+                return doFetchRunningApplicationsList(getMRRunningJobURL(), compressionType);
+            case COMPLETE_MR_JOB:
+                return doFetchFinishApplicationsList(getMRFinishedJobURL((String) parameter[0]), compressionType);
+            default:
+                throw new Exception("Not support resourceType :" + resourceType);
+        }
+    }
+
+    public List<AppInfo> getResource(Constants.ResourceType resourceType, Object... parameter) throws Exception {
+        try {
+            return getResource(resourceType, Constants.CompressionType.GZIP, parameter);
+        } catch (java.util.zip.ZipException ex) {
+            return getResource(resourceType, Constants.CompressionType.NONE, parameter);
+        }
+    }
+
+    private String getClusterInfoURL() {
+        return selector.getSelectedUrl() + "/" + Constants.YARN_API_CLUSTER_INFO + "?" + Constants.ANONYMOUS_PARAMETER;
+    }
+
+    public ClusterInfo getClusterInfo() throws Exception {
+        InputStream is = null;
+        try {
+            checkUrl();
+            final String urlString = getClusterInfoURL();
+            LOG.info("Calling yarn api to fetch cluster info: " + urlString);
+            is = InputStreamUtils.getInputStream(urlString, null, Constants.CompressionType.GZIP);
+            final ClusterInfoWrapper clusterInfoWrapper = OBJ_MAPPER.readValue(is, ClusterInfoWrapper.class);
+            if (clusterInfoWrapper != null && clusterInfoWrapper.getClusterInfo() != null) {
+                return clusterInfoWrapper.getClusterInfo();
+            }
+            return null;
+        } finally {
+            if (is != null) {
+                try {
+                    is.close();
+                } catch (Exception e) {
+                    LOG.warn("{}", e);
+                }
+            }
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/b5fd782c/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/ResourceFetcher.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/ResourceFetcher.java b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/ResourceFetcher.java
new file mode 100644
index 0000000..f920ddb
--- /dev/null
+++ b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/ResourceFetcher.java
@@ -0,0 +1,27 @@
+/*
+ * 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.eagle.jpm.util.resourcefetch;
+
+import org.apache.eagle.jpm.util.Constants;
+
+import java.util.List;
+
+public interface ResourceFetcher<T> {
+    //continue to refactor later
+    List<T> getResource(Constants.ResourceType resoureType, Object... parameter) throws Exception;
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/b5fd782c/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/SparkHistoryServerResourceFetcher.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/SparkHistoryServerResourceFetcher.java b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/SparkHistoryServerResourceFetcher.java
new file mode 100644
index 0000000..ce2d9b8
--- /dev/null
+++ b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/SparkHistoryServerResourceFetcher.java
@@ -0,0 +1,88 @@
+/*
+ *
+ *  * Licensed to the Apache Software Foundation (ASF) under one or more
+ *  * contributor license agreements.  See the NOTICE file distributed with
+ *  * this work for additional information regarding copyright ownership.
+ *  * The ASF licenses this file to You under the Apache License, Version 2.0
+ *  * (the "License"); you may not use this file except in compliance with
+ *  * the License.  You may obtain a copy of the License at
+ *  *
+ *  *    http://www.apache.org/licenses/LICENSE-2.0
+ *  *
+ *  * Unless required by applicable law or agreed to in writing, software
+ *  * distributed under the License is distributed on an "AS IS" BASIS,
+ *  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  * See the License for the specific language governing permissions and
+ *  * limitations under the License.
+ *
+ */
+
+package org.apache.eagle.jpm.util.resourcefetch;
+
+import org.apache.eagle.jpm.util.Constants;
+import org.apache.eagle.jpm.util.resourcefetch.connection.InputStreamUtils;
+import org.apache.eagle.jpm.util.resourcefetch.model.SparkApplication;
+import org.apache.eagle.jpm.util.resourcefetch.url.ServiceURLBuilder;
+import org.apache.eagle.jpm.util.resourcefetch.url.SparkJobServiceURLBuilderImpl;
+import org.apache.commons.codec.binary.Base64;
+import org.codehaus.jackson.JsonParser;
+import org.codehaus.jackson.map.ObjectMapper;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.FileNotFoundException;
+import java.io.InputStream;
+import java.util.Arrays;
+import java.util.List;
+
+public class SparkHistoryServerResourceFetcher implements ResourceFetcher<SparkApplication> {
+
+    private static final Logger LOG = LoggerFactory.getLogger(SparkHistoryServerResourceFetcher.class);
+
+    private static final ObjectMapper OBJ_MAPPER = new ObjectMapper();
+
+    private String historyServerURL;
+    private final ServiceURLBuilder sparkDetailJobServiceURLBuilder;
+    private String auth;
+
+    static {
+        OBJ_MAPPER.configure(JsonParser.Feature.ALLOW_NON_NUMERIC_NUMBERS, true);
+    }
+
+    public SparkHistoryServerResourceFetcher(String historyServerURL, String userName, String pwd) {
+        this.historyServerURL = historyServerURL;
+        this.sparkDetailJobServiceURLBuilder = new SparkJobServiceURLBuilderImpl();
+        this.auth = "Basic " + new String(new Base64().encode(String.format("%s:%s", userName, pwd).getBytes()));
+        ;
+    }
+
+    private List<SparkApplication> doFetchSparkApplicationDetail(String appId) throws Exception {
+        InputStream is = null;
+        try {
+            final String urlString = sparkDetailJobServiceURLBuilder.build(this.historyServerURL, appId);
+            LOG.info("Going to call spark history server api to fetch spark job: " + urlString);
+            is = InputStreamUtils.getInputStream(urlString, auth, Constants.CompressionType.NONE);
+            SparkApplication app = OBJ_MAPPER.readValue(is, SparkApplication.class);
+            return Arrays.asList(app);
+        } catch (FileNotFoundException e) {
+            return null;
+        } finally {
+            if (is != null) {
+                try {
+                    is.close();
+                } catch (Exception e) {
+                    LOG.warn("{}", e);
+                }
+            }
+        }
+    }
+
+    public List<SparkApplication> getResource(Constants.ResourceType resoureType, Object... parameter) throws Exception {
+        switch (resoureType) {
+            case SPARK_JOB_DETAIL:
+                return doFetchSparkApplicationDetail((String) parameter[0]);
+            default:
+                throw new Exception("Not support resourceType :" + resoureType);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/b5fd782c/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/connection/InputStreamUtils.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/connection/InputStreamUtils.java b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/connection/InputStreamUtils.java
new file mode 100644
index 0000000..e99e2aa
--- /dev/null
+++ b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/connection/InputStreamUtils.java
@@ -0,0 +1,68 @@
+/*
+ * 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.eagle.jpm.util.resourcefetch.connection;
+
+import org.apache.eagle.jpm.util.Constants;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.net.URL;
+import java.net.URLConnection;
+import java.util.zip.GZIPInputStream;
+
+public class InputStreamUtils {
+
+    private static final int CONNECTION_TIMEOUT = 10 * 1000;
+    private static final int READ_TIMEOUT = 5 * 60 * 1000;
+    private static final String GZIP_HTTP_HEADER = "Accept-Encoding";
+    private static final String GZIP_COMPRESSION = "gzip";
+
+    private static InputStream openGZIPInputStream(URL url, String auth, int timeout) throws IOException {
+        final URLConnection connection = url.openConnection();
+        connection.setConnectTimeout(CONNECTION_TIMEOUT);
+        connection.setReadTimeout(timeout);
+        connection.addRequestProperty(GZIP_HTTP_HEADER, GZIP_COMPRESSION);
+        if (null != auth) {
+            connection.setRequestProperty("Authorization", auth);
+        }
+        return new GZIPInputStream(connection.getInputStream());
+    }
+
+    private static InputStream openInputStream(URL url, String auth, int timeout) throws IOException {
+        URLConnection connection = url.openConnection();
+        connection.setConnectTimeout(timeout);
+        if (null != auth) {
+            connection.setRequestProperty("Authorization", auth);
+        }
+
+        return connection.getInputStream();
+    }
+
+    public static InputStream getInputStream(String urlString, String auth, Constants.CompressionType compressionType, int timeout) throws Exception {
+        final URL url = URLConnectionUtils.getUrl(urlString);
+        if (compressionType.equals(Constants.CompressionType.GZIP)) {
+            return openGZIPInputStream(url, auth, timeout);
+        } else { // CompressionType.NONE
+            return openInputStream(url, auth, timeout);
+        }
+    }
+
+    public static InputStream getInputStream(String urlString, String auth, Constants.CompressionType compressionType) throws Exception {
+        return getInputStream(urlString, auth, compressionType, READ_TIMEOUT);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/b5fd782c/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/connection/JobUtils.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/connection/JobUtils.java b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/connection/JobUtils.java
new file mode 100644
index 0000000..44d27e8
--- /dev/null
+++ b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/connection/JobUtils.java
@@ -0,0 +1,43 @@
+/*
+ * 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.eagle.jpm.util.resourcefetch.connection;
+
+import org.apache.eagle.jpm.util.Constants;
+
+public class JobUtils {
+
+    public static String checkAndAddLastSlash(String urlBase) {
+        if (!urlBase.endsWith("/")) {
+            return urlBase + "/";
+        }
+        return urlBase;
+    }
+
+    public static String getJobIDByAppID(String appID) {
+        if (appID.startsWith(Constants.APPLICATION_PREFIX)) {
+            return appID.replace(Constants.APPLICATION_PREFIX, Constants.JOB_PREFIX);
+        }
+        return null;
+    }
+
+    public static String getAppIDByJobID(String jobID) {
+        if (jobID.startsWith(Constants.JOB_PREFIX)) {
+            return jobID.replace(Constants.JOB_PREFIX, Constants.APPLICATION_PREFIX);
+        }
+        return null;
+    }
+}



[07/52] [abbrv] incubator-eagle git commit: [EAGLE-496] fix code style of jpm

Posted by yo...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/0b852cbc/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JobConfigurationCreationServiceListener.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JobConfigurationCreationServiceListener.java b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JobConfigurationCreationServiceListener.java
index 4163f7b..7293c89 100644
--- a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JobConfigurationCreationServiceListener.java
+++ b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JobConfigurationCreationServiceListener.java
@@ -33,7 +33,7 @@ public class JobConfigurationCreationServiceListener implements HistoryJobEntity
     private static final Logger logger = LoggerFactory.getLogger(JobConfigurationCreationServiceListener.class);
     private static final int MAX_RETRY_TIMES = 3;
     private MRHistoryJobConfig configManager;
-    private JobConfigurationAPIEntity m_jobConfigurationEntity;
+    private JobConfigurationAPIEntity jobConfigurationEntity;
 
     public JobConfigurationCreationServiceListener(MRHistoryJobConfig configManager) {
         this.configManager = configManager;
@@ -43,7 +43,7 @@ public class JobConfigurationCreationServiceListener implements HistoryJobEntity
     public void jobEntityCreated(JobBaseAPIEntity entity) throws Exception {
         if (entity != null) {
             if (entity instanceof JobConfigurationAPIEntity) {
-                this.m_jobConfigurationEntity = (JobConfigurationAPIEntity) entity;
+                this.jobConfigurationEntity = (JobConfigurationAPIEntity) entity;
             }
         }
     }
@@ -65,7 +65,7 @@ public class JobConfigurationCreationServiceListener implements HistoryJobEntity
 
         client.getJerseyClient().setReadTimeout(jobExtractorConfig.readTimeoutSeconds * 1000);
         List<JobConfigurationAPIEntity> list = new ArrayList<>();
-        list.add(m_jobConfigurationEntity);
+        list.add(jobConfigurationEntity);
 
         int tried = 0;
         while (tried <= MAX_RETRY_TIMES) {
@@ -82,7 +82,7 @@ public class JobConfigurationCreationServiceListener implements HistoryJobEntity
                 }
             } finally {
                 list.clear();
-                m_jobConfigurationEntity = null;
+                jobConfigurationEntity = null;
                 client.getJerseyClient().destroy();
                 client.close();
             }

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/0b852cbc/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JobEntityCreationEagleServiceListener.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JobEntityCreationEagleServiceListener.java b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JobEntityCreationEagleServiceListener.java
index abddf3b..e7b8a6b 100644
--- a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JobEntityCreationEagleServiceListener.java
+++ b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JobEntityCreationEagleServiceListener.java
@@ -62,7 +62,7 @@ public class JobEntityCreationEagleServiceListener implements HistoryJobEntityCr
     }
 
     /**
-     * We need save network bandwidth as well
+     * We need save network bandwidth as well.
      */
     @Override
     public void flush() throws Exception {

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/0b852cbc/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JobEntityCreationPublisher.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JobEntityCreationPublisher.java b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JobEntityCreationPublisher.java
index 8b85b26..a80462d 100644
--- a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JobEntityCreationPublisher.java
+++ b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JobEntityCreationPublisher.java
@@ -22,11 +22,6 @@ import org.apache.eagle.jpm.mr.historyentity.JobBaseAPIEntity;
 
 import java.util.Vector;
 
-/**
- * not thread safe
- *
- * @author yonzhang
- */
 public class JobEntityCreationPublisher {
     private Vector<HistoryJobEntityCreationListener> listeners = new Vector<>(2);
 

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/0b852cbc/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JobEntityLifecycleAggregator.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JobEntityLifecycleAggregator.java b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JobEntityLifecycleAggregator.java
index 594d8e2..ea0ba30 100644
--- a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JobEntityLifecycleAggregator.java
+++ b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JobEntityLifecycleAggregator.java
@@ -18,37 +18,36 @@
 
 package org.apache.eagle.jpm.mr.history.parser;
 
-import com.fasterxml.jackson.databind.ObjectMapper;
 import org.apache.eagle.jpm.mr.historyentity.JobBaseAPIEntity;
 import org.apache.eagle.jpm.mr.historyentity.JobExecutionAPIEntity;
 import org.apache.eagle.jpm.mr.historyentity.TaskAttemptExecutionAPIEntity;
 import org.apache.eagle.jpm.util.Constants;
 import org.apache.eagle.jpm.util.jobcounter.JobCounters;
+import com.fasterxml.jackson.databind.ObjectMapper;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-
 import java.io.IOException;
 import java.util.HashMap;
 import java.util.Map;
 
 public class JobEntityLifecycleAggregator implements HistoryJobEntityLifecycleListener {
-    private final static Logger LOG = LoggerFactory.getLogger(JobEntityLifecycleAggregator.class);
-    private JobExecutionAPIEntity m_jobExecutionAPIEntity;
-    private final JobCounterAggregateFunction m_mapTaskAttemptCounterAgg;
-    private final JobCounterAggregateFunction m_reduceTaskAttemptCounterAgg;
+    private static final Logger LOG = LoggerFactory.getLogger(JobEntityLifecycleAggregator.class);
+    private JobExecutionAPIEntity jobExecutionAPIEntity;
+    private final JobCounterAggregateFunction mapTaskAttemptCounterAgg;
+    private final JobCounterAggregateFunction reduceTaskAttemptCounterAgg;
 
-    private final JobCounterAggregateFunction m_mapFileSystemCounterAgg;
-    private final JobCounterAggregateFunction m_reduceFileSystemTaskCounterAgg;
+    private final JobCounterAggregateFunction mapFileSystemCounterAgg;
+    private final JobCounterAggregateFunction reduceFileSystemTaskCounterAgg;
 
-    private long m_mapAttemptDuration = 0;
-    private long m_reduceAttemptDuration = 0;
+    private long mapAttemptDuration = 0;
+    private long reduceAttemptDuration = 0;
     private boolean jobFinished = false;
 
     public JobEntityLifecycleAggregator() {
-        this.m_mapTaskAttemptCounterAgg = new JobCounterSumFunction();
-        this.m_reduceTaskAttemptCounterAgg = new JobCounterSumFunction();
-        this.m_mapFileSystemCounterAgg = new JobCounterSumFunction();
-        this.m_reduceFileSystemTaskCounterAgg = new JobCounterSumFunction();
+        this.mapTaskAttemptCounterAgg = new JobCounterSumFunction();
+        this.reduceTaskAttemptCounterAgg = new JobCounterSumFunction();
+        this.mapFileSystemCounterAgg = new JobCounterSumFunction();
+        this.reduceFileSystemTaskCounterAgg = new JobCounterSumFunction();
     }
 
     @Override
@@ -57,7 +56,7 @@ public class JobEntityLifecycleAggregator implements HistoryJobEntityLifecycleLi
             if (entity instanceof TaskAttemptExecutionAPIEntity) {
                 taskAttemptEntityCreated((TaskAttemptExecutionAPIEntity) entity);
             } else if (entity instanceof JobExecutionAPIEntity) {
-                this.m_jobExecutionAPIEntity = (JobExecutionAPIEntity) entity;
+                this.jobExecutionAPIEntity = (JobExecutionAPIEntity) entity;
             }
         }
     }
@@ -65,44 +64,44 @@ public class JobEntityLifecycleAggregator implements HistoryJobEntityLifecycleLi
     @Override
     public void jobFinish() {
         try {
-            if (m_jobExecutionAPIEntity == null) {
+            if (jobExecutionAPIEntity == null) {
                 throw new IOException("No JobExecutionAPIEntity found before flushing");
             }
 
             LOG.debug("Updating aggregated task attempts to job level counters");
 
-            JobCounters jobCounters = m_jobExecutionAPIEntity.getJobCounters();
+            JobCounters jobCounters = jobExecutionAPIEntity.getJobCounters();
 
             if (jobCounters == null) {
-                LOG.warn("no job counter found for " + this.m_jobExecutionAPIEntity);
+                LOG.warn("no job counter found for " + this.jobExecutionAPIEntity);
                 jobCounters = new JobCounters();
             }
 
             Map<String, Map<String, Long>> counters = jobCounters.getCounters();
 
-            Map<String, Long> mapTaskAttemptCounter = this.m_mapTaskAttemptCounterAgg.result();
+            Map<String, Long> mapTaskAttemptCounter = this.mapTaskAttemptCounterAgg.result();
             if (mapTaskAttemptCounter == null) {
                 mapTaskAttemptCounter = new HashMap<>();
             }
-            mapTaskAttemptCounter.put(Constants.TaskAttemptCounter.TASK_ATTEMPT_DURATION.toString(), this.m_mapAttemptDuration);
+            mapTaskAttemptCounter.put(Constants.TaskAttemptCounter.TASK_ATTEMPT_DURATION.toString(), this.mapAttemptDuration);
             counters.put(Constants.MAP_TASK_ATTEMPT_COUNTER, mapTaskAttemptCounter);
 
-            Map<String, Long> reduceTaskAttemptCounter = this.m_reduceTaskAttemptCounterAgg.result();
+            Map<String, Long> reduceTaskAttemptCounter = this.reduceTaskAttemptCounterAgg.result();
             if (reduceTaskAttemptCounter == null) {
                 reduceTaskAttemptCounter = new HashMap<>();
             }
-            reduceTaskAttemptCounter.put(Constants.TaskAttemptCounter.TASK_ATTEMPT_DURATION.toString(), this.m_reduceAttemptDuration);
+            reduceTaskAttemptCounter.put(Constants.TaskAttemptCounter.TASK_ATTEMPT_DURATION.toString(), this.reduceAttemptDuration);
             counters.put(Constants.REDUCE_TASK_ATTEMPT_COUNTER, reduceTaskAttemptCounter);
 
-            counters.put(Constants.MAP_TASK_ATTEMPT_FILE_SYSTEM_COUNTER, this.m_mapFileSystemCounterAgg.result());
-            counters.put(Constants.REDUCE_TASK_ATTEMPT_FILE_SYSTEM_COUNTER, this.m_reduceFileSystemTaskCounterAgg.result());
+            counters.put(Constants.MAP_TASK_ATTEMPT_FILE_SYSTEM_COUNTER, this.mapFileSystemCounterAgg.result());
+            counters.put(Constants.REDUCE_TASK_ATTEMPT_FILE_SYSTEM_COUNTER, this.reduceFileSystemTaskCounterAgg.result());
 
             jobCounters.setCounters(counters);
 
-            m_jobExecutionAPIEntity.setJobCounters(jobCounters);
+            jobExecutionAPIEntity.setJobCounters(jobCounters);
             jobFinished = true;
         } catch (Exception e) {
-            LOG.error("Failed to update job execution entity: " + this.m_jobExecutionAPIEntity.toString() + ", due to " + e.getMessage(), e);
+            LOG.error("Failed to update job execution entity: " + this.jobExecutionAPIEntity.toString() + ", due to " + e.getMessage(), e);
         }
     }
 
@@ -112,14 +111,14 @@ public class JobEntityLifecycleAggregator implements HistoryJobEntityLifecycleLi
 
         if (taskType != null && jobCounters != null && jobCounters.getCounters() != null) {
             if (Constants.TaskType.MAP.toString().equals(taskType.toUpperCase())) {
-                m_mapAttemptDuration += entity.getDuration();
-                this.m_mapTaskAttemptCounterAgg.accumulate(jobCounters.getCounters().get(Constants.TASK_COUNTER));
-                this.m_mapFileSystemCounterAgg.accumulate(jobCounters.getCounters().get(Constants.FILE_SYSTEM_COUNTER));
+                mapAttemptDuration += entity.getDuration();
+                this.mapTaskAttemptCounterAgg.accumulate(jobCounters.getCounters().get(Constants.TASK_COUNTER));
+                this.mapFileSystemCounterAgg.accumulate(jobCounters.getCounters().get(Constants.FILE_SYSTEM_COUNTER));
                 return;
             } else if (Constants.TaskType.REDUCE.toString().equals(taskType.toUpperCase())) {
-                m_reduceAttemptDuration += entity.getDuration();
-                this.m_reduceTaskAttemptCounterAgg.accumulate(jobCounters.getCounters().get(Constants.TASK_COUNTER));
-                this.m_reduceFileSystemTaskCounterAgg.accumulate(jobCounters.getCounters().get(Constants.FILE_SYSTEM_COUNTER));
+                reduceAttemptDuration += entity.getDuration();
+                this.reduceTaskAttemptCounterAgg.accumulate(jobCounters.getCounters().get(Constants.TASK_COUNTER));
+                this.reduceFileSystemTaskCounterAgg.accumulate(jobCounters.getCounters().get(Constants.FILE_SYSTEM_COUNTER));
                 return;
             }
         }
@@ -151,10 +150,7 @@ public class JobEntityLifecycleAggregator implements HistoryJobEntityLifecycleLi
         public JobCounterSumFunction() {
             result = new HashMap<>();
         }
-
-        /**
-         * @param counters
-         */
+        
         @Override
         public void accumulate(Map<String, Long> counters) {
             if (counters != null) {

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/0b852cbc/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/TaskFailureListener.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/TaskFailureListener.java b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/TaskFailureListener.java
index e0c3c6b..f95eaa2 100755
--- a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/TaskFailureListener.java
+++ b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/TaskFailureListener.java
@@ -37,7 +37,7 @@ import java.util.List;
 import java.util.Map;
 
 public class TaskFailureListener implements HistoryJobEntityCreationListener {
-    private static final Logger logger = LoggerFactory.getLogger(TaskFailureListener.class);
+    private static final Logger LOG = LoggerFactory.getLogger(TaskFailureListener.class);
     private static final String MR_ERROR_CATEGORY_CONFIG_FILE_NAME = "MRErrorCategory.config";
     private static final int BATCH_SIZE = 1000;
     private static final int MAX_RETRY_TIMES = 3;
@@ -53,7 +53,7 @@ public class TaskFailureListener implements HistoryJobEntityCreationListener {
             is = TaskFailureListener.class.getClassLoader().getResourceAsStream(MR_ERROR_CATEGORY_CONFIG_FILE_NAME);
             URL url = TaskFailureListener.class.getClassLoader().getResource(MR_ERROR_CATEGORY_CONFIG_FILE_NAME);
             if (url != null) {
-                logger.info("Feeder is going to load configuration file: " + url.toString());
+                LOG.info("Feeder is going to load configuration file: " + url.toString());
             }
             classifier = new MRErrorClassifier(is);
         } catch (IOException ex) {
@@ -63,6 +63,7 @@ public class TaskFailureListener implements HistoryJobEntityCreationListener {
                 try {
                     is.close();
                 } catch (IOException e) {
+                    LOG.warn("exception found {}", e);
                 }
             }
         }
@@ -121,16 +122,16 @@ public class TaskFailureListener implements HistoryJobEntityCreationListener {
         int tried = 0;
         while (tried <= MAX_RETRY_TIMES) {
             try {
-                logger.info("start flushing entities of total number " + failureTasks.size());
+                LOG.info("start flushing entities of total number " + failureTasks.size());
                 client.create(failureTasks);
-                logger.info("finish flushing entities of total number " + failureTasks.size());
+                LOG.info("finish flushing entities of total number " + failureTasks.size());
                 failureTasks.clear();
                 break;
             } catch (Exception ex) {
                 if (tried < MAX_RETRY_TIMES) {
-                    logger.error("Got exception to flush, retry as " + (tried + 1) + " times", ex);
+                    LOG.error("Got exception to flush, retry as " + (tried + 1) + " times", ex);
                 } else {
-                    logger.error("Got exception to flush, reach max retry times " + MAX_RETRY_TIMES, ex);
+                    LOG.error("Got exception to flush, reach max retry times " + MAX_RETRY_TIMES, ex);
                     throw ex;
                 }
             }

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/0b852cbc/eagle-jpm/eagle-jpm-mr-running/src/main/java/org/apache/eagle/jpm/mr/running/MRRunningJobMain.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-mr-running/src/main/java/org/apache/eagle/jpm/mr/running/MRRunningJobMain.java b/eagle-jpm/eagle-jpm-mr-running/src/main/java/org/apache/eagle/jpm/mr/running/MRRunningJobMain.java
index cef29fe..87079fd 100644
--- a/eagle-jpm/eagle-jpm-mr-running/src/main/java/org/apache/eagle/jpm/mr/running/MRRunningJobMain.java
+++ b/eagle-jpm/eagle-jpm-mr-running/src/main/java/org/apache/eagle/jpm/mr/running/MRRunningJobMain.java
@@ -18,18 +18,17 @@
 
 package org.apache.eagle.jpm.mr.running;
 
+
+import org.apache.eagle.jpm.mr.running.config.MRRunningConfigManager;
+import org.apache.eagle.jpm.mr.running.storm.MRRunningJobFetchSpout;
+import org.apache.eagle.jpm.mr.running.storm.MRRunningJobParseBolt;
+import org.apache.eagle.jpm.util.Constants;
 import backtype.storm.Config;
 import backtype.storm.LocalCluster;
 import backtype.storm.StormSubmitter;
 import backtype.storm.topology.TopologyBuilder;
 import backtype.storm.tuple.Fields;
-import org.apache.eagle.jpm.mr.running.config.MRRunningConfigManager;
-import org.apache.eagle.jpm.mr.running.storm.MRRunningJobFetchSpout;
-import org.apache.eagle.jpm.mr.running.storm.MRRunningJobParseBolt;
-import org.apache.eagle.jpm.util.Constants;
-
 import java.util.List;
-import java.util.regex.Pattern;
 
 public class MRRunningJobMain {
     public static void main(String[] args) {

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/0b852cbc/eagle-jpm/eagle-jpm-mr-running/src/main/java/org/apache/eagle/jpm/mr/running/config/MRRunningConfigManager.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-mr-running/src/main/java/org/apache/eagle/jpm/mr/running/config/MRRunningConfigManager.java b/eagle-jpm/eagle-jpm-mr-running/src/main/java/org/apache/eagle/jpm/mr/running/config/MRRunningConfigManager.java
index a91a493..42426e4 100644
--- a/eagle-jpm/eagle-jpm-mr-running/src/main/java/org/apache/eagle/jpm/mr/running/config/MRRunningConfigManager.java
+++ b/eagle-jpm/eagle-jpm-mr-running/src/main/java/org/apache/eagle/jpm/mr/running/config/MRRunningConfigManager.java
@@ -18,8 +18,8 @@
 
 package org.apache.eagle.jpm.mr.running.config;
 
-import com.typesafe.config.Config;
 import org.apache.eagle.common.config.ConfigOptionParser;
+import com.typesafe.config.Config;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -27,27 +27,35 @@ import java.io.Serializable;
 
 public class MRRunningConfigManager implements Serializable {
     private static final Logger LOG = LoggerFactory.getLogger(MRRunningConfigManager.class);
+
     public String getEnv() {
         return env;
     }
+
     private String env;
 
-    public ZKStateConfig getZkStateConfig() { return zkStateConfig; }
+    public ZKStateConfig getZkStateConfig() {
+        return zkStateConfig;
+    }
+
     private ZKStateConfig zkStateConfig;
 
     public EagleServiceConfig getEagleServiceConfig() {
         return eagleServiceConfig;
     }
+
     private EagleServiceConfig eagleServiceConfig;
 
     public JobExtractorConfig getJobExtractorConfig() {
         return jobExtractorConfig;
     }
+
     private JobExtractorConfig jobExtractorConfig;
 
     public EndpointConfig getEndpointConfig() {
         return endpointConfig;
     }
+
     private EndpointConfig endpointConfig;
 
     public static class ZKStateConfig implements Serializable {
@@ -82,6 +90,7 @@ public class MRRunningConfigManager implements Serializable {
     public Config getConfig() {
         return config;
     }
+
     private Config config;
 
     private static MRRunningConfigManager manager = new MRRunningConfigManager();

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/0b852cbc/eagle-jpm/eagle-jpm-mr-running/src/main/java/org/apache/eagle/jpm/mr/running/parser/MRJobParser.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-mr-running/src/main/java/org/apache/eagle/jpm/mr/running/parser/MRJobParser.java b/eagle-jpm/eagle-jpm-mr-running/src/main/java/org/apache/eagle/jpm/mr/running/parser/MRJobParser.java
index b7de79e..3b31d93 100644
--- a/eagle-jpm/eagle-jpm-mr-running/src/main/java/org/apache/eagle/jpm/mr/running/parser/MRJobParser.java
+++ b/eagle-jpm/eagle-jpm-mr-running/src/main/java/org/apache/eagle/jpm/mr/running/parser/MRJobParser.java
@@ -18,7 +18,6 @@
 
 package org.apache.eagle.jpm.mr.running.parser;
 
-import org.apache.commons.lang3.tuple.Pair;
 import org.apache.eagle.jpm.mr.running.config.MRRunningConfigManager;
 import org.apache.eagle.jpm.mr.running.recover.MRRunningJobManager;
 import org.apache.eagle.jpm.mr.runningentity.JobConfig;
@@ -29,11 +28,12 @@ import org.apache.eagle.jpm.util.Constants;
 import org.apache.eagle.jpm.util.JobNameNormalization;
 import org.apache.eagle.jpm.util.MRJobTagName;
 import org.apache.eagle.jpm.util.Utils;
-import org.apache.eagle.jpm.util.resourceFetch.ResourceFetcher;
-import org.apache.eagle.jpm.util.resourceFetch.connection.InputStreamUtils;
-import org.apache.eagle.jpm.util.resourceFetch.connection.URLConnectionUtils;
-import org.apache.eagle.jpm.util.resourceFetch.model.*;
-import org.apache.eagle.jpm.util.resourceFetch.model.JobCounters;
+import org.apache.eagle.jpm.util.resourcefetch.ResourceFetcher;
+import org.apache.eagle.jpm.util.resourcefetch.connection.InputStreamUtils;
+import org.apache.eagle.jpm.util.resourcefetch.connection.URLConnectionUtils;
+import org.apache.eagle.jpm.util.resourcefetch.model.*;
+import org.apache.eagle.jpm.util.resourcefetch.model.JobCounters;
+import org.apache.commons.lang3.tuple.Pair;
 import org.codehaus.jackson.JsonParser;
 import org.codehaus.jackson.map.ObjectMapper;
 import org.slf4j.Logger;
@@ -42,13 +42,12 @@ import org.w3c.dom.Document;
 import org.w3c.dom.Element;
 import org.w3c.dom.Node;
 import org.w3c.dom.NodeList;
-
-import javax.xml.parsers.DocumentBuilder;
-import javax.xml.parsers.DocumentBuilderFactory;
 import java.io.InputStream;
 import java.net.URLConnection;
 import java.util.*;
 import java.util.function.Function;
+import javax.xml.parsers.DocumentBuilder;
+import javax.xml.parsers.DocumentBuilderFactory;
 
 public class MRJobParser implements Runnable {
     private static final Logger LOG = LoggerFactory.getLogger(MRJobParser.class);
@@ -58,6 +57,7 @@ public class MRJobParser implements Runnable {
         FINISHED,
         APP_FINISHED
     }
+
     private AppInfo app;
     private static final int MAX_RETRY_TIMES = 2;
     private MRJobEntityCreationHandler mrJobEntityCreationHandler;
@@ -78,6 +78,7 @@ public class MRJobParser implements Runnable {
     private Set<String> finishedTaskIds;
     private List<String> configKeys;
     private MRRunningConfigManager.JobExtractorConfig jobExtractorConfig;
+
     static {
         OBJ_MAPPER.configure(JsonParser.Feature.ALLOW_NON_NUMERIC_NUMBERS, true);
     }
@@ -187,11 +188,11 @@ public class MRJobParser implements Runnable {
                 mrJobEntityMap.put(id, new JobExecutionAPIEntity());
             }
 
-            String jobDefId = JobNameNormalization.getInstance().normalize(mrJob.getName());
             JobExecutionAPIEntity jobExecutionAPIEntity = mrJobEntityMap.get(id);
             jobExecutionAPIEntity.setTags(new HashMap<>(commonTags));
             jobExecutionAPIEntity.getTags().put(MRJobTagName.JOB_ID.toString(), id);
             jobExecutionAPIEntity.getTags().put(MRJobTagName.JOB_NAME.toString(), mrJob.getName());
+            String jobDefId = JobNameNormalization.getInstance().normalize(mrJob.getName());
             jobExecutionAPIEntity.getTags().put(MRJobTagName.JOD_DEF_ID.toString(), jobDefId);
             if (mrJobConfigs.get(id) != null) {
                 JobConfig jobConfig = mrJobConfigs.get(id);
@@ -248,7 +249,9 @@ public class MRJobParser implements Runnable {
             Utils.closeInputStream(is);
         }
 
-        if (jobCounters.getCounterGroup() == null) return true;
+        if (jobCounters.getCounterGroup() == null) {
+            return true;
+        }
         JobExecutionAPIEntity jobExecutionAPIEntity = mrJobEntityMap.get(jobId);
         org.apache.eagle.jpm.util.jobcounter.JobCounters jobCounter = new org.apache.eagle.jpm.util.jobcounter.JobCounters();
         Map<String, Map<String, Long>> groups = new HashMap<>();
@@ -261,7 +264,9 @@ public class MRJobParser implements Runnable {
 
             Map<String, Long> counterValues = groups.get(counterGroupName);
             List<JobCounterItem> items = jobCounterGroup.getCounter();
-            if (items == null) continue;
+            if (items == null) {
+                continue;
+            }
             for (JobCounterItem item : items) {
                 String key = item.getName();
                 counterValues.put(key, item.getTotalCounterValue());
@@ -290,7 +295,11 @@ public class MRJobParser implements Runnable {
         org.apache.eagle.jpm.util.jobcounter.JobCounters jobCounter = new org.apache.eagle.jpm.util.jobcounter.JobCounters();
         String jobId = jobAndTaskId.getLeft();
         String taskId = jobAndTaskId.getRight();
-        String taskCounterURL = app.getTrackingUrl() + Constants.MR_JOBS_URL + "/" + jobId + "/" + Constants.MR_TASKS_URL + "/" + taskId + "/" + Constants.MR_JOB_COUNTERS_URL + "?" + Constants.ANONYMOUS_PARAMETER;
+        String taskCounterURL = app.getTrackingUrl()
+            + Constants.MR_JOBS_URL + "/"
+            + jobId + "/" + Constants.MR_TASKS_URL + "/"
+            + taskId + "/" + Constants.MR_JOB_COUNTERS_URL
+            + "?" + Constants.ANONYMOUS_PARAMETER;
         InputStream is = null;
         TaskCounters taskCounters = null;
         try {
@@ -304,7 +313,9 @@ public class MRJobParser implements Runnable {
             Utils.closeInputStream(is);
         }
 
-        if (taskCounters.getTaskCounterGroup() == null) return jobCounter;
+        if (taskCounters.getTaskCounterGroup() == null) {
+            return jobCounter;
+        }
         Map<String, Map<String, Long>> groups = new HashMap<>();
 
         for (TaskCounterGroup taskCounterGroup : taskCounters.getTaskCounterGroup()) {
@@ -314,7 +325,9 @@ public class MRJobParser implements Runnable {
 
             Map<String, Long> counterValues = groups.get(taskCounterGroup.getCounterGroupName());
             List<TaskCounterItem> items = taskCounterGroup.getCounter();
-            if (items == null) continue;
+            if (items == null) {
+                continue;
+            }
             for (TaskCounterItem item : items) {
                 counterValues.put(item.getName(), item.getValue());
             }
@@ -328,7 +341,10 @@ public class MRJobParser implements Runnable {
     private Function<Pair<String, String>, TaskAttemptExecutionAPIEntity> fetchTaskAttempt = jobAndTaskId -> {
         String jobId = jobAndTaskId.getLeft();
         String taskId = jobAndTaskId.getRight();
-        String taskAttemptURL = app.getTrackingUrl() + Constants.MR_JOBS_URL + "/" + jobId + "/" + Constants.MR_TASKS_URL + "/" + taskId + "/" + Constants.MR_TASK_ATTEMPTS_URL + "?" + Constants.ANONYMOUS_PARAMETER;
+        String taskAttemptURL = app.getTrackingUrl()
+            + Constants.MR_JOBS_URL + "/"
+            + jobId + "/" + Constants.MR_TASKS_URL + "/"
+            + taskId + "/" + Constants.MR_TASK_ATTEMPTS_URL + "?" + Constants.ANONYMOUS_PARAMETER;
         InputStream is = null;
         List<MRTaskAttempt> taskAttempts = null;
         try {
@@ -463,10 +479,10 @@ public class MRJobParser implements Runnable {
 
             mrJobEntityCreationHandler.add(taskExecutionAPIEntity);
 
-            if (task.getState().equals(Constants.TaskState.SUCCEEDED.toString()) ||
-                    task.getState().equals(Constants.TaskState.FAILED.toString()) ||
-                    task.getState().equals(Constants.TaskState.KILLED.toString()) ||
-                    task.getState().equals(Constants.TaskState.KILL_WAIT.toString())) {
+            if (task.getState().equals(Constants.TaskState.SUCCEEDED.toString())
+                || task.getState().equals(Constants.TaskState.FAILED.toString())
+                || task.getState().equals(Constants.TaskState.KILLED.toString())
+                || task.getState().equals(Constants.TaskState.KILL_WAIT.toString())) {
                 //LOG.info("mr job {} task {} has finished", jobId, task.getId());
                 this.finishedTaskIds.add(task.getId());
             }
@@ -546,10 +562,10 @@ public class MRJobParser implements Runnable {
                     mrJobEntityMap.keySet()
                             .stream()
                             .filter(
-                                    jobId -> mrJobEntityMap.get(jobId).getCurrentState().equals(Constants.AppState.FINISHED.toString()) ||
-                                            mrJobEntityMap.get(jobId).getCurrentState().equals(Constants.AppState.FAILED.toString()))
+                                jobId -> mrJobEntityMap.get(jobId).getCurrentState().equals(Constants.AppState.FINISHED.toString())
+                                    || mrJobEntityMap.get(jobId).getCurrentState().equals(Constants.AppState.FAILED.toString()))
                             .forEach(
-                                    jobId -> this.runningJobManager.delete(app.getId(), jobId));
+                                jobId -> this.runningJobManager.delete(app.getId(), jobId));
                 }
 
                 LOG.info("finish process yarn application " + app.getId());

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/0b852cbc/eagle-jpm/eagle-jpm-mr-running/src/main/java/org/apache/eagle/jpm/mr/running/recover/MRRunningJobManager.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-mr-running/src/main/java/org/apache/eagle/jpm/mr/running/recover/MRRunningJobManager.java b/eagle-jpm/eagle-jpm-mr-running/src/main/java/org/apache/eagle/jpm/mr/running/recover/MRRunningJobManager.java
index 978c3ec..75650b7 100644
--- a/eagle-jpm/eagle-jpm-mr-running/src/main/java/org/apache/eagle/jpm/mr/running/recover/MRRunningJobManager.java
+++ b/eagle-jpm/eagle-jpm-mr-running/src/main/java/org/apache/eagle/jpm/mr/running/recover/MRRunningJobManager.java
@@ -18,14 +18,14 @@
 
 package org.apache.eagle.jpm.mr.running.recover;
 
-import org.apache.commons.lang3.tuple.Pair;
 import org.apache.eagle.jpm.mr.running.config.MRRunningConfigManager;
 import org.apache.eagle.jpm.mr.runningentity.JobExecutionAPIEntity;
 import org.apache.eagle.jpm.util.jobrecover.RunningJobManager;
-import org.apache.eagle.jpm.util.resourceFetch.model.AppInfo;
-
+import org.apache.eagle.jpm.util.resourcefetch.model.AppInfo;
+import org.apache.commons.lang3.tuple.Pair;
 import java.io.Serializable;
-import java.util.*;
+import java.util.HashMap;
+import java.util.Map;
 
 public class MRRunningJobManager implements Serializable {
     private RunningJobManager runningJobManager;

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/0b852cbc/eagle-jpm/eagle-jpm-mr-running/src/main/java/org/apache/eagle/jpm/mr/running/storm/MRRunningJobFetchSpout.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-mr-running/src/main/java/org/apache/eagle/jpm/mr/running/storm/MRRunningJobFetchSpout.java b/eagle-jpm/eagle-jpm-mr-running/src/main/java/org/apache/eagle/jpm/mr/running/storm/MRRunningJobFetchSpout.java
index a701d74..ebb9144 100644
--- a/eagle-jpm/eagle-jpm-mr-running/src/main/java/org/apache/eagle/jpm/mr/running/storm/MRRunningJobFetchSpout.java
+++ b/eagle-jpm/eagle-jpm-mr-running/src/main/java/org/apache/eagle/jpm/mr/running/storm/MRRunningJobFetchSpout.java
@@ -29,9 +29,9 @@ import org.apache.eagle.jpm.mr.running.recover.MRRunningJobManager;
 import org.apache.eagle.jpm.mr.runningentity.JobExecutionAPIEntity;
 import org.apache.eagle.jpm.util.Constants;
 import org.apache.eagle.jpm.util.Utils;
-import org.apache.eagle.jpm.util.resourceFetch.RMResourceFetcher;
-import org.apache.eagle.jpm.util.resourceFetch.ResourceFetcher;
-import org.apache.eagle.jpm.util.resourceFetch.model.AppInfo;
+import org.apache.eagle.jpm.util.resourcefetch.RMResourceFetcher;
+import org.apache.eagle.jpm.util.resourcefetch.ResourceFetcher;
+import org.apache.eagle.jpm.util.resourcefetch.model.AppInfo;
 import org.apache.zookeeper.KeeperException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/0b852cbc/eagle-jpm/eagle-jpm-mr-running/src/main/java/org/apache/eagle/jpm/mr/running/storm/MRRunningJobParseBolt.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-mr-running/src/main/java/org/apache/eagle/jpm/mr/running/storm/MRRunningJobParseBolt.java b/eagle-jpm/eagle-jpm-mr-running/src/main/java/org/apache/eagle/jpm/mr/running/storm/MRRunningJobParseBolt.java
index 51307e1..0dccd70 100644
--- a/eagle-jpm/eagle-jpm-mr-running/src/main/java/org/apache/eagle/jpm/mr/running/storm/MRRunningJobParseBolt.java
+++ b/eagle-jpm/eagle-jpm-mr-running/src/main/java/org/apache/eagle/jpm/mr/running/storm/MRRunningJobParseBolt.java
@@ -28,9 +28,9 @@ import org.apache.eagle.jpm.mr.running.parser.MRJobParser;
 import org.apache.eagle.jpm.mr.running.recover.MRRunningJobManager;
 import org.apache.eagle.jpm.mr.runningentity.JobExecutionAPIEntity;
 import org.apache.eagle.jpm.util.Constants;
-import org.apache.eagle.jpm.util.resourceFetch.RMResourceFetcher;
-import org.apache.eagle.jpm.util.resourceFetch.ResourceFetcher;
-import org.apache.eagle.jpm.util.resourceFetch.model.AppInfo;
+import org.apache.eagle.jpm.util.resourcefetch.RMResourceFetcher;
+import org.apache.eagle.jpm.util.resourcefetch.ResourceFetcher;
+import org.apache.eagle.jpm.util.resourcefetch.model.AppInfo;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/0b852cbc/eagle-jpm/eagle-jpm-spark-history/src/main/java/org/apache/eagle/jpm/spark/history/config/SparkHistoryCrawlConfig.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-spark-history/src/main/java/org/apache/eagle/jpm/spark/history/config/SparkHistoryCrawlConfig.java b/eagle-jpm/eagle-jpm-spark-history/src/main/java/org/apache/eagle/jpm/spark/history/config/SparkHistoryCrawlConfig.java
index 4282a64..e6cd2f6 100644
--- a/eagle-jpm/eagle-jpm-spark-history/src/main/java/org/apache/eagle/jpm/spark/history/config/SparkHistoryCrawlConfig.java
+++ b/eagle-jpm/eagle-jpm-spark-history/src/main/java/org/apache/eagle/jpm/spark/history/config/SparkHistoryCrawlConfig.java
@@ -25,7 +25,7 @@ import com.typesafe.config.ConfigFactory;
 
 import java.io.Serializable;
 
-public class SparkHistoryCrawlConfig implements Serializable{
+public class SparkHistoryCrawlConfig implements Serializable {
     public ZKStateConfig zkStateConfig;
     public JobHistoryEndpointConfig jobHistoryConfig;
     public HDFSConfig hdfsConfig;
@@ -34,6 +34,7 @@ public class SparkHistoryCrawlConfig implements Serializable{
     public StormConfig stormConfig;
 
     private Config config;
+    
     public Config getConfig() {
         return config;
     }

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/0b852cbc/eagle-jpm/eagle-jpm-spark-history/src/main/java/org/apache/eagle/jpm/spark/history/status/JobHistoryZKStateManager.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-spark-history/src/main/java/org/apache/eagle/jpm/spark/history/status/JobHistoryZKStateManager.java b/eagle-jpm/eagle-jpm-spark-history/src/main/java/org/apache/eagle/jpm/spark/history/status/JobHistoryZKStateManager.java
index 0c475a9..382375f 100644
--- a/eagle-jpm/eagle-jpm-spark-history/src/main/java/org/apache/eagle/jpm/spark/history/status/JobHistoryZKStateManager.java
+++ b/eagle-jpm/eagle-jpm-spark-history/src/main/java/org/apache/eagle/jpm/spark/history/status/JobHistoryZKStateManager.java
@@ -19,18 +19,17 @@
 
 package org.apache.eagle.jpm.spark.history.status;
 
+import org.apache.eagle.jpm.spark.crawl.SparkApplicationInfo;
+import org.apache.eagle.jpm.spark.history.config.SparkHistoryCrawlConfig;
 import org.apache.curator.framework.CuratorFramework;
 import org.apache.curator.framework.CuratorFrameworkFactory;
 import org.apache.curator.framework.api.transaction.CuratorTransactionBridge;
 import org.apache.curator.framework.recipes.locks.InterProcessLock;
 import org.apache.curator.framework.recipes.locks.InterProcessReadWriteLock;
 import org.apache.curator.retry.RetryNTimes;
-import org.apache.eagle.jpm.spark.history.config.SparkHistoryCrawlConfig;
-import org.apache.eagle.jpm.spark.crawl.SparkApplicationInfo;
 import org.apache.zookeeper.CreateMode;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-
 import java.util.ArrayList;
 import java.util.Iterator;
 import java.util.List;
@@ -38,7 +37,7 @@ import java.util.List;
 public class JobHistoryZKStateManager {
     public static final Logger LOG = LoggerFactory.getLogger(JobHistoryZKStateManager.class);
     private String zkRoot;
-    private CuratorFramework _curator;
+    private CuratorFramework curator;
     private static String START_TIMESTAMP = "lastAppTime";
 
     private CuratorFramework newCurator(SparkHistoryCrawlConfig config) throws Exception {
@@ -54,46 +53,46 @@ public class JobHistoryZKStateManager {
         this.zkRoot = config.zkStateConfig.zkRoot + "/" + config.info.site;
 
         try {
-            _curator = newCurator(config);
-            _curator.start();
-;        } catch (Exception e) {
+            curator = newCurator(config);
+            curator.start();
+        } catch (Exception e) {
             LOG.error("Fail to connect to zookeeper", e);
             throw new RuntimeException(e);
         }
     }
 
     public void close() {
-        _curator.close();
-        _curator = null;
+        curator.close();
+        curator = null;
     }
 
-    public List<String> loadApplications(int limit){
+    public List<String> loadApplications(int limit) {
         String jobPath = zkRoot + "/jobs";
         List<String> apps = new ArrayList<>();
-        InterProcessLock lock = new InterProcessReadWriteLock(_curator,jobPath).writeLock();
-        try{
+        InterProcessLock lock = new InterProcessReadWriteLock(curator,jobPath).writeLock();
+        try {
             lock.acquire();
-            Iterator<String> iter =  _curator.getChildren().forPath(jobPath).iterator();
-            while(iter.hasNext()) {
+            Iterator<String> iter =  curator.getChildren().forPath(jobPath).iterator();
+            while (iter.hasNext()) {
                 String appId = iter.next();
                 String path = jobPath + "/" + appId;
-                if(_curator.checkExists().forPath(path) != null){
-                    if(new String(_curator.getData().forPath(path)).equals(ZKStateConstant.AppStatus.INIT.toString())){
+                if (curator.checkExists().forPath(path) != null) {
+                    if (new String(curator.getData().forPath(path)).equals(ZKStateConstant.AppStatus.INIT.toString())) {
                         apps.add(appId);
                     }
                 }
-                if(apps.size() == limit){
+                if (apps.size() == limit) {
                     break;
                 }
             }
             return apps;
-        }catch(Exception e){
+        } catch (Exception e) {
             LOG.error("fail to read unprocessed jobs", e);
             throw new RuntimeException(e);
-        }finally {
-            try{
+        } finally {
+            try {
                 lock.release();
-            }catch(Exception e){
+            } catch (Exception e) {
                 LOG.error("fail to release lock", e);
             }
 
@@ -102,18 +101,19 @@ public class JobHistoryZKStateManager {
 
     public void resetApplications() {
         String jobPath = zkRoot + "/jobs";
-        InterProcessLock lock = new InterProcessReadWriteLock(_curator,jobPath).writeLock();
+        InterProcessLock lock = new InterProcessReadWriteLock(curator,jobPath).writeLock();
         try {
             lock.acquire();
-            Iterator<String> iter =  _curator.getChildren().forPath(jobPath).iterator();
+            Iterator<String> iter =  curator.getChildren().forPath(jobPath).iterator();
             while (iter.hasNext()) {
                 String appId = iter.next();
                 String path = jobPath + "/" + appId;
                 try {
-                    if (_curator.checkExists().forPath(path) != null) {
-                        String status = new String(_curator.getData().forPath(path));
-                        if(!ZKStateConstant.AppStatus.INIT.toString().equals(status))
-                            _curator.setData().forPath(path, ZKStateConstant.AppStatus.INIT.toString().getBytes("UTF-8"));
+                    if (curator.checkExists().forPath(path) != null) {
+                        String status = new String(curator.getData().forPath(path));
+                        if (!ZKStateConstant.AppStatus.INIT.toString().equals(status)) {
+                            curator.setData().forPath(path, ZKStateConstant.AppStatus.INIT.toString().getBytes("UTF-8"));
+                        }
                     }
                 } catch (Exception e) {
                     LOG.error("fail to read unprocessed job", e);
@@ -127,136 +127,133 @@ public class JobHistoryZKStateManager {
         } finally {
             try {
                 lock.release();
-            } catch(Exception e) {
+            } catch (Exception e) {
                 LOG.error("fail to release lock", e);
             }
         }
     }
 
-    public SparkApplicationInfo getApplicationInfo(String appId){
+    public SparkApplicationInfo getApplicationInfo(String appId) {
 
-        String appPath = zkRoot + "/jobs/" + appId +"/info";
-        try{
+        String appPath = zkRoot + "/jobs/" + appId + "/info";
+        try {
             SparkApplicationInfo info = new SparkApplicationInfo();
-            if(_curator.checkExists().forPath(appPath)!= null){
-                String[] appStatus = new String(_curator.getData().forPath(appPath)).split("/");
+            if (curator.checkExists().forPath(appPath) != null) {
+                String[] appStatus = new String(curator.getData().forPath(appPath)).split("/");
                 info.setQueue(appStatus[0]);
                 info.setState(appStatus[1]);
                 info.setFinalStatus(appStatus[2]);
-                if(appStatus.length > 3){
+                if (appStatus.length > 3) {
                     info.setUser(appStatus[3]);
                     info.setName(appStatus[4]);
                 }
 
             }
             return info;
-        }catch(Exception e){
+        } catch (Exception e) {
             LOG.error("fail to read application attempt info", e);
             throw new RuntimeException(e);
         }
     }
 
-    public long readLastFinishedTimestamp(){
+    public long readLastFinishedTimestamp() {
         String lastTimeStampPath = zkRoot + "/" + START_TIMESTAMP;
 
-        try{
-            if(_curator.checkExists().forPath(lastTimeStampPath) == null){
-                return 0l;
-            }else{
-                return Long.valueOf(new String(_curator.getData().forPath(lastTimeStampPath)));
+        try {
+            if (curator.checkExists().forPath(lastTimeStampPath) == null) {
+                return 0L;
+            } else {
+                return Long.valueOf(new String(curator.getData().forPath(lastTimeStampPath)));
             }
-        }catch(Exception e){
+        } catch (Exception e) {
             LOG.error("fail to read last finished spark job timestamp", e);
             throw new RuntimeException(e);
         }
     }
 
-    public boolean hasApplication(String appId){
+    public boolean hasApplication(String appId) {
         String path = zkRoot + "/jobs/" + appId;
         try {
-            if (_curator.checkExists().forPath(path) != null) {
+            if (curator.checkExists().forPath(path) != null) {
                 return true;
             }
             return false;
-        }catch (Exception e){
+        } catch (Exception e) {
             LOG.error("fail to check whether application exists", e);
             throw new RuntimeException(e);
         }
     }
 
-    public void addFinishedApplication(String appId, String queue, String yarnState, String yarnStatus, String user, String name){
+    public void addFinishedApplication(String appId, String queue, String yarnState, String yarnStatus, String user, String name) {
         String path = zkRoot + "/jobs/" + appId;
-
-
-        try{
-            if(_curator.checkExists().forPath(path) != null){
-                _curator.delete().deletingChildrenIfNeeded().forPath(path);
+        try {
+            if (curator.checkExists().forPath(path) != null) {
+                curator.delete().deletingChildrenIfNeeded().forPath(path);
             }
 
             name = name.replace("/","_");
-            if(name.length() > 50){
+            if (name.length() > 50) {
                 name = name.substring(0, 50);
             }
 
-            CuratorTransactionBridge result =  _curator.inTransaction().create().withMode(CreateMode.PERSISTENT).forPath(path, ZKStateConstant.AppStatus.INIT.toString().getBytes("UTF-8"));
+            CuratorTransactionBridge result =  curator.inTransaction().create().withMode(CreateMode.PERSISTENT).forPath(path, ZKStateConstant.AppStatus.INIT.toString().getBytes("UTF-8"));
             result = result.and().create().withMode(CreateMode.PERSISTENT).forPath(path + "/info", String.format("%s/%s/%s/%s/%s", queue, yarnState, yarnStatus, user, name).getBytes("UTF-8"));
 
             result.and().commit();
-        }catch (Exception e){
+        } catch (Exception e) {
             LOG.error("fail adding finished application", e);
             throw new RuntimeException(e);
         }
     }
 
 
-    public void updateLastUpdateTime(Long updateTime){
+    public void updateLastUpdateTime(Long updateTime) {
         String lastTimeStampPath = zkRoot + "/" + START_TIMESTAMP;
-        try{
-            if(_curator.checkExists().forPath(lastTimeStampPath) == null){
-                _curator.create().creatingParentsIfNeeded().withMode(CreateMode.PERSISTENT).forPath(lastTimeStampPath, updateTime.toString().getBytes("UTF-8"));
-            }else{
+        try {
+            if (curator.checkExists().forPath(lastTimeStampPath) == null) {
+                curator.create().creatingParentsIfNeeded().withMode(CreateMode.PERSISTENT).forPath(lastTimeStampPath, updateTime.toString().getBytes("UTF-8"));
+            } else {
                 long originalEndTime = this.readLastFinishedTimestamp();
-                if(originalEndTime < updateTime){
-                   _curator.setData().forPath(lastTimeStampPath, updateTime.toString().getBytes("UTF-8"));
+                if (originalEndTime < updateTime) {
+                    curator.setData().forPath(lastTimeStampPath, updateTime.toString().getBytes("UTF-8"));
                 }
             }
-        }catch (Exception e){
+        } catch (Exception e) {
             LOG.error("fail to update last finished time", e);
             throw new RuntimeException(e);
         }
 
     }
 
-    public void updateApplicationStatus(String appId, Enum<ZKStateConstant.AppStatus> status){
+    public void updateApplicationStatus(String appId, Enum<ZKStateConstant.AppStatus> status) {
 
         String path = zkRoot + "/jobs/" + appId ;
-        InterProcessLock lock = new InterProcessReadWriteLock(_curator,zkRoot+"/jobs").readLock();
-        try{
-            if(_curator.checkExists().forPath(path) != null){
-                if(status.equals(ZKStateConstant.AppStatus.FINISHED)){
+        InterProcessLock lock = new InterProcessReadWriteLock(curator,zkRoot + "/jobs").readLock();
+        try {
+            if (curator.checkExists().forPath(path) != null) {
+                if (status.equals(ZKStateConstant.AppStatus.FINISHED)) {
                     lock.acquire();
-                    _curator.delete().deletingChildrenIfNeeded().forPath(path);
-                }else{
-                    _curator.setData().forPath(path, status.toString().getBytes("UTF-8"));
+                    curator.delete().deletingChildrenIfNeeded().forPath(path);
+                } else {
+                    curator.setData().forPath(path, status.toString().getBytes("UTF-8"));
                 }
-            }else{
+            } else {
                 String errorMsg = String.format("fail to update for application with path %s", path);
                 LOG.error(errorMsg);
             }
-        }catch (Exception e){
+        } catch (Exception e) {
             LOG.error("fail to update application status", e);
             throw new RuntimeException(e);
-        }finally{
-            try{
-                if(lock.isAcquiredInThisProcess())
+        } finally {
+            try {
+                if (lock.isAcquiredInThisProcess()) {
                     lock.release();
-            }catch (Exception e){
+                }
+            } catch (Exception e) {
                 LOG.error("fail to release lock",e);
             }
 
         }
 
     }
-
-
 }

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/0b852cbc/eagle-jpm/eagle-jpm-spark-history/src/main/java/org/apache/eagle/jpm/spark/history/status/ZKStateConstant.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-spark-history/src/main/java/org/apache/eagle/jpm/spark/history/status/ZKStateConstant.java b/eagle-jpm/eagle-jpm-spark-history/src/main/java/org/apache/eagle/jpm/spark/history/status/ZKStateConstant.java
index 40efa50..578ba3c 100644
--- a/eagle-jpm/eagle-jpm-spark-history/src/main/java/org/apache/eagle/jpm/spark/history/status/ZKStateConstant.java
+++ b/eagle-jpm/eagle-jpm-spark-history/src/main/java/org/apache/eagle/jpm/spark/history/status/ZKStateConstant.java
@@ -21,7 +21,7 @@ package org.apache.eagle.jpm.spark.history.status;
 
 public class ZKStateConstant {
 
-    public enum AppStatus{
+    public enum AppStatus {
         INIT, SENT_FOR_PARSE, FINISHED, FAILED
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/0b852cbc/eagle-jpm/eagle-jpm-spark-history/src/main/java/org/apache/eagle/jpm/spark/history/storm/FinishedSparkJobSpout.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-spark-history/src/main/java/org/apache/eagle/jpm/spark/history/storm/FinishedSparkJobSpout.java b/eagle-jpm/eagle-jpm-spark-history/src/main/java/org/apache/eagle/jpm/spark/history/storm/FinishedSparkJobSpout.java
index 8404eda..8965d3d 100644
--- a/eagle-jpm/eagle-jpm-spark-history/src/main/java/org/apache/eagle/jpm/spark/history/storm/FinishedSparkJobSpout.java
+++ b/eagle-jpm/eagle-jpm-spark-history/src/main/java/org/apache/eagle/jpm/spark/history/storm/FinishedSparkJobSpout.java
@@ -19,26 +19,24 @@
 
 package org.apache.eagle.jpm.spark.history.storm;
 
+import org.apache.eagle.jpm.spark.history.config.SparkHistoryCrawlConfig;
+import org.apache.eagle.jpm.spark.history.status.JobHistoryZKStateManager;
+import org.apache.eagle.jpm.spark.history.status.ZKStateConstant;
+import org.apache.eagle.jpm.util.Constants;
+import org.apache.eagle.jpm.util.resourcefetch.RMResourceFetcher;
+import org.apache.eagle.jpm.util.resourcefetch.ResourceFetcher;
+import org.apache.eagle.jpm.util.resourcefetch.model.AppInfo;
 import backtype.storm.spout.SpoutOutputCollector;
 import backtype.storm.task.TopologyContext;
 import backtype.storm.topology.OutputFieldsDeclarer;
 import backtype.storm.topology.base.BaseRichSpout;
 import backtype.storm.tuple.Fields;
 import backtype.storm.tuple.Values;
-import org.apache.eagle.jpm.spark.history.config.SparkHistoryCrawlConfig;
-import org.apache.eagle.jpm.spark.history.status.JobHistoryZKStateManager;
-import org.apache.eagle.jpm.spark.history.status.ZKStateConstant;
-import org.apache.eagle.jpm.util.Constants;
-import org.apache.eagle.jpm.util.resourceFetch.RMResourceFetcher;
-import org.apache.eagle.jpm.util.resourceFetch.ResourceFetcher;
-import org.apache.eagle.jpm.util.resourceFetch.model.AppInfo;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-
 import java.util.Calendar;
 import java.util.HashMap;
 import java.util.List;
-import java.util.ArrayList;
 import java.util.Map;
 
 public class FinishedSparkJobSpout extends BaseRichSpout {
@@ -52,7 +50,7 @@ public class FinishedSparkJobSpout extends BaseRichSpout {
 
     private static final int FAIL_MAX_TIMES = 5;
 
-    public FinishedSparkJobSpout(SparkHistoryCrawlConfig config){
+    public FinishedSparkJobSpout(SparkHistoryCrawlConfig config) {
         this.config = config;
     }
 
@@ -102,15 +100,14 @@ public class FinishedSparkJobSpout extends BaseRichSpout {
             }
         } catch (Exception e) {
             LOG.error("Fail to run next tuple", e);
-           // this.takeRest(10);
         }
-
     }
 
     private void takeRest(int seconds) {
         try {
             Thread.sleep(seconds * 1000);
-        } catch(InterruptedException e) {
+        } catch (InterruptedException e) {
+            LOG.warn("exception found {}", e);
         }
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/0b852cbc/eagle-jpm/eagle-jpm-spark-history/src/main/java/org/apache/eagle/jpm/spark/history/storm/SparkHistoryTopology.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-spark-history/src/main/java/org/apache/eagle/jpm/spark/history/storm/SparkHistoryTopology.java b/eagle-jpm/eagle-jpm-spark-history/src/main/java/org/apache/eagle/jpm/spark/history/storm/SparkHistoryTopology.java
index bd0eb85..423dbef 100644
--- a/eagle-jpm/eagle-jpm-spark-history/src/main/java/org/apache/eagle/jpm/spark/history/storm/SparkHistoryTopology.java
+++ b/eagle-jpm/eagle-jpm-spark-history/src/main/java/org/apache/eagle/jpm/spark/history/storm/SparkHistoryTopology.java
@@ -19,32 +19,32 @@
 
 package org.apache.eagle.jpm.spark.history.storm;
 
+import org.apache.eagle.jpm.spark.history.config.SparkHistoryCrawlConfig;
 import backtype.storm.Config;
 import backtype.storm.LocalCluster;
 import backtype.storm.StormSubmitter;
 import backtype.storm.topology.TopologyBuilder;
-import org.apache.eagle.jpm.spark.history.config.SparkHistoryCrawlConfig;
 
 public class SparkHistoryTopology {
 
-    private SparkHistoryCrawlConfig SHConfig;
+    private SparkHistoryCrawlConfig sparkHistoryCrawlConfig;
 
-    public SparkHistoryTopology(SparkHistoryCrawlConfig config){
-        this.SHConfig = config;
+    public SparkHistoryTopology(SparkHistoryCrawlConfig config) {
+        this.sparkHistoryCrawlConfig = config;
     }
 
     public TopologyBuilder getBuilder() {
         TopologyBuilder builder = new TopologyBuilder();
         String spoutName = "sparkHistoryJobSpout";
         String boltName = "sparkHistoryJobBolt";
-        com.typesafe.config.Config config = this.SHConfig.getConfig();
+        com.typesafe.config.Config config = this.sparkHistoryCrawlConfig.getConfig();
         builder.setSpout(spoutName,
-                new FinishedSparkJobSpout(SHConfig),
+                new FinishedSparkJobSpout(sparkHistoryCrawlConfig),
                 config.getInt("storm.parallelismConfig." + spoutName)
         ).setNumTasks(config.getInt("storm.tasks." + spoutName));
 
         builder.setBolt(boltName,
-                new SparkJobParseBolt(SHConfig),
+                new SparkJobParseBolt(sparkHistoryCrawlConfig),
                 config.getInt("storm.parallelismConfig." + boltName)
         ).setNumTasks(config.getInt("storm.tasks." + boltName)).shuffleGrouping(spoutName);
         return builder;

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/0b852cbc/eagle-jpm/eagle-jpm-spark-history/src/main/java/org/apache/eagle/jpm/spark/history/storm/SparkJobParseBolt.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-spark-history/src/main/java/org/apache/eagle/jpm/spark/history/storm/SparkJobParseBolt.java b/eagle-jpm/eagle-jpm-spark-history/src/main/java/org/apache/eagle/jpm/spark/history/storm/SparkJobParseBolt.java
index 23d5152..f00fa1b 100644
--- a/eagle-jpm/eagle-jpm-spark-history/src/main/java/org/apache/eagle/jpm/spark/history/storm/SparkJobParseBolt.java
+++ b/eagle-jpm/eagle-jpm-spark-history/src/main/java/org/apache/eagle/jpm/spark/history/storm/SparkJobParseBolt.java
@@ -19,11 +19,6 @@
 
 package org.apache.eagle.jpm.spark.history.storm;
 
-import backtype.storm.task.OutputCollector;
-import backtype.storm.task.TopologyContext;
-import backtype.storm.topology.OutputFieldsDeclarer;
-import backtype.storm.topology.base.BaseRichBolt;
-import backtype.storm.tuple.Tuple;
 import org.apache.eagle.jpm.spark.crawl.JHFInputStreamReader;
 import org.apache.eagle.jpm.spark.crawl.SparkApplicationInfo;
 import org.apache.eagle.jpm.spark.crawl.SparkFilesystemInputStreamReaderImpl;
@@ -31,9 +26,14 @@ import org.apache.eagle.jpm.spark.history.config.SparkHistoryCrawlConfig;
 import org.apache.eagle.jpm.spark.history.status.JobHistoryZKStateManager;
 import org.apache.eagle.jpm.spark.history.status.ZKStateConstant;
 import org.apache.eagle.jpm.util.HDFSUtil;
-import org.apache.eagle.jpm.util.resourceFetch.ResourceFetcher;
-import org.apache.eagle.jpm.util.resourceFetch.SparkHistoryServerResourceFetcher;
-import org.apache.eagle.jpm.util.resourceFetch.model.SparkApplication;
+import org.apache.eagle.jpm.util.resourcefetch.ResourceFetcher;
+import org.apache.eagle.jpm.util.resourcefetch.SparkHistoryServerResourceFetcher;
+import org.apache.eagle.jpm.util.resourcefetch.model.SparkApplication;
+import backtype.storm.task.OutputCollector;
+import backtype.storm.task.TopologyContext;
+import backtype.storm.topology.OutputFieldsDeclarer;
+import backtype.storm.topology.base.BaseRichBolt;
+import backtype.storm.tuple.Tuple;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/0b852cbc/eagle-jpm/eagle-jpm-spark-history/src/main/java/org/apache/eagle/jpm/spark/history/storm/TestHDFS.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-spark-history/src/main/java/org/apache/eagle/jpm/spark/history/storm/TestHDFS.java b/eagle-jpm/eagle-jpm-spark-history/src/main/java/org/apache/eagle/jpm/spark/history/storm/TestHDFS.java
index eb30f5e..f4284e1 100644
--- a/eagle-jpm/eagle-jpm-spark-history/src/main/java/org/apache/eagle/jpm/spark/history/storm/TestHDFS.java
+++ b/eagle-jpm/eagle-jpm-spark-history/src/main/java/org/apache/eagle/jpm/spark/history/storm/TestHDFS.java
@@ -28,9 +28,9 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 public class TestHDFS {
-
     private static final Logger LOG = LoggerFactory.getLogger(TestHDFS.class);
-    public static void main(String[] args) throws Exception{
+
+    public static void main(String[] args) throws Exception {
         SparkHistoryCrawlConfig config = new SparkHistoryCrawlConfig();
 
         Configuration conf  = new Configuration();

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/0b852cbc/eagle-jpm/eagle-jpm-spark-running/src/main/java/org/apache/eagle/jpm/spark/running/SparkRunningJobApp.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-spark-running/src/main/java/org/apache/eagle/jpm/spark/running/SparkRunningJobApp.java b/eagle-jpm/eagle-jpm-spark-running/src/main/java/org/apache/eagle/jpm/spark/running/SparkRunningJobApp.java
index 61c0751..5e21406 100644
--- a/eagle-jpm/eagle-jpm-spark-running/src/main/java/org/apache/eagle/jpm/spark/running/SparkRunningJobApp.java
+++ b/eagle-jpm/eagle-jpm-spark-running/src/main/java/org/apache/eagle/jpm/spark/running/SparkRunningJobApp.java
@@ -16,14 +16,15 @@
  */
 package org.apache.eagle.jpm.spark.running;
 
-import backtype.storm.generated.StormTopology;
-import backtype.storm.topology.TopologyBuilder;
-import backtype.storm.tuple.Fields;
-import com.typesafe.config.Config;
+
 import org.apache.eagle.app.StormApplication;
 import org.apache.eagle.app.environment.impl.StormEnvironment;
 import org.apache.eagle.jpm.spark.running.storm.SparkRunningJobFetchSpout;
 import org.apache.eagle.jpm.spark.running.storm.SparkRunningJobParseBolt;
+import backtype.storm.generated.StormTopology;
+import backtype.storm.topology.TopologyBuilder;
+import backtype.storm.tuple.Fields;
+import com.typesafe.config.Config;
 
 public class SparkRunningJobApp extends StormApplication {
     @Override

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/0b852cbc/eagle-jpm/eagle-jpm-spark-running/src/main/java/org/apache/eagle/jpm/spark/running/SparkRunningJobAppConfig.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-spark-running/src/main/java/org/apache/eagle/jpm/spark/running/SparkRunningJobAppConfig.java b/eagle-jpm/eagle-jpm-spark-running/src/main/java/org/apache/eagle/jpm/spark/running/SparkRunningJobAppConfig.java
index 5988273..d9c66e3 100644
--- a/eagle-jpm/eagle-jpm-spark-running/src/main/java/org/apache/eagle/jpm/spark/running/SparkRunningJobAppConfig.java
+++ b/eagle-jpm/eagle-jpm-spark-running/src/main/java/org/apache/eagle/jpm/spark/running/SparkRunningJobAppConfig.java
@@ -18,8 +18,8 @@
 
 package org.apache.eagle.jpm.spark.running;
 
-import com.typesafe.config.Config;
 import org.apache.eagle.common.config.ConfigOptionParser;
+import com.typesafe.config.Config;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -33,28 +33,36 @@ public class SparkRunningJobAppConfig implements Serializable {
     public String getEnv() {
         return env;
     }
+
     private String env;
 
-    ZKStateConfig getZkStateConfig() { return zkStateConfig; }
+    ZKStateConfig getZkStateConfig() {
+        return zkStateConfig;
+    }
+
     private ZKStateConfig zkStateConfig;
     private TopologyConfig topologyConfig;
-    public TopologyConfig getTopologyConfig(){
+
+    public TopologyConfig getTopologyConfig() {
         return topologyConfig;
     }
 
     public EagleServiceConfig getEagleServiceConfig() {
         return eagleServiceConfig;
     }
+
     private EagleServiceConfig eagleServiceConfig;
 
     public JobExtractorConfig getJobExtractorConfig() {
         return jobExtractorConfig;
     }
+
     private JobExtractorConfig jobExtractorConfig;
 
     public EndpointConfig getEndpointConfig() {
         return endpointConfig;
     }
+
     private EndpointConfig endpointConfig;
 
     public static class TopologyConfig implements Serializable {
@@ -100,6 +108,7 @@ public class SparkRunningJobAppConfig implements Serializable {
     public Config getConfig() {
         return config;
     }
+
     private Config config;
 
     private static SparkRunningJobAppConfig manager = new SparkRunningJobAppConfig();
@@ -127,7 +136,7 @@ public class SparkRunningJobAppConfig implements Serializable {
         return manager;
     }
 
-    private void init(Config config){
+    private void init(Config config) {
         this.config = config;
         this.env = config.getString("envContextConfig.env");
         this.zkStateConfig.zkQuorum = config.getString("zookeeperConfig.zkQuorum");

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/0b852cbc/eagle-jpm/eagle-jpm-spark-running/src/main/java/org/apache/eagle/jpm/spark/running/entities/SparkAppEntity.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-spark-running/src/main/java/org/apache/eagle/jpm/spark/running/entities/SparkAppEntity.java b/eagle-jpm/eagle-jpm-spark-running/src/main/java/org/apache/eagle/jpm/spark/running/entities/SparkAppEntity.java
index 7b8f648..7de1530 100644
--- a/eagle-jpm/eagle-jpm-spark-running/src/main/java/org/apache/eagle/jpm/spark/running/entities/SparkAppEntity.java
+++ b/eagle-jpm/eagle-jpm-spark-running/src/main/java/org/apache/eagle/jpm/spark/running/entities/SparkAppEntity.java
@@ -17,11 +17,11 @@
 
 package org.apache.eagle.jpm.spark.running.entities;
 
-import org.apache.eagle.jpm.util.resourceFetch.model.AppInfo;
+import org.apache.eagle.jpm.util.Constants;
+import org.apache.eagle.jpm.util.resourcefetch.model.AppInfo;
 import org.apache.eagle.log.base.taggedlog.TaggedLogAPIEntity;
 import org.apache.eagle.log.entity.meta.*;
 import org.codehaus.jackson.annotate.JsonIgnoreProperties;
-import org.apache.eagle.jpm.util.Constants;
 
 @Table("eagleSparkRunningApps")
 @ColumnFamily("f")
@@ -265,11 +265,14 @@ public class SparkAppEntity extends TaggedLogAPIEntity {
         return driveMemoryBytes;
     }
 
-    public int getCompleteTasks(){ return completeTasks;}
+    public int getCompleteTasks() {
+        return completeTasks;
+    }
 
     public JobConfig getConfig() {
         return config;
     }
+
     public void setStartTime(long startTime) {
         this.startTime = startTime;
         valueChanged("startTime");
@@ -420,7 +423,7 @@ public class SparkAppEntity extends TaggedLogAPIEntity {
         valueChanged("driveMemoryBytes");
     }
 
-    public void setCompleteTasks(int completeTasks){
+    public void setCompleteTasks(int completeTasks) {
         this.completeTasks = completeTasks;
         valueChanged("completeTasks");
     }

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/0b852cbc/eagle-jpm/eagle-jpm-spark-running/src/main/java/org/apache/eagle/jpm/spark/running/entities/SparkExecutorEntity.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-spark-running/src/main/java/org/apache/eagle/jpm/spark/running/entities/SparkExecutorEntity.java b/eagle-jpm/eagle-jpm-spark-running/src/main/java/org/apache/eagle/jpm/spark/running/entities/SparkExecutorEntity.java
index f4de84c..89549ca 100644
--- a/eagle-jpm/eagle-jpm-spark-running/src/main/java/org/apache/eagle/jpm/spark/running/entities/SparkExecutorEntity.java
+++ b/eagle-jpm/eagle-jpm-spark-running/src/main/java/org/apache/eagle/jpm/spark/running/entities/SparkExecutorEntity.java
@@ -17,10 +17,10 @@
 
 package org.apache.eagle.jpm.spark.running.entities;
 
+import org.apache.eagle.jpm.util.Constants;
 import org.apache.eagle.log.base.taggedlog.TaggedLogAPIEntity;
 import org.apache.eagle.log.entity.meta.*;
 import org.codehaus.jackson.annotate.JsonIgnoreProperties;
-import org.apache.eagle.jpm.util.Constants;
 
 @Table("eagleSparkRunningExecutors")
 @ColumnFamily("f")

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/0b852cbc/eagle-jpm/eagle-jpm-spark-running/src/main/java/org/apache/eagle/jpm/spark/running/entities/SparkJobEntity.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-spark-running/src/main/java/org/apache/eagle/jpm/spark/running/entities/SparkJobEntity.java b/eagle-jpm/eagle-jpm-spark-running/src/main/java/org/apache/eagle/jpm/spark/running/entities/SparkJobEntity.java
index 1c2caa4..bb56b52 100644
--- a/eagle-jpm/eagle-jpm-spark-running/src/main/java/org/apache/eagle/jpm/spark/running/entities/SparkJobEntity.java
+++ b/eagle-jpm/eagle-jpm-spark-running/src/main/java/org/apache/eagle/jpm/spark/running/entities/SparkJobEntity.java
@@ -17,10 +17,10 @@
 
 package org.apache.eagle.jpm.spark.running.entities;
 
+import org.apache.eagle.jpm.util.Constants;
 import org.apache.eagle.log.base.taggedlog.TaggedLogAPIEntity;
 import org.apache.eagle.log.entity.meta.*;
 import org.codehaus.jackson.annotate.JsonIgnoreProperties;
-import org.apache.eagle.jpm.util.Constants;
 
 import java.util.List;
 

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/0b852cbc/eagle-jpm/eagle-jpm-spark-running/src/main/java/org/apache/eagle/jpm/spark/running/entities/SparkStageEntity.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-spark-running/src/main/java/org/apache/eagle/jpm/spark/running/entities/SparkStageEntity.java b/eagle-jpm/eagle-jpm-spark-running/src/main/java/org/apache/eagle/jpm/spark/running/entities/SparkStageEntity.java
index 72dbe40..be0ffd0 100644
--- a/eagle-jpm/eagle-jpm-spark-running/src/main/java/org/apache/eagle/jpm/spark/running/entities/SparkStageEntity.java
+++ b/eagle-jpm/eagle-jpm-spark-running/src/main/java/org/apache/eagle/jpm/spark/running/entities/SparkStageEntity.java
@@ -17,10 +17,10 @@
 
 package org.apache.eagle.jpm.spark.running.entities;
 
+import org.apache.eagle.jpm.util.Constants;
 import org.apache.eagle.log.base.taggedlog.TaggedLogAPIEntity;
 import org.apache.eagle.log.entity.meta.*;
 import org.codehaus.jackson.annotate.JsonIgnoreProperties;
-import org.apache.eagle.jpm.util.Constants;
 
 @Table("eagleSparkRunningStages")
 @ColumnFamily("f")
@@ -40,27 +40,27 @@ public class SparkStageEntity extends TaggedLogAPIEntity {
     @Column("d")
     private int numFailedTasks = 0;
     @Column("e")
-    private long executorRunTime = 0l;
+    private long executorRunTime = 0L;
     @Column("f")
-    private long inputBytes = 0l;
+    private long inputBytes = 0L;
     @Column("g")
-    private long inputRecords = 0l;
+    private long inputRecords = 0L;
     @Column("h")
-    private long outputBytes = 0l;
+    private long outputBytes = 0L;
     @Column("i")
-    private long outputRecords = 0l;
+    private long outputRecords = 0L;
     @Column("j")
-    private long shuffleReadBytes = 0l;
+    private long shuffleReadBytes = 0L;
     @Column("k")
-    private long shuffleReadRecords = 0l;
+    private long shuffleReadRecords = 0L;
     @Column("l")
-    private long shuffleWriteBytes = 0l;
+    private long shuffleWriteBytes = 0L;
     @Column("m")
-    private long shuffleWriteRecords = 0l;
+    private long shuffleWriteRecords = 0L;
     @Column("n")
-    private long memoryBytesSpilled = 0l;
+    private long memoryBytesSpilled = 0L;
     @Column("o")
-    private long diskBytesSpilled = 0l;
+    private long diskBytesSpilled = 0L;
     @Column("p")
     private String name;
     @Column("q")

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/0b852cbc/eagle-jpm/eagle-jpm-spark-running/src/main/java/org/apache/eagle/jpm/spark/running/entities/SparkTaskEntity.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-spark-running/src/main/java/org/apache/eagle/jpm/spark/running/entities/SparkTaskEntity.java b/eagle-jpm/eagle-jpm-spark-running/src/main/java/org/apache/eagle/jpm/spark/running/entities/SparkTaskEntity.java
index 183a62a..e531806 100644
--- a/eagle-jpm/eagle-jpm-spark-running/src/main/java/org/apache/eagle/jpm/spark/running/entities/SparkTaskEntity.java
+++ b/eagle-jpm/eagle-jpm-spark-running/src/main/java/org/apache/eagle/jpm/spark/running/entities/SparkTaskEntity.java
@@ -17,10 +17,10 @@
 
 package org.apache.eagle.jpm.spark.running.entities;
 
+import org.apache.eagle.jpm.util.Constants;
 import org.apache.eagle.log.base.taggedlog.TaggedLogAPIEntity;
 import org.apache.eagle.log.entity.meta.*;
 import org.codehaus.jackson.annotate.JsonIgnoreProperties;
-import org.apache.eagle.jpm.util.Constants;
 
 @Table("eagleSparkRunningTasks")
 @ColumnFamily("f")

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/0b852cbc/eagle-jpm/eagle-jpm-spark-running/src/main/java/org/apache/eagle/jpm/spark/running/parser/SparkApplicationParser.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-spark-running/src/main/java/org/apache/eagle/jpm/spark/running/parser/SparkApplicationParser.java b/eagle-jpm/eagle-jpm-spark-running/src/main/java/org/apache/eagle/jpm/spark/running/parser/SparkApplicationParser.java
index b2a5b63..6411018 100644
--- a/eagle-jpm/eagle-jpm-spark-running/src/main/java/org/apache/eagle/jpm/spark/running/parser/SparkApplicationParser.java
+++ b/eagle-jpm/eagle-jpm-spark-running/src/main/java/org/apache/eagle/jpm/spark/running/parser/SparkApplicationParser.java
@@ -18,7 +18,6 @@
 
 package org.apache.eagle.jpm.spark.running.parser;
 
-import org.apache.commons.lang3.tuple.Pair;
 import org.apache.eagle.jpm.spark.crawl.EventType;
 import org.apache.eagle.jpm.spark.running.SparkRunningJobAppConfig;
 import org.apache.eagle.jpm.spark.running.entities.*;
@@ -27,9 +26,10 @@ import org.apache.eagle.jpm.util.Constants;
 import org.apache.eagle.jpm.util.HDFSUtil;
 import org.apache.eagle.jpm.util.SparkJobTagName;
 import org.apache.eagle.jpm.util.Utils;
-import org.apache.eagle.jpm.util.resourceFetch.ResourceFetcher;
-import org.apache.eagle.jpm.util.resourceFetch.connection.InputStreamUtils;
-import org.apache.eagle.jpm.util.resourceFetch.model.*;
+import org.apache.eagle.jpm.util.resourcefetch.ResourceFetcher;
+import org.apache.eagle.jpm.util.resourcefetch.connection.InputStreamUtils;
+import org.apache.eagle.jpm.util.resourcefetch.model.*;
+import org.apache.commons.lang3.tuple.Pair;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -189,12 +189,12 @@ public class SparkApplicationParser implements Runnable {
                     //we must flush entities before delete from zk in case of missing finish state of jobs
                     //delete from zk if needed
                     sparkAppEntityMap.keySet()
-                            .stream()
-                            .filter(
-                                    jobId -> sparkAppEntityMap.get(jobId).getYarnState().equals(Constants.AppState.FINISHED.toString()) ||
-                                            sparkAppEntityMap.get(jobId).getYarnState().equals(Constants.AppState.FAILED.toString()))
-                            .forEach(
-                                    jobId -> this.sparkRunningJobManager.delete(app.getId(), jobId));
+                        .stream()
+                        .filter(
+                            jobId -> sparkAppEntityMap.get(jobId).getYarnState().equals(Constants.AppState.FINISHED.toString())
+                                || sparkAppEntityMap.get(jobId).getYarnState().equals(Constants.AppState.FAILED.toString()))
+                        .forEach(
+                            jobId -> this.sparkRunningJobManager.delete(app.getId(), jobId));
                 }
 
                 LOG.info("finish process yarn application " + app.getId());
@@ -243,12 +243,12 @@ public class SparkApplicationParser implements Runnable {
         JobConfig jobConfig = null;
 
         try (FileSystem hdfs = HDFSUtil.getFileSystem(this.hdfsConf)) {
-//             // For Yarn version >= 2.7,
-//             // log name: "application_1468625664674_0003_appattempt_1468625664674_0003_000001"
-//             String attemptIdFormatted = String.format("%06d", attemptId);
-//             // remove "application_" to get the number part of appID.
-//             String sparkAppIdNum = sparkAppId.substring(12);
-//             String attemptIdString = "appattempt_" + sparkAppIdNum + "_" + attemptIdFormatted;
+            //             // For Yarn version >= 2.7,
+            //             // log name: "application_1468625664674_0003_appattempt_1468625664674_0003_000001"
+            //             String attemptIdFormatted = String.format("%06d", attemptId);
+            //             // remove "application_" to get the number part of appID.
+            //             String sparkAppIdNum = sparkAppId.substring(12);
+            //             String attemptIdString = "appattempt_" + sparkAppIdNum + "_" + attemptIdFormatted;
 
             // For Yarn version 2.4.x
             // log name: application_1464382345557_269065_1
@@ -277,8 +277,8 @@ public class SparkApplicationParser implements Runnable {
     }
 
     private boolean isClientMode(JobConfig jobConfig) {
-        return jobConfig.containsKey(Constants.SPARK_MASTER_KEY) &&
-               jobConfig.get(Constants.SPARK_MASTER_KEY).equalsIgnoreCase("yarn-client");
+        return jobConfig.containsKey(Constants.SPARK_MASTER_KEY)
+            && jobConfig.get(Constants.SPARK_MASTER_KEY).equalsIgnoreCase("yarn-client");
     }
 
     private boolean fetchSparkApps() {
@@ -315,10 +315,10 @@ public class SparkApplicationParser implements Runnable {
                 lastSavedAttempt = Integer.parseInt(sparkAppEntityMap.get(id).getTags().get(SparkJobTagName.SPARK_APP_ATTEMPT_ID.toString()));
             }
             for (int j = lastSavedAttempt; j <= currentAttempt; j++) {
-                SparkAppEntity attemptEntity = new SparkAppEntity();
                 commonTags.put(SparkJobTagName.SPARK_APP_NAME.toString(), sparkApplication.getName());
                 commonTags.put(SparkJobTagName.SPARK_APP_ATTEMPT_ID.toString(), "" + j);
                 commonTags.put(SparkJobTagName.SPARK_APP_ID.toString(), id);
+                SparkAppEntity attemptEntity = new SparkAppEntity();
                 attemptEntity.setTags(new HashMap<>(commonTags));
                 attemptEntity.setAppInfo(app);
 
@@ -340,9 +340,9 @@ public class SparkApplicationParser implements Runnable {
                     JobConfig jobConfig = attemptEntity.getConfig();
                     attemptEntity.setExecMemoryBytes(Utils.parseMemory(jobConfig.get(Constants.SPARK_EXECUTOR_MEMORY_KEY)));
 
-                    attemptEntity.setDriveMemoryBytes(isClientMode(jobConfig) ?
-                            0 :
-                            Utils.parseMemory(jobConfig.get(Constants.SPARK_DRIVER_MEMORY_KEY)));
+                    attemptEntity.setDriveMemoryBytes(isClientMode(jobConfig)
+                        ? 0
+                        : Utils.parseMemory(jobConfig.get(Constants.SPARK_DRIVER_MEMORY_KEY)));
                     attemptEntity.setExecutorCores(Integer.parseInt(jobConfig.get(Constants.SPARK_EXECUTOR_CORES_KEY)));
                     // spark.driver.cores may not be set.
                     String driverCoresStr = jobConfig.get(Constants.SPARK_DRIVER_CORES_KEY);

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/0b852cbc/eagle-jpm/eagle-jpm-spark-running/src/main/java/org/apache/eagle/jpm/spark/running/recover/SparkRunningJobManager.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-spark-running/src/main/java/org/apache/eagle/jpm/spark/running/recover/SparkRunningJobManager.java b/eagle-jpm/eagle-jpm-spark-running/src/main/java/org/apache/eagle/jpm/spark/running/recover/SparkRunningJobManager.java
index 11f7909..3fb6371 100644
--- a/eagle-jpm/eagle-jpm-spark-running/src/main/java/org/apache/eagle/jpm/spark/running/recover/SparkRunningJobManager.java
+++ b/eagle-jpm/eagle-jpm-spark-running/src/main/java/org/apache/eagle/jpm/spark/running/recover/SparkRunningJobManager.java
@@ -18,14 +18,15 @@
 
 package org.apache.eagle.jpm.spark.running.recover;
 
-import org.apache.commons.lang3.tuple.Pair;
 import org.apache.eagle.jpm.spark.running.SparkRunningJobAppConfig;
 import org.apache.eagle.jpm.spark.running.entities.SparkAppEntity;
 import org.apache.eagle.jpm.util.jobrecover.RunningJobManager;
-import org.apache.eagle.jpm.util.resourceFetch.model.AppInfo;
+import org.apache.eagle.jpm.util.resourcefetch.model.AppInfo;
+import org.apache.commons.lang3.tuple.Pair;
 
 import java.io.Serializable;
-import java.util.*;
+import java.util.HashMap;
+import java.util.Map;
 
 public class SparkRunningJobManager implements Serializable {
     private RunningJobManager runningJobManager;

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/0b852cbc/eagle-jpm/eagle-jpm-spark-running/src/main/java/org/apache/eagle/jpm/spark/running/storm/SparkRunningJobFetchSpout.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-spark-running/src/main/java/org/apache/eagle/jpm/spark/running/storm/SparkRunningJobFetchSpout.java b/eagle-jpm/eagle-jpm-spark-running/src/main/java/org/apache/eagle/jpm/spark/running/storm/SparkRunningJobFetchSpout.java
index 256829e..ccdfe79 100644
--- a/eagle-jpm/eagle-jpm-spark-running/src/main/java/org/apache/eagle/jpm/spark/running/storm/SparkRunningJobFetchSpout.java
+++ b/eagle-jpm/eagle-jpm-spark-running/src/main/java/org/apache/eagle/jpm/spark/running/storm/SparkRunningJobFetchSpout.java
@@ -28,9 +28,9 @@ import org.apache.eagle.jpm.spark.running.SparkRunningJobAppConfig;
 import org.apache.eagle.jpm.spark.running.entities.SparkAppEntity;
 import org.apache.eagle.jpm.spark.running.recover.SparkRunningJobManager;
 import org.apache.eagle.jpm.util.Constants;
-import org.apache.eagle.jpm.util.resourceFetch.RMResourceFetcher;
-import org.apache.eagle.jpm.util.resourceFetch.ResourceFetcher;
-import org.apache.eagle.jpm.util.resourceFetch.model.AppInfo;
+import org.apache.eagle.jpm.util.resourcefetch.RMResourceFetcher;
+import org.apache.eagle.jpm.util.resourcefetch.ResourceFetcher;
+import org.apache.eagle.jpm.util.resourcefetch.model.AppInfo;
 import org.apache.zookeeper.KeeperException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/0b852cbc/eagle-jpm/eagle-jpm-spark-running/src/main/java/org/apache/eagle/jpm/spark/running/storm/SparkRunningJobParseBolt.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-spark-running/src/main/java/org/apache/eagle/jpm/spark/running/storm/SparkRunningJobParseBolt.java b/eagle-jpm/eagle-jpm-spark-running/src/main/java/org/apache/eagle/jpm/spark/running/storm/SparkRunningJobParseBolt.java
index 21a6ef2..c41804b 100644
--- a/eagle-jpm/eagle-jpm-spark-running/src/main/java/org/apache/eagle/jpm/spark/running/storm/SparkRunningJobParseBolt.java
+++ b/eagle-jpm/eagle-jpm-spark-running/src/main/java/org/apache/eagle/jpm/spark/running/storm/SparkRunningJobParseBolt.java
@@ -23,9 +23,9 @@ import org.apache.eagle.jpm.spark.running.entities.SparkAppEntity;
 import org.apache.eagle.jpm.spark.running.parser.SparkApplicationParser;
 import org.apache.eagle.jpm.spark.running.recover.SparkRunningJobManager;
 import org.apache.eagle.jpm.util.Constants;
-import org.apache.eagle.jpm.util.resourceFetch.RMResourceFetcher;
-import org.apache.eagle.jpm.util.resourceFetch.ResourceFetcher;
-import org.apache.eagle.jpm.util.resourceFetch.model.AppInfo;
+import org.apache.eagle.jpm.util.resourcefetch.RMResourceFetcher;
+import org.apache.eagle.jpm.util.resourcefetch.ResourceFetcher;
+import org.apache.eagle.jpm.util.resourcefetch.model.AppInfo;
 
 import backtype.storm.task.OutputCollector;
 import backtype.storm.task.TopologyContext;

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/0b852cbc/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/Constants.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/Constants.java b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/Constants.java
index 07850f9..1a6c09c 100644
--- a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/Constants.java
+++ b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/Constants.java
@@ -21,19 +21,19 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 public class Constants {
-    private final static Logger LOG = LoggerFactory.getLogger(Constants.class);
+    private static final Logger LOG = LoggerFactory.getLogger(Constants.class);
 
     //SPARK
-    public final static String SPARK_APP_SERVICE_ENDPOINT_NAME = "SparkAppService";
-    public final static String SPARK_JOB_SERVICE_ENDPOINT_NAME = "SparkJobService";
-    public final static String SPARK_STAGE_SERVICE_ENDPOINT_NAME = "SparkStageService";
-    public final static String SPARK_TASK_SERVICE_ENDPOINT_NAME = "SparkTaskService";
-    public final static String SPARK_EXECUTOR_SERVICE_ENDPOINT_NAME = "SparkExecutorService";
-    public final static String RUNNING_SPARK_APP_SERVICE_ENDPOINT_NAME = "RunningSparkAppService";
-    public final static String RUNNING_SPARK_JOB_SERVICE_ENDPOINT_NAME = "RunningSparkJobService";
-    public final static String RUNNING_SPARK_STAGE_SERVICE_ENDPOINT_NAME = "RunningSparkStageService";
-    public final static String RUNNING_SPARK_TASK_SERVICE_ENDPOINT_NAME = "RunningSparkTaskService";
-    public final static String RUNNING_SPARK_EXECUTOR_SERVICE_ENDPOINT_NAME = "RunningSparkExecutorService";
+    public static final String SPARK_APP_SERVICE_ENDPOINT_NAME = "SparkAppService";
+    public static final String SPARK_JOB_SERVICE_ENDPOINT_NAME = "SparkJobService";
+    public static final String SPARK_STAGE_SERVICE_ENDPOINT_NAME = "SparkStageService";
+    public static final String SPARK_TASK_SERVICE_ENDPOINT_NAME = "SparkTaskService";
+    public static final String SPARK_EXECUTOR_SERVICE_ENDPOINT_NAME = "SparkExecutorService";
+    public static final String RUNNING_SPARK_APP_SERVICE_ENDPOINT_NAME = "RunningSparkAppService";
+    public static final String RUNNING_SPARK_JOB_SERVICE_ENDPOINT_NAME = "RunningSparkJobService";
+    public static final String RUNNING_SPARK_STAGE_SERVICE_ENDPOINT_NAME = "RunningSparkStageService";
+    public static final String RUNNING_SPARK_TASK_SERVICE_ENDPOINT_NAME = "RunningSparkTaskService";
+    public static final String RUNNING_SPARK_EXECUTOR_SERVICE_ENDPOINT_NAME = "RunningSparkExecutorService";
     public static final String APPLICATION_PREFIX = "application";
     public static final String JOB_PREFIX = "job";
     public static final String V2_APPS_URL = "ws/v1/cluster/apps";
@@ -53,7 +53,7 @@ public class Constants {
     public static final String SPARK_YARN_DRIVER_MEMORY_OVERHEAD_KEY = "spark.yarn.driver.memoryOverhead";
     public static final String SPARK_YARN_am_MEMORY_OVERHEAD_KEY = "spark.yarn.am.memoryOverhead";
 
-    public static final String SPARK_APPS_URL ="api/v1/applications";
+    public static final String SPARK_APPS_URL = "api/v1/applications";
     public static final String SPARK_EXECUTORS_URL = "executors";
     public static final String SPARK_JOBS_URL = "jobs";
     public static final String SPARK_STAGES_URL = "stages";
@@ -68,21 +68,27 @@ public class Constants {
     public enum CompressionType {
         GZIP, NONE
     }
+
     public enum JobState {
         NEW, INITED, RUNNING, SUCCEEDED, FAILED, KILL_WAIT, KILLED, ERROR, FINISHED, ALL
     }
+
     public enum TaskState {
         NEW, SCHEDULED, RUNNING, SUCCEEDED, FAILED, KILL_WAIT, KILLED
     }
+
     public enum StageState {
         ACTIVE, COMPLETE, PENDING
     }
+
     public enum AppState {
         NEW, NEW_SAVING, SUBMITTED, ACCEPTED, RUNNING, FINISHED, FAILED, KILLED
     }
+
     public enum AppStatus {
         UNDEFINED, SUCCEEDED, FAILED, KILLED
     }
+
     public enum ResourceType {
         COMPLETE_SPARK_JOB, SPARK_JOB_DETAIL, RUNNING_SPARK_JOB, RUNNING_MR_JOB, CLUSTER_INFO, JOB_CONFIGURATION,
         COMPLETE_MR_JOB
@@ -117,7 +123,7 @@ public class Constants {
     public static final String HIVE_QUERY_STRING = "hive.query.string";
 
     /**
-     * MR task types
+     * MR task types.
      */
     public enum TaskType {
         SETUP, MAP, REDUCE, CLEANUP
@@ -128,9 +134,10 @@ public class Constants {
         NOTAVALIABLE("N/A")
         ;
         private String value;
-        JobType(String value){
+        JobType(String value) {
             this.value = value;
         }
+
         @Override
         public String toString() {
             return this.value;



[19/52] [abbrv] incubator-eagle git commit: EAGLE-493 Create alert metadata based on application stream sink configuration

Posted by yo...@apache.org.
EAGLE-493 Create alert metadata based on application stream sink configuration

Author: @yonzhang <yo...@apache.org>

Closes: #389


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

Branch: refs/heads/master
Commit: c5d05abd1d5f71d78dbe9a577a3975dc5b964d51
Parents: 7f37267
Author: yonzhang <yo...@gmail.com>
Authored: Thu Aug 25 14:13:45 2016 -0700
Committer: yonzhang <yo...@gmail.com>
Committed: Thu Aug 25 14:13:45 2016 -0700

----------------------------------------------------------------------
 .../src/main/resources/application.conf         |  2 +-
 eagle-core/eagle-app/eagle-app-base/pom.xml     |  5 ++
 .../eagle/app/service/ApplicationContext.java   | 49 ++++++++++++++++++--
 .../impl/ApplicationManagementServiceImpl.java  | 14 ++++--
 .../apache/eagle/app/sink/KafkaStreamSink.java  |  2 +-
 ...ecurity.auditlog.HdfsAuditLogAppProvider.xml |  6 +--
 6 files changed, 63 insertions(+), 15 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/c5d05abd/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/main/resources/application.conf
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/main/resources/application.conf b/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/main/resources/application.conf
index 60595b1..7030e45 100644
--- a/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/main/resources/application.conf
+++ b/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/main/resources/application.conf
@@ -48,7 +48,7 @@
   "metadataService": {
     "context" : "/rest",
     "host" : "localhost",
-    "port" : 8080
+    "port" : 9090
   },
   "coordinatorService": {
     "host": "localhost",

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/c5d05abd/eagle-core/eagle-app/eagle-app-base/pom.xml
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-app/eagle-app-base/pom.xml b/eagle-core/eagle-app/eagle-app-base/pom.xml
index d89b78c..12b056c 100644
--- a/eagle-core/eagle-app/eagle-app-base/pom.xml
+++ b/eagle-core/eagle-app/eagle-app-base/pom.xml
@@ -78,6 +78,11 @@
             </exclusions>
         </dependency>
         <dependency>
+            <groupId>org.apache.eagle</groupId>
+            <artifactId>alert-engine</artifactId>
+            <version>${project.version}</version>
+        </dependency>
+        <dependency>
             <groupId>com.sun.jersey</groupId>
             <artifactId>jersey-server</artifactId>
             <version>${jersey.version}</version>

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/c5d05abd/eagle-core/eagle-app/eagle-app-base/src/main/java/org/apache/eagle/app/service/ApplicationContext.java
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-app/eagle-app-base/src/main/java/org/apache/eagle/app/service/ApplicationContext.java b/eagle-core/eagle-app/eagle-app-base/src/main/java/org/apache/eagle/app/service/ApplicationContext.java
index 91d33ca..52eb628 100644
--- a/eagle-core/eagle-app/eagle-app-base/src/main/java/org/apache/eagle/app/service/ApplicationContext.java
+++ b/eagle-core/eagle-app/eagle-app-base/src/main/java/org/apache/eagle/app/service/ApplicationContext.java
@@ -19,18 +19,23 @@ package org.apache.eagle.app.service;
 import com.google.common.base.Preconditions;
 import com.typesafe.config.Config;
 import com.typesafe.config.ConfigFactory;
+import org.apache.eagle.alert.coordination.model.Kafka2TupleMetadata;
+import org.apache.eagle.alert.coordination.model.Tuple2StreamMetadata;
+import org.apache.eagle.alert.engine.coordinator.StreamDefinition;
+import org.apache.eagle.alert.engine.scheme.JsonScheme;
+import org.apache.eagle.alert.engine.scheme.JsonStringStreamNameSelector;
+import org.apache.eagle.alert.metadata.IMetadataDao;
 import org.apache.eagle.app.Application;
 import org.apache.eagle.app.ApplicationLifecycle;
 import org.apache.eagle.app.environment.ExecutionRuntime;
 import org.apache.eagle.app.environment.ExecutionRuntimeManager;
+import org.apache.eagle.app.sink.KafkaStreamSinkConfig;
 import org.apache.eagle.metadata.model.ApplicationEntity;
 import org.apache.eagle.metadata.model.StreamDesc;
 import org.apache.eagle.metadata.model.StreamSinkConfig;
 
 import java.io.Serializable;
-import java.util.Collections;
-import java.util.List;
-import java.util.Map;
+import java.util.*;
 import java.util.stream.Collectors;
 
 /**
@@ -47,12 +52,13 @@ public class ApplicationContext implements Serializable, ApplicationLifecycle {
     private final Application application;
     private final ExecutionRuntime runtime;
     private final ApplicationEntity metadata;
+    private final IMetadataDao alertMetadataService;
 
     /**
      * @param metadata ApplicationEntity
      * @param application Application
      */
-    public ApplicationContext(Application application, ApplicationEntity metadata, Config envConfig){
+    public ApplicationContext(Application application, ApplicationEntity metadata, Config envConfig, IMetadataDao alertMetadataService){
         Preconditions.checkNotNull(application,"Application is null");
         Preconditions.checkNotNull(metadata,"ApplicationEntity is null");
         this.application = application;
@@ -69,6 +75,7 @@ public class ApplicationContext implements Serializable, ApplicationLifecycle {
         executionConfig.put("appId", metadata.getAppId());
         executionConfig.put("jarPath", metadata.getJarPath());
         this.config = ConfigFactory.parseMap(executionConfig).withFallback(envConfig);
+        this.alertMetadataService = alertMetadataService;
     }
 
     @Override
@@ -83,12 +90,44 @@ public class ApplicationContext implements Serializable, ApplicationLifecycle {
                 return streamDesc;
             })).collect(Collectors.toList());
             metadata.setStreams(streamDescCollection);
+
+            // iterate each stream descriptor and create alert datasource for each
+            for(StreamDesc streamDesc : streamDescCollection) {
+                // only take care of Kafka sink
+                if (streamDesc.getSink() instanceof KafkaStreamSinkConfig) {
+                    KafkaStreamSinkConfig kafkaCfg = (KafkaStreamSinkConfig) streamDesc.getSink();
+                    Kafka2TupleMetadata datasource = new Kafka2TupleMetadata();
+                    datasource.setType("KAFKA");
+                    datasource.setName(metadata.getAppId());
+                    datasource.setTopic(kafkaCfg.getTopicId());
+                    datasource.setSchemeCls(JsonScheme.class.getCanonicalName());
+                    Tuple2StreamMetadata tuple2Stream = new Tuple2StreamMetadata();
+                    Set<String> activeStreamNames = new HashSet<>();
+                    activeStreamNames.add(streamDesc.getSchema().getStreamId());
+                    tuple2Stream.setActiveStreamNames(activeStreamNames);
+                    tuple2Stream.setTimestampColumn("timestamp");
+                    tuple2Stream.setStreamNameSelectorCls(JsonStringStreamNameSelector.class.getCanonicalName());
+                    datasource.setCodec(tuple2Stream);
+                    alertMetadataService.addDataSource(datasource);
+
+                    StreamDefinition sd = streamDesc.getSchema();
+                    sd.setDataSource(metadata.getAppId());
+                    alertMetadataService.createStream(streamDesc.getSchema());
+                }
+            }
         }
     }
 
     @Override
     public void onUninstall() {
-        //
+        // we should remove alert data source and stream definition while we do uninstall
+        if(metadata.getStreams() == null)
+            return;
+        // iterate each stream descriptor and create alert datasource for each
+        for(StreamDesc streamDesc : metadata.getStreams()) {
+            alertMetadataService.removeDataSource(metadata.getAppId());
+            alertMetadataService.removeStream(streamDesc.getStreamId());
+        }
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/c5d05abd/eagle-core/eagle-app/eagle-app-base/src/main/java/org/apache/eagle/app/service/impl/ApplicationManagementServiceImpl.java
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-app/eagle-app-base/src/main/java/org/apache/eagle/app/service/impl/ApplicationManagementServiceImpl.java b/eagle-core/eagle-app/eagle-app-base/src/main/java/org/apache/eagle/app/service/impl/ApplicationManagementServiceImpl.java
index 314b0fb..c355a10 100644
--- a/eagle-core/eagle-app/eagle-app-base/src/main/java/org/apache/eagle/app/service/impl/ApplicationManagementServiceImpl.java
+++ b/eagle-core/eagle-app/eagle-app-base/src/main/java/org/apache/eagle/app/service/impl/ApplicationManagementServiceImpl.java
@@ -21,6 +21,7 @@ import com.google.inject.Inject;
 import com.google.inject.Singleton;
 import com.typesafe.config.Config;
 import com.typesafe.config.ConfigFactory;
+import org.apache.eagle.alert.metadata.IMetadataDao;
 import org.apache.eagle.app.service.ApplicationContext;
 import org.apache.eagle.app.service.ApplicationOperations;
 import org.apache.eagle.app.service.ApplicationManagementService;
@@ -45,6 +46,7 @@ public class ApplicationManagementServiceImpl implements ApplicationManagementSe
     private final SiteEntityService siteEntityService;
     private final ApplicationProviderService applicationProviderService;
     private final ApplicationEntityService applicationEntityService;
+    private final IMetadataDao alertMetadataService;
     private final Config config;
     private final static Logger LOGGER = LoggerFactory.getLogger(ApplicationManagementServiceImpl.class);
 
@@ -53,11 +55,13 @@ public class ApplicationManagementServiceImpl implements ApplicationManagementSe
             Config config,
             SiteEntityService siteEntityService,
             ApplicationProviderService applicationProviderService,
-            ApplicationEntityService applicationEntityService){
+            ApplicationEntityService applicationEntityService,
+            IMetadataDao alertMetadataService){
         this.config = config;
         this.siteEntityService = siteEntityService;
         this.applicationProviderService = applicationProviderService;
         this.applicationEntityService = applicationEntityService;
+        this.alertMetadataService = alertMetadataService;
     }
 
     @Override
@@ -97,7 +101,7 @@ public class ApplicationManagementServiceImpl implements ApplicationManagementSe
         applicationEntity.setConfiguration(appConfig);
         ApplicationContext applicationContext = new ApplicationContext(
                 applicationProviderService.getApplicationProviderByType(applicationEntity.getDescriptor().getType()).getApplication(),
-                applicationEntity,config);
+                applicationEntity,config, alertMetadataService);
         applicationContext.onInstall();
         return applicationEntityService.create(applicationEntity);
     }
@@ -107,7 +111,7 @@ public class ApplicationManagementServiceImpl implements ApplicationManagementSe
         ApplicationEntity applicationEntity = applicationEntityService.getByUUIDOrAppId(operation.getUuid(),operation.getAppId());
         ApplicationContext applicationContext = new ApplicationContext(
                 applicationProviderService.getApplicationProviderByType(applicationEntity.getDescriptor().getType()).getApplication(),
-                applicationEntity,config);
+                applicationEntity,config, alertMetadataService);
         // TODO: Check status, skip stop if already STOPPED
         try {
             applicationContext.onStop();
@@ -123,7 +127,7 @@ public class ApplicationManagementServiceImpl implements ApplicationManagementSe
         ApplicationEntity applicationEntity = applicationEntityService.getByUUIDOrAppId(operation.getUuid(),operation.getAppId());
         ApplicationContext applicationContext = new ApplicationContext(
                 applicationProviderService.getApplicationProviderByType(applicationEntity.getDescriptor().getType()).getApplication(),
-                applicationEntity,config);
+                applicationEntity,config, alertMetadataService);
         applicationContext.onStart();
         return applicationEntity;
     }
@@ -133,7 +137,7 @@ public class ApplicationManagementServiceImpl implements ApplicationManagementSe
         ApplicationEntity applicationEntity = applicationEntityService.getByUUIDOrAppId(operation.getUuid(),operation.getAppId());
         ApplicationContext applicationContext = new ApplicationContext(
                 applicationProviderService.getApplicationProviderByType(applicationEntity.getDescriptor().getType()).getApplication(),
-                applicationEntity,config);
+                applicationEntity,config, alertMetadataService);
         applicationContext.onStop();
         return applicationEntity;
     }

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/c5d05abd/eagle-core/eagle-app/eagle-app-base/src/main/java/org/apache/eagle/app/sink/KafkaStreamSink.java
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-app/eagle-app-base/src/main/java/org/apache/eagle/app/sink/KafkaStreamSink.java b/eagle-core/eagle-app/eagle-app-base/src/main/java/org/apache/eagle/app/sink/KafkaStreamSink.java
index 5c33c94..27848d9 100644
--- a/eagle-core/eagle-app/eagle-app-base/src/main/java/org/apache/eagle/app/sink/KafkaStreamSink.java
+++ b/eagle-core/eagle-app/eagle-app-base/src/main/java/org/apache/eagle/app/sink/KafkaStreamSink.java
@@ -57,7 +57,7 @@ public class KafkaStreamSink extends StormStreamSink<KafkaStreamSinkConfig> {
     protected void execute(Object key, Map event,BasicOutputCollector collector) {
         try {
             String output = new ObjectMapper().writeValueAsString(event);
-            producer.send(new KeyedMessage(this.topicId, event.get("user"), output));
+            producer.send(new KeyedMessage(this.topicId, key, output));
         }catch(Exception ex){
             LOG.error(ex.getMessage(), ex);
             collector.reportError(ex);

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/c5d05abd/eagle-security/eagle-security-hdfs-auditlog/src/main/resources/META-INF/providers/org.apache.eagle.security.auditlog.HdfsAuditLogAppProvider.xml
----------------------------------------------------------------------
diff --git a/eagle-security/eagle-security-hdfs-auditlog/src/main/resources/META-INF/providers/org.apache.eagle.security.auditlog.HdfsAuditLogAppProvider.xml b/eagle-security/eagle-security-hdfs-auditlog/src/main/resources/META-INF/providers/org.apache.eagle.security.auditlog.HdfsAuditLogAppProvider.xml
index 2419747..2a8ff0f 100644
--- a/eagle-security/eagle-security-hdfs-auditlog/src/main/resources/META-INF/providers/org.apache.eagle.security.auditlog.HdfsAuditLogAppProvider.xml
+++ b/eagle-security/eagle-security-hdfs-auditlog/src/main/resources/META-INF/providers/org.apache.eagle.security.auditlog.HdfsAuditLogAppProvider.xml
@@ -117,7 +117,7 @@
         <property>
             <name>dataSinkConfig.topic</name>
             <displayName>dataSinkConfig.topic</displayName>
-            <value>hdfs_audit_log_parsed</value>
+            <value>hdfs_audit_log_enriched</value>
             <description>topic for kafka data sink</description>
         </property>
         <property>
@@ -149,8 +149,8 @@
     </configuration>
     <streams>
         <stream>
-            <streamId>hdfs_audit_log_stream</streamId>
-            <description>Hdfs Audit Log Stream</description>
+            <streamId>hdfs_audit_log_enriched_stream</streamId>
+            <description>Hdfs Audit Log Enriched Stream</description>
             <validate>true</validate>
             <timeseries>true</timeseries>
             <columns>


[17/52] [abbrv] incubator-eagle git commit: [EAGLE-500] fix process-stopping issue of mongodb unit test case

Posted by yo...@apache.org.
[EAGLE-500] fix process-stopping issue of mongodb unit test case

https://issues.apache.org/jira/browse/EAGLE-500

Author: anyway1021 <mw...@apache.org>

Closes #387 from anyway1021/EAGLE-500.


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

Branch: refs/heads/master
Commit: a846c401c0b9fad912578cc17b0b0e31d52e983a
Parents: b5fd782
Author: anyway1021 <mw...@apache.org>
Authored: Thu Aug 25 17:44:37 2016 +0800
Committer: Hao Chen <ha...@apache.org>
Committed: Thu Aug 25 17:44:37 2016 +0800

----------------------------------------------------------------------
 .../service/alert/resource/impl/MongoImplTest.java | 17 ++++++++++++++++-
 1 file changed, 16 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/a846c401/eagle-core/eagle-alert-parent/eagle-alert/alert-metadata-parent/alert-metadata/src/test/java/org/apache/eagle/service/alert/resource/impl/MongoImplTest.java
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-alert-parent/eagle-alert/alert-metadata-parent/alert-metadata/src/test/java/org/apache/eagle/service/alert/resource/impl/MongoImplTest.java b/eagle-core/eagle-alert-parent/eagle-alert/alert-metadata-parent/alert-metadata/src/test/java/org/apache/eagle/service/alert/resource/impl/MongoImplTest.java
index a48ee7d..95f2682 100644
--- a/eagle-core/eagle-alert-parent/eagle-alert/alert-metadata-parent/alert-metadata/src/test/java/org/apache/eagle/service/alert/resource/impl/MongoImplTest.java
+++ b/eagle-core/eagle-alert-parent/eagle-alert/alert-metadata-parent/alert-metadata/src/test/java/org/apache/eagle/service/alert/resource/impl/MongoImplTest.java
@@ -25,6 +25,7 @@ import de.flapdoodle.embed.mongo.config.MongodConfigBuilder;
 import de.flapdoodle.embed.mongo.config.Net;
 import de.flapdoodle.embed.mongo.distribution.Version;
 import de.flapdoodle.embed.process.runtime.Network;
+import org.apache.commons.lang3.exception.ExceptionUtils;
 import org.apache.eagle.alert.coordination.model.*;
 import org.apache.eagle.alert.coordination.model.internal.MonitoredStream;
 import org.apache.eagle.alert.coordination.model.internal.PolicyAssignment;
@@ -79,7 +80,21 @@ public class MongoImplTest {
     @AfterClass
     public static void teardown() {
         if (mongod != null) {
-            mongod.stop();
+            try {
+                mongod.stop();
+            }
+            catch (IllegalStateException e) {
+                // catch this exception for the unstable stopping mongodb
+                // reason: the exception is usually thrown out with below message format when stop() returns null value,
+                //         but actually this should have been captured in ProcessControl.stopOrDestroyProcess() by destroying
+                //         the process ultimately
+                if (e.getMessage() != null && e.getMessage().matches("^Couldn't kill.*process!.*")) {
+                    // if matches, do nothing, just ignore the exception
+                }
+                else {
+                    LOG.warn(String.format("Ignored error for stopping mongod process, see stack trace: %s", ExceptionUtils.getStackTrace(e)));
+                }
+            }
             mongodExe.stop();
         }
     }


[26/52] [abbrv] incubator-eagle git commit: [EAGLE-512] add job count(total&fail) for mr history feeder

Posted by yo...@apache.org.
[EAGLE-512] add job count(total&fail) for mr history feeder

Author: wujinhu <wu...@126.com>

Closes #406 from wujinhu/EAGLE-512.


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

Branch: refs/heads/master
Commit: 4aa5b45260e9bf4ea029a84dd04ed87eae4dc38b
Parents: da7f552
Author: wujinhu <wu...@126.com>
Authored: Tue Aug 30 20:23:38 2016 +0800
Committer: Qingwen Zhao <qi...@gmail.com>
Committed: Tue Aug 30 20:23:38 2016 +0800

----------------------------------------------------------------------
 .../mr/historyentity/JPAEntityRepository.java   |   1 +
 .../jpm/mr/historyentity/JobCountEntity.java    |  56 ++++++
 .../history/crawler/JHFCrawlerDriverImpl.java   |  51 ++++-
 .../JobEntityCreationEagleServiceListener.java  |  20 ++
 .../jpm/mr/history/storm/JobHistorySpout.java   |   4 +-
 .../mr/history/zkres/JobHistoryZKStateLCM.java  |   6 +
 .../history/zkres/JobHistoryZKStateManager.java |  46 +++++
 .../org/apache/eagle/jpm/util/Constants.java    |   1 +
 .../src/main/resources/JobCounter.conf          | 187 +++++++++++++++++++
 9 files changed, 370 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/4aa5b452/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/mr/historyentity/JPAEntityRepository.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/mr/historyentity/JPAEntityRepository.java b/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/mr/historyentity/JPAEntityRepository.java
index 49095ed..2303869 100644
--- a/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/mr/historyentity/JPAEntityRepository.java
+++ b/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/mr/historyentity/JPAEntityRepository.java
@@ -36,5 +36,6 @@ public class JPAEntityRepository extends EntityRepository {
         entitySet.add(TaskFailureCountAPIEntity.class);
         entitySet.add(TaskAttemptCounterAPIEntity.class);
         entitySet.add(JobProcessTimeStampEntity.class);
+        entitySet.add(JobCountEntity.class);
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/4aa5b452/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/mr/historyentity/JobCountEntity.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/mr/historyentity/JobCountEntity.java b/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/mr/historyentity/JobCountEntity.java
new file mode 100644
index 0000000..f18188a
--- /dev/null
+++ b/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/mr/historyentity/JobCountEntity.java
@@ -0,0 +1,56 @@
+/*
+ * 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.eagle.jpm.mr.historyentity;
+
+import org.apache.eagle.jpm.util.Constants;
+import org.apache.eagle.log.base.taggedlog.TaggedLogAPIEntity;
+import org.apache.eagle.log.entity.meta.*;
+import org.codehaus.jackson.map.annotate.JsonSerialize;
+
+@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
+@Table("eaglejcount")
+@ColumnFamily("f")
+@Prefix("jcount")
+@Service(Constants.JPA_JOB_COUNT_SERVICE_NAME)
+@TimeSeries(true)
+@Partition({"site"})
+public class JobCountEntity extends TaggedLogAPIEntity {
+    @Column("a")
+    private int total;
+    @Column("b")
+    private int fail;
+
+    public int getTotal() {
+        return total;
+    }
+
+    public void setTotal(int total) {
+        this.total = total;
+        valueChanged("total");
+    }
+
+    public int getFail() {
+        return fail;
+    }
+
+    public void setFail(int fail) {
+        this.fail = fail;
+        valueChanged("fail");
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/4aa5b452/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/crawler/JHFCrawlerDriverImpl.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/crawler/JHFCrawlerDriverImpl.java b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/crawler/JHFCrawlerDriverImpl.java
index 278599d..e16ecce 100644
--- a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/crawler/JHFCrawlerDriverImpl.java
+++ b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/crawler/JHFCrawlerDriverImpl.java
@@ -19,9 +19,13 @@
 package org.apache.eagle.jpm.mr.history.crawler;
 
 import org.apache.eagle.jpm.mr.history.MRHistoryJobConfig;
+import org.apache.eagle.jpm.mr.history.parser.EagleJobStatus;
 import org.apache.eagle.jpm.mr.history.zkres.JobHistoryZKStateLCM;
+import org.apache.eagle.jpm.mr.historyentity.JobCountEntity;
 import org.apache.eagle.jpm.util.JobIdFilter;
 import org.apache.commons.lang3.tuple.Pair;
+import org.apache.eagle.service.client.IEagleServiceClient;
+import org.apache.eagle.service.client.impl.EagleServiceClientImpl;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import java.util.*;
@@ -59,11 +63,16 @@ public class JHFCrawlerDriverImpl implements JHFCrawlerDriver {
     private JobIdFilter jobFilter;
     private int partitionId;
     private TimeZone timeZone;
+    private MRHistoryJobConfig.EagleServiceConfig eagleServiceConfig;
+    private MRHistoryJobConfig.JobExtractorConfig jobExtractorConfig;
 
-    public JHFCrawlerDriverImpl(MRHistoryJobConfig.JobHistoryEndpointConfig jobHistoryConfig,
+    public JHFCrawlerDriverImpl(MRHistoryJobConfig.EagleServiceConfig eagleServiceConfig,
+                                MRHistoryJobConfig.JobExtractorConfig jobExtractorConfig,
                                 MRHistoryJobConfig.ControlConfig controlConfig, JHFInputStreamCallback reader,
                                 JobHistoryZKStateLCM zkStateLCM,
                                 JobHistoryLCM historyLCM, JobIdFilter jobFilter, int partitionId) throws Exception {
+        this.eagleServiceConfig = eagleServiceConfig;
+        this.jobExtractorConfig = jobExtractorConfig;
         this.zeroBasedMonth = controlConfig.zeroBasedMonth;
         this.dryRun = controlConfig.dryRun;
         if (this.dryRun)  {
@@ -185,6 +194,7 @@ public class JHFCrawlerDriverImpl implements JHFCrawlerDriver {
                 jobHistoryFile);
         processedJobFileNames.add(jobHistoryFile);
 
+        flushJobCount();
         Long modifiedTime = item.getLeft();
         return modifiedTime;
     }
@@ -230,7 +240,46 @@ public class JHFCrawlerDriverImpl implements JHFCrawlerDriver {
         }
     }
 
+    private void flushJobCount() throws Exception {
+        List<Pair<String, String>> jobs = zkStateLcm.getProcessedJobs(
+            String.format(FORMAT_JOB_PROCESS_DATE, this.processDate.year, this.processDate.month + 1, this.processDate.day)
+        );
+        JobCountEntity entity = new JobCountEntity();
+        entity.setTotal(jobs.size());
+        entity.setFail(0);
+        jobs.stream().filter(job -> !job.getRight().equals(EagleJobStatus.SUCCESS.toString())).forEach(
+            job -> entity.setFail(1 + entity.getFail())
+        );
+
+        IEagleServiceClient client = new EagleServiceClientImpl(
+            eagleServiceConfig.eagleServiceHost,
+            eagleServiceConfig.eagleServicePort,
+            eagleServiceConfig.username,
+            eagleServiceConfig.password);
+
+
+        GregorianCalendar cal = new GregorianCalendar(this.processDate.year, this.processDate.month, this.processDate.day, 0, 0, 0);
+        cal.setTimeZone(timeZone);
+        entity.setTimestamp(cal.getTimeInMillis());
+        @SuppressWarnings("serial")
+        Map<String, String> baseTags = new HashMap<String, String>() {
+            {
+                put("site", jobExtractorConfig.site);
+            }
+        };
+        entity.setTags(baseTags);
+        List<JobCountEntity> entities = new ArrayList<>();
+        entities.add(entity);
+
+        LOG.info("start flushing entities of total number " + entities.size());
+        client.create(entities);
+        LOG.info("finish flushing entities of total number " + entities.size());
+        client.getJerseyClient().destroy();
+        client.close();
+    }
+
     private void advanceOneDay() throws Exception {
+        //flushJobCount();
         GregorianCalendar cal = new GregorianCalendar(timeZone);
         cal.set(this.processDate.year, this.processDate.month, this.processDate.day, 0, 0, 0);
         cal.add(Calendar.DATE, 1);

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/4aa5b452/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JobEntityCreationEagleServiceListener.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JobEntityCreationEagleServiceListener.java b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JobEntityCreationEagleServiceListener.java
index e7b8a6b..a681aca 100644
--- a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JobEntityCreationEagleServiceListener.java
+++ b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JobEntityCreationEagleServiceListener.java
@@ -19,7 +19,9 @@
 package org.apache.eagle.jpm.mr.history.parser;
 
 import org.apache.eagle.jpm.mr.history.MRHistoryJobConfig;
+import org.apache.eagle.jpm.mr.history.zkres.JobHistoryZKStateManager;
 import org.apache.eagle.jpm.mr.historyentity.*;
+import org.apache.eagle.jpm.util.MRJobTagName;
 import org.apache.eagle.log.entity.GenericServiceAPIResponseEntity;
 import org.apache.eagle.service.client.IEagleServiceClient;
 import org.apache.eagle.service.client.impl.EagleServiceClientImpl;
@@ -27,7 +29,9 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import java.util.ArrayList;
+import java.util.GregorianCalendar;
 import java.util.List;
+import java.util.TimeZone;
 
 public class JobEntityCreationEagleServiceListener implements HistoryJobEntityCreationListener {
     private static final Logger logger = LoggerFactory.getLogger(JobEntityCreationEagleServiceListener.class);
@@ -39,6 +43,8 @@ public class JobEntityCreationEagleServiceListener implements HistoryJobEntityCr
     List<JobEventAPIEntity> jobEvents = new ArrayList<>();
     List<TaskExecutionAPIEntity> taskExecs = new ArrayList<>();
     List<TaskAttemptExecutionAPIEntity> taskAttemptExecs = new ArrayList<>();
+    private JobHistoryZKStateManager zkState;
+    private TimeZone timeZone;
 
     public JobEntityCreationEagleServiceListener(MRHistoryJobConfig configManager) {
         this(configManager, BATCH_SIZE);
@@ -50,6 +56,8 @@ public class JobEntityCreationEagleServiceListener implements HistoryJobEntityCr
             throw new IllegalArgumentException("batchSize must be greater than 0 when it is provided");
         }
         this.batchSize = batchSize;
+        zkState = new JobHistoryZKStateManager(configManager.getZkStateConfig());
+        timeZone = TimeZone.getTimeZone(configManager.getControlConfig().timeZone);
     }
 
     @Override
@@ -61,6 +69,15 @@ public class JobEntityCreationEagleServiceListener implements HistoryJobEntityCr
         }
     }
 
+    private String timeStamp2Date(long timeStamp) {
+        GregorianCalendar cal = new GregorianCalendar(timeZone);
+        cal.setTimeInMillis(timeStamp);
+        return  String.format("%4d%02d%02d",
+            cal.get(GregorianCalendar.YEAR),
+            cal.get(GregorianCalendar.MONTH) + 1,
+            cal.get(GregorianCalendar.DAY_OF_MONTH));
+    }
+
     /**
      * We need save network bandwidth as well.
      */
@@ -80,6 +97,9 @@ public class JobEntityCreationEagleServiceListener implements HistoryJobEntityCr
             JobBaseAPIEntity entity = list.get(i);
             if (entity instanceof JobExecutionAPIEntity) {
                 jobs.add((JobExecutionAPIEntity) entity);
+                this.zkState.updateProcessedJob(timeStamp2Date(entity.getTimestamp()),
+                    entity.getTags().get(MRJobTagName.JOB_ID.toString()),
+                    ((JobExecutionAPIEntity) entity).getCurrentState());
             } else if (entity instanceof JobEventAPIEntity) {
                 jobEvents.add((JobEventAPIEntity) entity);
             } else if (entity instanceof TaskExecutionAPIEntity) {

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/4aa5b452/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/storm/JobHistorySpout.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/storm/JobHistorySpout.java b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/storm/JobHistorySpout.java
index ca4a94f..402f93e 100644
--- a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/storm/JobHistorySpout.java
+++ b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/storm/JobHistorySpout.java
@@ -149,7 +149,9 @@ public class JobHistorySpout extends BaseRichSpout {
 
         try {
             jhfLCM = new JobHistoryDAOImpl(configManager.getJobHistoryEndpointConfig());
-            driver = new JHFCrawlerDriverImpl(configManager.getJobHistoryEndpointConfig(),
+            driver = new JHFCrawlerDriverImpl(
+                configManager.getEagleServiceConfig(),
+                configManager.getJobExtractorConfig(),
                 configManager.getControlConfig(),
                 callback,
                 zkState,

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/4aa5b452/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/zkres/JobHistoryZKStateLCM.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/zkres/JobHistoryZKStateLCM.java b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/zkres/JobHistoryZKStateLCM.java
index cbde88c..0adcffe 100644
--- a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/zkres/JobHistoryZKStateLCM.java
+++ b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/zkres/JobHistoryZKStateLCM.java
@@ -18,6 +18,8 @@
 
 package org.apache.eagle.jpm.mr.history.zkres;
 
+import org.apache.commons.lang3.tuple.Pair;
+
 import java.util.List;
 
 public interface JobHistoryZKStateLCM {
@@ -31,6 +33,8 @@ public interface JobHistoryZKStateLCM {
 
     void addProcessedJob(String date, String jobId);
 
+    void updateProcessedJob(String date, String jobId, String status);
+
     void truncateProcessedJob(String date);
 
     void truncateEverything();
@@ -38,4 +42,6 @@ public interface JobHistoryZKStateLCM {
     long readProcessedTimeStamp(int partitionId);
 
     void updateProcessedTimeStamp(int partitionId, long timeStamp);
+
+    List<Pair<String, String>> getProcessedJobs(String date);
 }

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/4aa5b452/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/zkres/JobHistoryZKStateManager.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/zkres/JobHistoryZKStateManager.java b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/zkres/JobHistoryZKStateManager.java
index feb896e..c61d05a 100644
--- a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/zkres/JobHistoryZKStateManager.java
+++ b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/zkres/JobHistoryZKStateManager.java
@@ -18,6 +18,7 @@
 
 package org.apache.eagle.jpm.mr.history.zkres;
 
+import org.apache.commons.lang3.tuple.Pair;
 import org.apache.eagle.jpm.mr.history.MRHistoryJobConfig.ZKStateConfig;
 
 import org.apache.curator.framework.CuratorFramework;
@@ -29,6 +30,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import java.text.SimpleDateFormat;
+import java.util.ArrayList;
 import java.util.Calendar;
 import java.util.List;
 
@@ -40,6 +42,7 @@ public class JobHistoryZKStateManager implements JobHistoryZKStateLCM {
     public static final String ZNODE_FORCE_START_FROM = "forceStartFrom";
     public static final String ZNODE_PARTITIONS = "partitions";
     public static final String ZNODE_JOBS = "jobs";
+    public static final String ZNODE_JOB_IDS = "jobIds";
     public static final String ZNODE_TIMESTAMPS = "timeStamps";
 
     public static final int BACKOFF_DAYS = 0;
@@ -268,6 +271,12 @@ public class JobHistoryZKStateManager implements JobHistoryZKStateLCM {
                 _curator.delete().deletingChildrenIfNeeded().forPath(path);
                 LOG.info("really truncated all data for day " + date);
             }
+
+            String jobIdPath = zkRoot + "/" + ZNODE_JOB_IDS + "/" + date;
+            if (_curator.checkExists().forPath(jobIdPath) != null) {
+                _curator.delete().deletingChildrenIfNeeded().forPath(jobIdPath);
+                LOG.info("really truncated all jobIds for day " + date);
+            }
         } catch (Exception e) {
             LOG.error("fail truncating processed jobs", e);
             throw new RuntimeException(e);
@@ -345,4 +354,41 @@ public class JobHistoryZKStateManager implements JobHistoryZKStateLCM {
             throw new RuntimeException(e);
         }
     }
+
+    @Override
+    public List<Pair<String, String>> getProcessedJobs(String date) {
+        List<Pair<String, String>> result = new ArrayList<>();
+        String path = zkRoot + "/" + ZNODE_JOB_IDS + "/" + date;
+        try {
+            if (_curator.checkExists().forPath(path) != null) {
+                List<String> jobs = _curator.getChildren().forPath(path);
+                for (String job : jobs) {
+                    String jobPath = path + "/" + job;
+                    String status = new String(_curator.getData().forPath(jobPath), "UTF-8");
+                    result.add(Pair.of(job, status));
+                }
+            }
+        } catch (Exception e) {
+            LOG.error("fail read processed jobs", e);
+            throw new RuntimeException(e);
+        }
+        return result;
+    }
+
+    @Override
+    public void updateProcessedJob(String date, String jobId, String status) {
+        String path = zkRoot + "/" + ZNODE_JOB_IDS + "/" + date + "/" + jobId;
+        try {
+            if (_curator.checkExists().forPath(path) == null) {
+                _curator.create()
+                    .creatingParentsIfNeeded()
+                    .withMode(CreateMode.PERSISTENT)
+                    .forPath(path);
+            }
+            _curator.setData().forPath(path, status.getBytes("UTF-8"));
+        } catch (Exception e) {
+            LOG.error("fail adding processed jobs", e);
+            throw new RuntimeException(e);
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/4aa5b452/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/Constants.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/Constants.java b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/Constants.java
index 1a6c09c..7dce0a2 100644
--- a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/Constants.java
+++ b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/Constants.java
@@ -101,6 +101,7 @@ public class Constants {
     public static final String JPA_JOB_CONFIG_SERVICE_NAME = "JobConfigService";
     public static final String JPA_JOB_EVENT_SERVICE_NAME = "JobEventService";
     public static final String JPA_JOB_EXECUTION_SERVICE_NAME = "JobExecutionService";
+    public static final String JPA_JOB_COUNT_SERVICE_NAME = "JobCountService";
     public static final String JPA_RUNNING_JOB_EXECUTION_SERVICE_NAME = "RunningJobExecutionService";
     public static final String JPA_TASK_ATTEMPT_EXECUTION_SERVICE_NAME = "TaskAttemptExecutionService";
     public static final String JPA_TASK_FAILURE_COUNT_SERVICE_NAME = "TaskFailureCountService";

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/4aa5b452/eagle-jpm/eagle-jpm-util/src/main/resources/JobCounter.conf
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-util/src/main/resources/JobCounter.conf b/eagle-jpm/eagle-jpm-util/src/main/resources/JobCounter.conf
new file mode 100644
index 0000000..1524e61
--- /dev/null
+++ b/eagle-jpm/eagle-jpm-util/src/main/resources/JobCounter.conf
@@ -0,0 +1,187 @@
+# 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.
+
+#### Sample configuration:
+## counter.group0.name = groupname1
+## counter.group0.counter0.names = counterName1,counterName2,...
+## counter.group0.counter0.description = counter description...
+
+counter.group0.name = org.apache.hadoop.mapreduce.FileSystemCounter
+counter.group0.description = File System Counters
+counter.group0.counter0.names = FILE_BYTES_READ
+counter.group0.counter0.description = FILE: Number of bytes read
+counter.group0.counter1.names = FILE_BYTES_WRITTEN
+counter.group0.counter1.description = FILE: Number of bytes written
+counter.group0.counter2.names = FILE_READ_OPS
+counter.group0.counter2.description = FILE: Number of read operations
+counter.group0.counter3.names = FILE_LARGE_READ_OPS
+counter.group0.counter3.description = FILE: Number of large read operations
+counter.group0.counter4.names = FILE_WRITE_OPS
+counter.group0.counter4.description = FILE: Number of write operations
+counter.group0.counter5.names = HDFS_BYTES_READ
+counter.group0.counter5.description = HDFS: Number of bytes read
+counter.group0.counter6.names = HDFS_BYTES_WRITTEN
+counter.group0.counter6.description = HDFS: Number of bytes written
+counter.group0.counter7.names = HDFS_READ_OPS
+counter.group0.counter7.description = HDFS: Number of read operations
+counter.group0.counter8.names = HDFS_LARGE_READ_OPS
+counter.group0.counter8.description = HDFS: Number of large read operations
+counter.group0.counter9.names = HDFS_WRITE_OPS
+counter.group0.counter9.description = HDFS: Number of write operations
+
+counter.group1.name = org.apache.hadoop.mapreduce.TaskCounter
+counter.group1.description = Map-Reduce Framework
+counter.group1.counter0.names = MAP_INPUT_RECORDS
+counter.group1.counter0.description = Map input records
+counter.group1.counter1.names = MAP_OUTPUT_RECORDS
+counter.group1.counter1.description = Map output records
+counter.group1.counter2.names = SPLIT_RAW_BYTES
+counter.group1.counter2.description = Input split bytes
+counter.group1.counter3.names = SPILLED_RECORDS
+counter.group1.counter3.description = Spilled Records
+counter.group1.counter4.names = CPU_MILLISECONDS
+counter.group1.counter4.description = CPU time spent (ms)
+counter.group1.counter5.names = PHYSICAL_MEMORY_BYTES
+counter.group1.counter5.description = Physical memory (bytes) snapshot
+counter.group1.counter6.names = VIRTUAL_MEMORY_BYTES
+counter.group1.counter6.description = Virtual memory (bytes) snapshot
+counter.group1.counter7.names = COMMITTED_HEAP_BYTES
+counter.group1.counter7.description = Total committed heap usage (bytes)
+counter.group1.counter8.names = REDUCE_SHUFFLE_BYTES
+counter.group1.counter8.description = Reduce shuffle bytes (bytes)
+counter.group1.counter9.names = GC_TIME_MILLIS
+counter.group1.counter9.description = GC time milliseconds
+counter.group1.counter10.names = MAP_OUTPUT_BYTES
+counter.group1.counter10.description = map output bytes
+counter.group1.counter11.names = REDUCE_INPUT_RECORDS
+counter.group1.counter11.description = reduce input records
+counter.group1.counter12.names = COMBINE_INPUT_RECORDS
+counter.group1.counter12.description = combine input records
+counter.group1.counter13.names = COMBINE_OUTPUT_RECORDS
+counter.group1.counter13.description = combine output records
+counter.group1.counter14.names = REDUCE_INPUT_GROUPS
+counter.group1.counter14.description = reduce input groups
+counter.group1.counter15.names = REDUCE_OUTPUT_RECORDS
+counter.group1.counter15.description = reduce output records
+counter.group1.counter16.names = SHUFFLED_MAPS
+counter.group1.counter16.description = shuffled maps
+counter.group1.counter17.names = MAP_OUTPUT_MATERIALIZED_BYTES
+counter.group1.counter17.description = MAP_OUTPUT_MATERIALIZED_BYTES
+counter.group1.counter18.names = MERGED_MAP_OUTPUTS
+counter.group1.counter18.description = MERGED_MAP_OUTPUTS
+counter.group1.counter19.names = FAILED_SHUFFLE
+counter.group1.counter19.description = FAILED_SHUFFLE
+
+counter.group2.name = org.apache.hadoop.mapreduce.JobCounter
+counter.group2.description = Map-Reduce Job Counter
+counter.group2.counter0.names = MB_MILLIS_MAPS
+counter.group2.counter0.description = Total megabyte-seconds taken by all map tasks
+counter.group2.counter1.names = MB_MILLIS_REDUCES
+counter.group2.counter1.description = Total megabyte-seconds taken by all reduce tasks
+counter.group2.counter2.names = VCORES_MILLIS_MAPS
+counter.group2.counter2.description = Total vcore-seconds taken by all map tasks
+counter.group2.counter3.names = VCORES_MILLIS_REDUCES
+counter.group2.counter3.description = Total vcore-seconds taken by all reduce tasks
+counter.group2.counter4.names = OTHER_LOCAL_MAPS
+counter.group2.counter4.description = Total vcore-seconds taken by all reduce tasks
+counter.group2.counter5.names = DATA_LOCAL_MAPS
+counter.group2.counter5.description = Total vcore-seconds taken by all reduce tasks
+counter.group2.counter6.names = MILLIS_MAPS
+counter.group2.counter6.description = Total vcore-seconds taken by all reduce tasks
+counter.group2.counter7.names = MILLIS_REDUCES
+counter.group2.counter7.description = Total vcore-seconds taken by all reduce tasks
+counter.group2.counter8.names = TOTAL_LAUNCHED_MAPS
+counter.group2.counter8.description = Total vcore-seconds taken by all reduce tasks
+counter.group2.counter9.names = TOTAL_LAUNCHED_REDUCES
+counter.group2.counter9.description = Total vcore-seconds taken by all reduce tasks
+counter.group2.counter10.names = SLOTS_MILLIS_MAPS
+counter.group2.counter10.description = Total vcore-seconds taken by all reduce tasks
+counter.group2.counter11.names = SLOTS_MILLIS_REDUCES
+counter.group2.counter11.description = Total vcore-seconds taken by all reduce tasks
+counter.group2.counter12.names = RACK_LOCAL_MAPS
+counter.group2.counter12.description = Total vcore-seconds taken by all reduce tasks
+
+counter.group3.name = MapTaskAttemptCounter
+counter.group3.description = Reduce Task Attempt Counter Aggregation
+counter.group3.counter0.names = MAP_OUTPUT_MATERIALIZED_BYTES
+counter.group3.counter1.names = MAP_INPUT_RECORDS
+counter.group3.counter2.names = MERGED_MAP_OUTPUTS
+counter.group3.counter3.names = SPILLED_RECORDS
+counter.group3.counter4.names = MAP_OUTPUT_BYTES
+counter.group3.counter5.names = COMMITTED_HEAP_BYTES
+counter.group3.counter6.names = FAILED_SHUFFLE
+counter.group3.counter7.names = CPU_MILLISECONDS
+counter.group3.counter8.names = SPLIT_RAW_BYTES
+counter.group3.counter9.names = COMBINE_INPUT_RECORDS
+counter.group3.counter10.names = PHYSICAL_MEMORY_BYTES
+counter.group3.counter11.names = TASK_ATTEMPT_DURATION
+counter.group3.counter12.names = VIRTUAL_MEMORY_BYTES
+counter.group3.counter13.names = MAP_OUTPUT_RECORDS
+counter.group3.counter14.names = GC_TIME_MILLIS
+counter.group3.counter15.names = COMBINE_OUTPUT_RECORDS
+counter.group3.counter16.names = REDUCE_INPUT_GROUPS
+counter.group3.counter17.names = REDUCE_INPUT_RECORDS
+counter.group3.counter18.names = REDUCE_OUTPUT_RECORDS
+counter.group3.counter19.names = REDUCE_SHUFFLE_BYTES
+counter.group3.counter20.names = SHUFFLED_MAPS
+
+counter.group4.name = ReduceTaskAttemptCounter
+counter.group4.description = Reduce Task Attempt Counter Aggregation
+counter.group4.counter0.names = MAP_OUTPUT_MATERIALIZED_BYTES
+counter.group4.counter1.names = MAP_INPUT_RECORDS
+counter.group4.counter2.names = MERGED_MAP_OUTPUTS
+counter.group4.counter3.names = SPILLED_RECORDS
+counter.group4.counter4.names = MAP_OUTPUT_BYTES
+counter.group4.counter5.names = COMMITTED_HEAP_BYTES
+counter.group4.counter6.names = FAILED_SHUFFLE
+counter.group4.counter7.names = CPU_MILLISECONDS
+counter.group4.counter8.names = SPLIT_RAW_BYTES
+counter.group4.counter9.names = COMBINE_INPUT_RECORDS
+counter.group4.counter10.names = PHYSICAL_MEMORY_BYTES
+counter.group4.counter11.names = TASK_ATTEMPT_DURATION
+counter.group4.counter12.names = VIRTUAL_MEMORY_BYTES
+counter.group4.counter13.names = MAP_OUTPUT_RECORDS
+counter.group4.counter14.names = GC_TIME_MILLIS
+counter.group4.counter15.names = COMBINE_OUTPUT_RECORDS
+counter.group4.counter16.names = REDUCE_INPUT_GROUPS
+counter.group4.counter17.names = REDUCE_INPUT_RECORDS
+counter.group4.counter18.names = REDUCE_OUTPUT_RECORDS
+counter.group4.counter19.names = REDUCE_SHUFFLE_BYTES
+counter.group4.counter20.names = SHUFFLED_MAPS
+
+counter.group5.name = MapTaskAttemptFileSystemCounter
+counter.group5.description = Map Task Attempt File System Counter Aggregation
+counter.group5.counter0.names = FILE_READ_OPS
+counter.group5.counter1.names = FILE_WRITE_OPS
+counter.group5.counter2.names = FILE_BYTES_READ
+counter.group5.counter3.names = FILE_LARGE_READ_OPS
+counter.group5.counter4.names = HDFS_BYTES_READ
+counter.group5.counter5.names = FILE_BYTES_WRITTEN
+counter.group5.counter6.names = HDFS_LARGE_READ_OPS
+counter.group5.counter7.names = HDFS_BYTES_WRITTEN
+counter.group5.counter8.names = HDFS_READ_OPS
+
+counter.group6.name = ReduceTaskAttemptFileSystemCounter
+counter.group6.description = Reduce Task Attempt File System Counter Aggregation
+counter.group6.description = Map-Reduce Job Counter
+counter.group6.counter0.names = FILE_READ_OPS
+counter.group6.counter1.names = FILE_WRITE_OPS
+counter.group6.counter2.names = FILE_BYTES_READ
+counter.group6.counter3.names = FILE_LARGE_READ_OPS
+counter.group6.counter4.names = HDFS_BYTES_READ
+counter.group6.counter5.names = FILE_BYTES_WRITTEN
+counter.group6.counter6.names = HDFS_LARGE_READ_OPS
+counter.group6.counter7.names = HDFS_BYTES_WRITTEN
+counter.group6.counter8.names = HDFS_READ_OPS
\ No newline at end of file


[52/52] [abbrv] incubator-eagle git commit: [EAGLE-520] Fix and decouple co-processor from eagle aggreation query service

Posted by yo...@apache.org.
[EAGLE-520] Fix and decouple co-processor from eagle aggreation query service

## Changes
* Fix AggregateResultCallback test case
* Fix configuration and Unit test on hbase
* Support GenericInMemoryAggregateQuery when coprocessor is not enabled and faster-xml as default json library
* Fix unit test: TestGenericServiceAPIResponseEntity
* Fix coprocessor-based aggregation query.
* Add integration test cases: `CoprocessorITSuite` and `CoprocessorToolITSuite`
* Fix coprocessor to support `hbase-0.98` and `protobuf-2.5.0`
* Add coprocessor tool:

~~~
usage: java org.apache.eagle.storage.hbase.tools.CoprocessorTool
            [--register/--unregister] [OPTIONS]
 -config <arg>     Configuration file
 -jar <arg>        Coprocessor target jar path
 -localJar <arg>   Coprocessor local source jar path
 -register         Register coprocessor
 -table <arg>      HBase table name, separated with comma, for example,
                   table1,table2,..
 -unregister       Unregister coprocessor
~~~

Author: Hao Chen <ha...@apache.org>

Closes #417 from haoch/EAGLE-520.


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

Branch: refs/heads/master
Commit: 21187b55c9e549de9d8b797a59516b0c99507f90
Parents: 3324279
Author: Hao Chen <ha...@apache.org>
Authored: Wed Sep 7 20:30:33 2016 +0800
Committer: Hao Chen <ha...@apache.org>
Committed: Wed Sep 7 20:30:33 2016 +0800

----------------------------------------------------------------------
 eagle-assembly/pom.xml                          |  258 +-
 .../eagle-alert/alert-common/pom.xml            |   10 +-
 .../alert/coordination/model/RouterSpec.java    |    3 +-
 .../alert/coordination/model/SpoutSpec.java     |    4 +-
 .../model/StreamRepartitionMetadata.java        |    3 +-
 .../model/internal/StreamGroup.java             |    1 -
 .../model/internal/StreamWorkSlotQueue.java     |    2 -
 .../engine/coordinator/PolicyDefinition.java    |    2 +-
 .../alert/engine/coordinator/Publishment.java   |    2 +-
 .../engine/coordinator/PublishmentType.java     |    2 +-
 .../alert/engine/coordinator/StreamColumn.java  |    1 -
 .../engine/coordinator/StreamPartition.java     |    2 +-
 .../engine/coordinator/StreamSortSpec.java      |    2 +-
 .../service/MetadataServiceClientImpl.java      |    2 +-
 .../org/apache/eagle/alert/utils/JsonUtils.java |    7 +-
 .../eagle/alert/utils/ZookeeperEmbedded.java    |   13 +-
 .../service/TestMetadataServiceClientImpl.java  |    2 +-
 .../apache/eagle/correlation/meta/JsonTest.java |    2 +-
 .../eagle-alert/alert-coordinator/pom.xml       |   24 +-
 .../eagle-alert/alert-engine/pom.xml            |    8 +-
 .../alert/engine/utils/SerializableUtils.java   |  175 +-
 .../alert/engine/e2e/CoordinatorClient.java     |    3 +-
 .../alert-metadata-service/pom.xml              |   24 +-
 .../environment/impl/StormExecutionRuntime.java |    6 +-
 .../impl/ApplicationProviderSPILoader.java      |    3 +-
 .../eagle/common/EagleExceptionWrapper.java     |   50 +-
 .../common/config/EagleConfigConstants.java     |   30 +-
 .../src/test/resources/test-service-config.conf |   27 +-
 .../apache/eagle/query/parser/TokenType.java    |   46 +-
 .../eagle-query/eagle-client-base/pom.xml       |   12 +-
 .../client/impl/EagleServiceBaseClient.java     |   45 +-
 .../client/TestEagleServiceClientImpl.java      |   40 +-
 .../eagle-query/eagle-entity-base/pom.xml       |    8 +
 .../log/base/taggedlog/EntityJsonModule.java    |   27 +
 .../log/base/taggedlog/MapEntrySerializer.java  |   38 +
 .../log/base/taggedlog/TaggedLogAPIEntity.java  |  409 +-
 .../eagle/log/entity/GenericMetricEntity.java   |    2 +-
 .../entity/GenericServiceAPIResponseEntity.java |   11 +-
 ...ricServiceAPIResponseEntityDeserializer.java |   29 +-
 .../eagle/log/entity/MetricMetadataEntity.java  |   16 +-
 .../eagle/log/entity/test/TestEntity.java       |   10 +-
 .../eagle/log/entity/test/TestLogAPIEntity.java |   14 +-
 .../entity/test/TestTimeSeriesAPIEntity.java    |   14 +-
 .../TestGenericServiceAPIResponseEntity.java    |    4 +-
 .../base/taggedlog/TestTaggedLogAPIEntity.java  |   40 +-
 .../eagle/query/aggregate/raw/GroupbyKey.java   |   15 +-
 .../query/aggregate/test/TestBucketQuery.java   |    4 +-
 .../eagle-query/eagle-service-base/pom.xml      |    8 +-
 .../generic/GenericEntityServiceResource.java   |    5 +
 .../generic/GenericObjectMapperProvider.java    |    4 +-
 .../service/selfcheck/ServiceResource.java      |   57 +-
 .../eagle-query/eagle-storage-base/pom.xml      |    2 +-
 .../src/test/resources/application.conf         |    4 +-
 .../eagle-query/eagle-storage-hbase/pom.xml     |    2 +-
 .../src/assembly/hbase-coprocessor.xml          |    1 +
 .../storage/hbase/HBaseEntitySchemaManager.java |    5 +-
 .../eagle/storage/hbase/HBaseStorage.java       |  157 +-
 .../eagle/storage/hbase/HBaseStorageAudit.java  |  204 +-
 .../hbase/query/GenericQueryBuilder.java        |  270 +-
 .../query/aggregate/GenericAggregateQuery.java  |  423 -
 .../query/aggregate/GenericAggregateReader.java |  175 +-
 .../GenericCoprocessorAggregateQuery.java       |  394 +
 .../GenericInMemoryAggregateQuery.java          |  211 +
 .../query/coprocessor/AggregateClient.java      |   69 +-
 .../query/coprocessor/AggregateProtocol.java    |   45 +-
 .../coprocessor/AggregateProtocolEndPoint.java  |  592 +-
 .../query/coprocessor/AggregateResult.java      |  171 +-
 .../coprocessor/AggregateResultCallback.java    |   24 +-
 .../query/coprocessor/ProtoBufConverter.java    |   76 +-
 .../coprocessor/_AggregateProtocolEndPoint.java |  139 -
 .../coprocessor/generated/AggregateProtos.java  | 8450 +++++++++---------
 .../coprocessor/impl/AggregateClientImpl.java   |  204 +-
 .../impl/AggregateResultCallbackImpl.java       |  194 +-
 .../hbase/query/coprocessor/package-info.java   |   99 +-
 .../hbase/spi/HBaseStorageServiceProvider.java  |    5 +-
 .../storage/hbase/tools/CoprocessorTool.java    |  159 +
 .../src/main/protobuf/AggregateProtocol.proto   |   10 +-
 .../eagle/storage/hbase/TestHBaseStatement.java |    6 +-
 .../eagle/storage/hbase/TestHBaseStorage.java   |   34 -
 .../TestAggregateResultCallback.java            |  121 +-
 .../coprocessor/TestGroupAggregateClient.java   |  399 +-
 .../TestGroupAggregateTimeSeriesClient.java     |  440 +-
 .../integrationtest/CoprocessorITSuite.java     |  246 +
 .../integrationtest/CoprocessorJarUtils.java    |   41 +
 .../integrationtest/CoprocessorToolITSuite.java |   92 +
 .../storage/hbase/spi/TestHBaseStorage.java     |   29 -
 .../hbase/spi/TestHBaseStorageLoader.java       |    3 +-
 .../src/test/resources/application-sandbox.conf |   23 +
 .../src/test/resources/application.conf         |   22 +-
 .../src/test/resources/hbase-site-sandbox.xml   |   40 +
 .../src/test/resources/log4j.properties         |    2 -
 .../eagle/storage/jdbc/JdbcConstants.java       |   16 +-
 .../jdbc/schema/JdbcEntitySchemaManager.java    |   13 +-
 .../schema/serializer/MetricJdbcSerDeser.java   |   27 +-
 .../src/test/resources/application-derby.conf   |    4 +-
 .../src/test/resources/application-mysql.conf   |    4 +-
 .../src/test/resources/application.conf         |    4 +-
 .../JobConfigurationAPIEntity.java              |    6 +-
 .../jpm/mr/historyentity/JobEventAPIEntity.java |    2 +-
 .../mr/historyentity/JobExecutionAPIEntity.java |   20 +-
 .../JobProcessTimeStampEntity.java              |    2 +-
 .../TaskAttemptCounterAPIEntity.java            |    6 +-
 .../TaskAttemptExecutionAPIEntity.java          |   14 +-
 .../historyentity/TaskExecutionAPIEntity.java   |   12 +-
 .../TaskFailureCountAPIEntity.java              |    6 +-
 .../hdfs/rest/HDFSResourceWebResource.java      |   94 +-
 eagle-server-assembly/pom.xml                   |    1 -
 eagle-server-assembly/src/main/bin/eagle-env.sh |   62 +-
 .../src/main/bin/eagle-service.sh               |  108 -
 .../eagle/server/RESTExceptionMapper.java       |    7 +-
 .../apache/eagle/server/ServerApplication.java  |   19 +-
 .../org/apache/eagle/server/ServerConfig.java   |   33 +-
 .../org/apache/eagle/server/ServerMain.java     |    6 +-
 .../eagle/server/module/GuideBundleLoader.java  |   31 +-
 .../eagle/server/module/ServerModule.java       |    5 +-
 .../src/main/resources/application.conf         |  124 +-
 .../src/main/resources/configuration.yml        |    3 +-
 .../src/main/resources/log4j.properties         |    2 -
 .../src/main/resources/providers-disabled.xml   |   27 -
 .../eagle/server/GuideBundleLoaderTest.java     |    2 +-
 .../eagle/server/ServerApplicationTest.java     |    2 +-
 .../src/main/resources/application-derby.conf   |    4 +-
 .../src/main/resources/application-hbase.conf   |    4 +-
 .../src/main/resources/application-mysql.conf   |    4 +-
 .../src/main/resources/application.conf         |   77 +-
 pom.xml                                         |   35 +-
 126 files changed, 8205 insertions(+), 7690 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/21187b55/eagle-assembly/pom.xml
----------------------------------------------------------------------
diff --git a/eagle-assembly/pom.xml b/eagle-assembly/pom.xml
index 90ee03c..2849ccd 100644
--- a/eagle-assembly/pom.xml
+++ b/eagle-assembly/pom.xml
@@ -1,129 +1,129 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
-  ~ Licensed to the Apache Software Foundation (ASF) under one or more
-  ~ contributor license agreements.  See the NOTICE file distributed with
-  ~ this work for additional information regarding copyright ownership.
-  ~ The ASF licenses this file to You under the Apache License, Version 2.0
-  ~ (the "License"); you may not use this file except in compliance with
-  ~ the License.  You may obtain a copy of the License at
-  ~
-  ~    http://www.apache.org/licenses/LICENSE-2.0
-  ~
-  ~ Unless required by applicable law or agreed to in writing, software
-  ~ distributed under the License is distributed on an "AS IS" BASIS,
-  ~ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-  ~ See the License for the specific language governing permissions and
-  ~ limitations under the License.
-  -->
-
-<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
-         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
-    <parent>
-        <artifactId>eagle-parent</artifactId>
-        <groupId>org.apache.eagle</groupId>
-        <version>0.5.0-incubating-SNAPSHOT</version>
-        <relativePath>../pom.xml</relativePath>
-    </parent>
-    <modelVersion>4.0.0</modelVersion>
-    <artifactId>eagle-assembly</artifactId>
-    <dependencies>
-        <!-- tomcat -->
-        <dependency>
-            <groupId>org.apache.tomcat</groupId>
-            <artifactId>tomcat-catalina</artifactId>
-            <version>${tomcat.version}</version>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.tomcat</groupId>
-            <artifactId>tomcat-jdbc</artifactId>
-            <version>${tomcat.version}</version>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.tomcat</groupId>
-            <artifactId>tomcat-coyote</artifactId>
-            <version>${tomcat.version}</version>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.tomcat</groupId>
-            <artifactId>tomcat-dbcp</artifactId>
-            <version>${tomcat.version}</version>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.tomcat</groupId>
-            <artifactId>tomcat-catalina-ha</artifactId>
-            <version>${tomcat.version}</version>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.tomcat</groupId>
-            <artifactId>tomcat-catalina-ant</artifactId>
-            <version>${tomcat.version}</version>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.tomcat</groupId>
-            <artifactId>tomcat-jasper</artifactId>
-            <version>${tomcat.version}</version>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.tomcat</groupId>
-            <artifactId>tomcat-catalina-ws</artifactId>
-            <version>${tomcat.version}</version>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.tomcat</groupId>
-            <artifactId>tomcat-catalina-jmx-remote</artifactId>
-            <version>${tomcat.version}</version>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.eagle</groupId>
-            <artifactId>eagle-topology-assembly</artifactId>
-            <version>${project.version}</version>
-        </dependency>
-        <dependency>
-            <groupId>org.ow2.asm</groupId>
-            <artifactId>asm</artifactId>
-            <version>4.0</version>
-        </dependency>
-    </dependencies>
-    <build>
-        <plugins>
-            <plugin>
-                <groupId>org.apache.maven.plugins</groupId>
-                <artifactId>maven-compiler-plugin</artifactId>
-            </plugin>
-
-            <plugin>
-                <artifactId>maven-dependency-plugin</artifactId>
-                <executions>
-                    <execution>
-                        <phase>package</phase>
-                        <goals>
-                            <goal>copy-dependencies</goal>
-                        </goals>
-                        <configuration>
-                            <outputDirectory>${project.build.directory}/lib</outputDirectory>
-                        </configuration>
-                    </execution>
-                </executions>
-            </plugin>
-
-            <plugin>
-                <artifactId>maven-assembly-plugin</artifactId>
-                <configuration>
-                    <descriptor>src/assembly/eagle-bin.xml</descriptor>
-                    <finalName>eagle-${project.version}</finalName>
-                </configuration>
-                <executions>
-                    <execution>
-                        <phase>package</phase>
-                        <goals>
-                            <goal>single</goal>
-                        </goals>
-                        <configuration>
-                            <tarLongFileMode>posix</tarLongFileMode>
-                        </configuration>
-                    </execution>
-                </executions>
-            </plugin>
-        </plugins>
-    </build>
-</project>
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  ~ Licensed to the Apache Software Foundation (ASF) under one or more
+  ~ contributor license agreements.  See the NOTICE file distributed with
+  ~ this work for additional information regarding copyright ownership.
+  ~ The ASF licenses this file to You under the Apache License, Version 2.0
+  ~ (the "License"); you may not use this file except in compliance with
+  ~ the License.  You may obtain a copy of the License at
+  ~
+  ~    http://www.apache.org/licenses/LICENSE-2.0
+  ~
+  ~ Unless required by applicable law or agreed to in writing, software
+  ~ distributed under the License is distributed on an "AS IS" BASIS,
+  ~ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  ~ See the License for the specific language governing permissions and
+  ~ limitations under the License.
+  -->
+
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <parent>
+        <artifactId>eagle-parent</artifactId>
+        <groupId>org.apache.eagle</groupId>
+        <version>0.5.0-incubating-SNAPSHOT</version>
+        <relativePath>../pom.xml</relativePath>
+    </parent>
+    <modelVersion>4.0.0</modelVersion>
+    <artifactId>eagle-assembly</artifactId>
+    <dependencies>
+        <!-- tomcat -->
+        <dependency>
+            <groupId>org.apache.tomcat</groupId>
+            <artifactId>tomcat-catalina</artifactId>
+            <version>${tomcat.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.tomcat</groupId>
+            <artifactId>tomcat-jdbc</artifactId>
+            <version>${tomcat.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.tomcat</groupId>
+            <artifactId>tomcat-coyote</artifactId>
+            <version>${tomcat.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.tomcat</groupId>
+            <artifactId>tomcat-dbcp</artifactId>
+            <version>${tomcat.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.tomcat</groupId>
+            <artifactId>tomcat-catalina-ha</artifactId>
+            <version>${tomcat.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.tomcat</groupId>
+            <artifactId>tomcat-catalina-ant</artifactId>
+            <version>${tomcat.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.tomcat</groupId>
+            <artifactId>tomcat-jasper</artifactId>
+            <version>${tomcat.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.tomcat</groupId>
+            <artifactId>tomcat-catalina-ws</artifactId>
+            <version>${tomcat.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.tomcat</groupId>
+            <artifactId>tomcat-catalina-jmx-remote</artifactId>
+            <version>${tomcat.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.eagle</groupId>
+            <artifactId>eagle-topology-assembly</artifactId>
+            <version>${project.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>org.ow2.asm</groupId>
+            <artifactId>asm</artifactId>
+            <version>4.0</version>
+        </dependency>
+    </dependencies>
+    <build>
+        <plugins>
+            <plugin>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-compiler-plugin</artifactId>
+            </plugin>
+
+            <plugin>
+                <artifactId>maven-dependency-plugin</artifactId>
+                <executions>
+                    <execution>
+                        <phase>package</phase>
+                        <goals>
+                            <goal>copy-dependencies</goal>
+                        </goals>
+                        <configuration>
+                            <outputDirectory>${project.build.directory}/lib</outputDirectory>
+                        </configuration>
+                    </execution>
+                </executions>
+            </plugin>
+
+            <plugin>
+                <artifactId>maven-assembly-plugin</artifactId>
+                <configuration>
+                    <descriptor>src/assembly/eagle-bin.xml</descriptor>
+                    <finalName>eagle-${project.version}</finalName>
+                </configuration>
+                <executions>
+                    <execution>
+                        <phase>package</phase>
+                        <goals>
+                            <goal>single</goal>
+                        </goals>
+                        <configuration>
+                            <tarLongFileMode>posix</tarLongFileMode>
+                        </configuration>
+                    </execution>
+                </executions>
+            </plugin>
+        </plugins>
+    </build>
+</project>

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/21187b55/eagle-core/eagle-alert-parent/eagle-alert/alert-common/pom.xml
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-alert-parent/eagle-alert/alert-common/pom.xml b/eagle-core/eagle-alert-parent/eagle-alert/alert-common/pom.xml
index b56e27f..0fb2bb7 100644
--- a/eagle-core/eagle-alert-parent/eagle-alert/alert-common/pom.xml
+++ b/eagle-core/eagle-alert-parent/eagle-alert/alert-common/pom.xml
@@ -18,6 +18,7 @@
         <groupId>org.apache.eagle</groupId>
         <artifactId>eagle-alert</artifactId>
         <version>0.5.0-incubating-SNAPSHOT</version>
+        <relativePath>..</relativePath>
     </parent>
 
     <artifactId>alert-common</artifactId>
@@ -77,11 +78,14 @@
             <groupId>com.fasterxml.jackson.core</groupId>
             <artifactId>jackson-databind</artifactId>
         </dependency>
-
         <dependency>
-            <groupId>org.codehaus.jackson</groupId>
-            <artifactId>jackson-jaxrs</artifactId>
+            <groupId>com.fasterxml.jackson.jaxrs</groupId>
+            <artifactId>jackson-jaxrs-json-provider</artifactId>
         </dependency>
+        <!--<dependency>-->
+            <!--<groupId>org.codehaus.jackson</groupId>-->
+            <!--<artifactId>jackson-jaxrs</artifactId>-->
+        <!--</dependency>-->
         <dependency>
             <groupId>joda-time</groupId>
             <artifactId>joda-time</artifactId>

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/21187b55/eagle-core/eagle-alert-parent/eagle-alert/alert-common/src/main/java/org/apache/eagle/alert/coordination/model/RouterSpec.java
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-alert-parent/eagle-alert/alert-common/src/main/java/org/apache/eagle/alert/coordination/model/RouterSpec.java b/eagle-core/eagle-alert-parent/eagle-alert/alert-common/src/main/java/org/apache/eagle/alert/coordination/model/RouterSpec.java
index 5241920..9a576a3 100644
--- a/eagle-core/eagle-alert-parent/eagle-alert/alert-common/src/main/java/org/apache/eagle/alert/coordination/model/RouterSpec.java
+++ b/eagle-core/eagle-alert-parent/eagle-alert/alert-common/src/main/java/org/apache/eagle/alert/coordination/model/RouterSpec.java
@@ -16,10 +16,11 @@
  */
 package org.apache.eagle.alert.coordination.model;
 
+import com.fasterxml.jackson.annotation.JsonIgnore;
+
 import java.util.ArrayList;
 import java.util.List;
 
-import org.codehaus.jackson.annotate.JsonIgnore;
 
 /**
  * @since Apr 29, 2016

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/21187b55/eagle-core/eagle-alert-parent/eagle-alert/alert-common/src/main/java/org/apache/eagle/alert/coordination/model/SpoutSpec.java
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-alert-parent/eagle-alert/alert-common/src/main/java/org/apache/eagle/alert/coordination/model/SpoutSpec.java b/eagle-core/eagle-alert-parent/eagle-alert/alert-common/src/main/java/org/apache/eagle/alert/coordination/model/SpoutSpec.java
index a197858..c0e8ec1 100644
--- a/eagle-core/eagle-alert-parent/eagle-alert/alert-common/src/main/java/org/apache/eagle/alert/coordination/model/SpoutSpec.java
+++ b/eagle-core/eagle-alert-parent/eagle-alert/alert-common/src/main/java/org/apache/eagle/alert/coordination/model/SpoutSpec.java
@@ -16,6 +16,8 @@
  */
 package org.apache.eagle.alert.coordination.model;
 
+import com.fasterxml.jackson.annotation.JsonIgnore;
+
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
@@ -81,7 +83,7 @@ public class SpoutSpec {
         return kafka2TupleMetadataMap;
     }
 
-    @org.codehaus.jackson.annotate.JsonIgnore
+    @JsonIgnore
     public StreamRepartitionMetadata getStream(String streamName) {
         for (List<StreamRepartitionMetadata> meta : this.streamRepartitionMetadataMap.values()) {
             Optional<StreamRepartitionMetadata> m = meta.stream().filter((t) -> t.getStreamId().equalsIgnoreCase(streamName)).findFirst();

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/21187b55/eagle-core/eagle-alert-parent/eagle-alert/alert-common/src/main/java/org/apache/eagle/alert/coordination/model/StreamRepartitionMetadata.java
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-alert-parent/eagle-alert/alert-common/src/main/java/org/apache/eagle/alert/coordination/model/StreamRepartitionMetadata.java b/eagle-core/eagle-alert-parent/eagle-alert/alert-common/src/main/java/org/apache/eagle/alert/coordination/model/StreamRepartitionMetadata.java
index dac04e8..86b1d27 100644
--- a/eagle-core/eagle-alert-parent/eagle-alert/alert-common/src/main/java/org/apache/eagle/alert/coordination/model/StreamRepartitionMetadata.java
+++ b/eagle-core/eagle-alert-parent/eagle-alert/alert-common/src/main/java/org/apache/eagle/alert/coordination/model/StreamRepartitionMetadata.java
@@ -16,10 +16,11 @@
  */
 package org.apache.eagle.alert.coordination.model;
 
+import com.fasterxml.jackson.annotation.JsonIgnore;
+
 import java.util.ArrayList;
 import java.util.List;
 
-import org.codehaus.jackson.annotate.JsonIgnore;
 
 /**
  * @since Apr 25, 2016

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/21187b55/eagle-core/eagle-alert-parent/eagle-alert/alert-common/src/main/java/org/apache/eagle/alert/coordination/model/internal/StreamGroup.java
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-alert-parent/eagle-alert/alert-common/src/main/java/org/apache/eagle/alert/coordination/model/internal/StreamGroup.java b/eagle-core/eagle-alert-parent/eagle-alert/alert-common/src/main/java/org/apache/eagle/alert/coordination/model/internal/StreamGroup.java
index 9cbd841..d87d62b 100644
--- a/eagle-core/eagle-alert-parent/eagle-alert/alert-common/src/main/java/org/apache/eagle/alert/coordination/model/internal/StreamGroup.java
+++ b/eagle-core/eagle-alert-parent/eagle-alert/alert-common/src/main/java/org/apache/eagle/alert/coordination/model/internal/StreamGroup.java
@@ -48,7 +48,6 @@ public class StreamGroup {
         this.streamPartitions.addAll(sps);
     }
 
-    @org.codehaus.jackson.annotate.JsonIgnore
     @JsonIgnore
     public String getStreamId() {
         StringBuilder sb = new StringBuilder("SG[");

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/21187b55/eagle-core/eagle-alert-parent/eagle-alert/alert-common/src/main/java/org/apache/eagle/alert/coordination/model/internal/StreamWorkSlotQueue.java
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-alert-parent/eagle-alert/alert-common/src/main/java/org/apache/eagle/alert/coordination/model/internal/StreamWorkSlotQueue.java b/eagle-core/eagle-alert-parent/eagle-alert/alert-common/src/main/java/org/apache/eagle/alert/coordination/model/internal/StreamWorkSlotQueue.java
index fab6217..f4f6142 100644
--- a/eagle-core/eagle-alert-parent/eagle-alert/alert-common/src/main/java/org/apache/eagle/alert/coordination/model/internal/StreamWorkSlotQueue.java
+++ b/eagle-core/eagle-alert-parent/eagle-alert/alert-common/src/main/java/org/apache/eagle/alert/coordination/model/internal/StreamWorkSlotQueue.java
@@ -76,7 +76,6 @@ public class StreamWorkSlotQueue {
         this.dedicated = dedicated;
     }
 
-    @org.codehaus.jackson.annotate.JsonIgnore
     @JsonIgnore
     public int getQueueSize() {
         return workingSlots.size();
@@ -104,7 +103,6 @@ public class StreamWorkSlotQueue {
         this.topoGroupStartIndex = topoGroupStartIndex;
     }
 
-    @org.codehaus.jackson.annotate.JsonIgnore
     @JsonIgnore
     public int getTopologyGroupStartIndex(String topo) {
         if (topoGroupStartIndex.containsKey(topo)) {

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/21187b55/eagle-core/eagle-alert-parent/eagle-alert/alert-common/src/main/java/org/apache/eagle/alert/engine/coordinator/PolicyDefinition.java
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-alert-parent/eagle-alert/alert-common/src/main/java/org/apache/eagle/alert/engine/coordinator/PolicyDefinition.java b/eagle-core/eagle-alert-parent/eagle-alert/alert-common/src/main/java/org/apache/eagle/alert/engine/coordinator/PolicyDefinition.java
index b154514..c54955f 100644
--- a/eagle-core/eagle-alert-parent/eagle-alert/alert-common/src/main/java/org/apache/eagle/alert/engine/coordinator/PolicyDefinition.java
+++ b/eagle-core/eagle-alert-parent/eagle-alert/alert-common/src/main/java/org/apache/eagle/alert/engine/coordinator/PolicyDefinition.java
@@ -16,10 +16,10 @@
  */
 package org.apache.eagle.alert.engine.coordinator;
 
+import com.fasterxml.jackson.annotation.JsonIgnoreProperties;
 import org.apache.commons.collections.CollectionUtils;
 import org.apache.commons.collections.ListUtils;
 import org.apache.commons.lang3.builder.HashCodeBuilder;
-import org.codehaus.jackson.annotate.JsonIgnoreProperties;
 
 import java.io.Serializable;
 import java.util.*;

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/21187b55/eagle-core/eagle-alert-parent/eagle-alert/alert-common/src/main/java/org/apache/eagle/alert/engine/coordinator/Publishment.java
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-alert-parent/eagle-alert/alert-common/src/main/java/org/apache/eagle/alert/engine/coordinator/Publishment.java b/eagle-core/eagle-alert-parent/eagle-alert/alert-common/src/main/java/org/apache/eagle/alert/engine/coordinator/Publishment.java
index 3c1da58..e3b4e33 100644
--- a/eagle-core/eagle-alert-parent/eagle-alert/alert-common/src/main/java/org/apache/eagle/alert/engine/coordinator/Publishment.java
+++ b/eagle-core/eagle-alert-parent/eagle-alert/alert-common/src/main/java/org/apache/eagle/alert/engine/coordinator/Publishment.java
@@ -16,8 +16,8 @@
  */
 package org.apache.eagle.alert.engine.coordinator;
 
+import com.fasterxml.jackson.annotation.JsonIgnoreProperties;
 import org.apache.commons.lang3.builder.HashCodeBuilder;
-import org.codehaus.jackson.annotate.JsonIgnoreProperties;
 
 import java.util.List;
 import java.util.Map;

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/21187b55/eagle-core/eagle-alert-parent/eagle-alert/alert-common/src/main/java/org/apache/eagle/alert/engine/coordinator/PublishmentType.java
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-alert-parent/eagle-alert/alert-common/src/main/java/org/apache/eagle/alert/engine/coordinator/PublishmentType.java b/eagle-core/eagle-alert-parent/eagle-alert/alert-common/src/main/java/org/apache/eagle/alert/engine/coordinator/PublishmentType.java
index a3f2356..daecab4 100644
--- a/eagle-core/eagle-alert-parent/eagle-alert/alert-common/src/main/java/org/apache/eagle/alert/engine/coordinator/PublishmentType.java
+++ b/eagle-core/eagle-alert-parent/eagle-alert/alert-common/src/main/java/org/apache/eagle/alert/engine/coordinator/PublishmentType.java
@@ -20,8 +20,8 @@ package org.apache.eagle.alert.engine.coordinator;
 
 import java.util.Objects;
 
+import com.fasterxml.jackson.annotation.JsonIgnoreProperties;
 import org.apache.commons.lang3.builder.HashCodeBuilder;
-import org.codehaus.jackson.annotate.JsonIgnoreProperties;
 
 @JsonIgnoreProperties(ignoreUnknown = true)
 public class PublishmentType {

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/21187b55/eagle-core/eagle-alert-parent/eagle-alert/alert-common/src/main/java/org/apache/eagle/alert/engine/coordinator/StreamColumn.java
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-alert-parent/eagle-alert/alert-common/src/main/java/org/apache/eagle/alert/engine/coordinator/StreamColumn.java b/eagle-core/eagle-alert-parent/eagle-alert/alert-common/src/main/java/org/apache/eagle/alert/engine/coordinator/StreamColumn.java
index c0d355e..4483fe4 100644
--- a/eagle-core/eagle-alert-parent/eagle-alert/alert-common/src/main/java/org/apache/eagle/alert/engine/coordinator/StreamColumn.java
+++ b/eagle-core/eagle-alert-parent/eagle-alert/alert-common/src/main/java/org/apache/eagle/alert/engine/coordinator/StreamColumn.java
@@ -134,7 +134,6 @@ public class StreamColumn implements Serializable {
             return name;
         }
 
-        @org.codehaus.jackson.annotate.JsonCreator
         @com.fasterxml.jackson.annotation.JsonCreator
         public static Type getEnumFromValue(String value) {
             for (Type testEnum : values()) {

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/21187b55/eagle-core/eagle-alert-parent/eagle-alert/alert-common/src/main/java/org/apache/eagle/alert/engine/coordinator/StreamPartition.java
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-alert-parent/eagle-alert/alert-common/src/main/java/org/apache/eagle/alert/engine/coordinator/StreamPartition.java b/eagle-core/eagle-alert-parent/eagle-alert/alert-common/src/main/java/org/apache/eagle/alert/engine/coordinator/StreamPartition.java
index cdf0dbf..47e15c0 100644
--- a/eagle-core/eagle-alert-parent/eagle-alert/alert-common/src/main/java/org/apache/eagle/alert/engine/coordinator/StreamPartition.java
+++ b/eagle-core/eagle-alert-parent/eagle-alert/alert-common/src/main/java/org/apache/eagle/alert/engine/coordinator/StreamPartition.java
@@ -19,10 +19,10 @@ package org.apache.eagle.alert.engine.coordinator;
 import java.io.Serializable;
 import java.util.*;
 
+import com.fasterxml.jackson.annotation.JsonIgnoreProperties;
 import org.apache.commons.collections.CollectionUtils;
 import org.apache.commons.lang3.StringUtils;
 import org.apache.commons.lang3.builder.HashCodeBuilder;
-import org.codehaus.jackson.annotate.JsonIgnoreProperties;
 
 /**
  * StreamPartition defines how a data stream is partitioned and sorted

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/21187b55/eagle-core/eagle-alert-parent/eagle-alert/alert-common/src/main/java/org/apache/eagle/alert/engine/coordinator/StreamSortSpec.java
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-alert-parent/eagle-alert/alert-common/src/main/java/org/apache/eagle/alert/engine/coordinator/StreamSortSpec.java b/eagle-core/eagle-alert-parent/eagle-alert/alert-common/src/main/java/org/apache/eagle/alert/engine/coordinator/StreamSortSpec.java
index a8b57f2..962a8ee 100644
--- a/eagle-core/eagle-alert-parent/eagle-alert/alert-common/src/main/java/org/apache/eagle/alert/engine/coordinator/StreamSortSpec.java
+++ b/eagle-core/eagle-alert-parent/eagle-alert/alert-common/src/main/java/org/apache/eagle/alert/engine/coordinator/StreamSortSpec.java
@@ -17,9 +17,9 @@
 package org.apache.eagle.alert.engine.coordinator;
 
 
+import com.fasterxml.jackson.annotation.JsonIgnoreProperties;
 import org.apache.commons.lang.builder.HashCodeBuilder;
 import org.apache.eagle.alert.utils.TimePeriodUtils;
-import org.codehaus.jackson.annotate.JsonIgnoreProperties;
 import org.joda.time.Period;
 
 import java.io.Serializable;

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/21187b55/eagle-core/eagle-alert-parent/eagle-alert/alert-common/src/main/java/org/apache/eagle/alert/service/MetadataServiceClientImpl.java
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-alert-parent/eagle-alert/alert-common/src/main/java/org/apache/eagle/alert/service/MetadataServiceClientImpl.java b/eagle-core/eagle-alert-parent/eagle-alert/alert-common/src/main/java/org/apache/eagle/alert/service/MetadataServiceClientImpl.java
index 636020e..2ddba55 100644
--- a/eagle-core/eagle-alert-parent/eagle-alert/alert-common/src/main/java/org/apache/eagle/alert/service/MetadataServiceClientImpl.java
+++ b/eagle-core/eagle-alert-parent/eagle-alert/alert-common/src/main/java/org/apache/eagle/alert/service/MetadataServiceClientImpl.java
@@ -25,6 +25,7 @@ import java.util.List;
 
 import javax.ws.rs.core.MediaType;
 
+import com.fasterxml.jackson.jaxrs.json.JacksonJsonProvider;
 import org.apache.eagle.alert.coordination.model.Kafka2TupleMetadata;
 import org.apache.eagle.alert.coordination.model.ScheduleState;
 import org.apache.eagle.alert.coordination.model.SpoutSpec;
@@ -33,7 +34,6 @@ import org.apache.eagle.alert.engine.coordinator.PolicyDefinition;
 import org.apache.eagle.alert.engine.coordinator.Publishment;
 import org.apache.eagle.alert.engine.coordinator.StreamDefinition;
 import org.apache.eagle.alert.engine.coordinator.StreamingCluster;
-import org.codehaus.jackson.jaxrs.JacksonJsonProvider;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/21187b55/eagle-core/eagle-alert-parent/eagle-alert/alert-common/src/main/java/org/apache/eagle/alert/utils/JsonUtils.java
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-alert-parent/eagle-alert/alert-common/src/main/java/org/apache/eagle/alert/utils/JsonUtils.java b/eagle-core/eagle-alert-parent/eagle-alert/alert-common/src/main/java/org/apache/eagle/alert/utils/JsonUtils.java
index 09ee5f4..2ee1a5f 100644
--- a/eagle-core/eagle-alert-parent/eagle-alert/alert-common/src/main/java/org/apache/eagle/alert/utils/JsonUtils.java
+++ b/eagle-core/eagle-alert-parent/eagle-alert/alert-common/src/main/java/org/apache/eagle/alert/utils/JsonUtils.java
@@ -16,15 +16,10 @@
  */
 package org.apache.eagle.alert.utils;
 
-import org.codehaus.jackson.map.ObjectMapper;
+import com.fasterxml.jackson.databind.ObjectMapper;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-
-/**
- * @since May 1, 2016
- *
- */
 public class JsonUtils {
 
     public static final ObjectMapper mapper = new ObjectMapper();

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/21187b55/eagle-core/eagle-alert-parent/eagle-alert/alert-common/src/main/java/org/apache/eagle/alert/utils/ZookeeperEmbedded.java
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-alert-parent/eagle-alert/alert-common/src/main/java/org/apache/eagle/alert/utils/ZookeeperEmbedded.java b/eagle-core/eagle-alert-parent/eagle-alert/alert-common/src/main/java/org/apache/eagle/alert/utils/ZookeeperEmbedded.java
index 6140677..c7c358f 100644
--- a/eagle-core/eagle-alert-parent/eagle-alert/alert-common/src/main/java/org/apache/eagle/alert/utils/ZookeeperEmbedded.java
+++ b/eagle-core/eagle-alert-parent/eagle-alert/alert-common/src/main/java/org/apache/eagle/alert/utils/ZookeeperEmbedded.java
@@ -52,15 +52,18 @@ public class ZookeeperEmbedded {
 
     public void shutdown() {
         try {
-            if (!zookeeper.getState().equals(CuratorFrameworkState.STOPPED)) {
-                zookeeper.close();
+            if(zookeeper!=null) {
+                if (!zookeeper.getState().equals(CuratorFrameworkState.STOPPED)) {
+                    zookeeper.close();
+                }
             }
-
         } catch (Throwable e) {
             e.printStackTrace();
         } finally {
             try {
-                server.close();
+                if(server!=null) {
+                    server.close();
+                }
             } catch (IOException e) {
                 e.printStackTrace();
             } finally {
@@ -68,4 +71,4 @@ public class ZookeeperEmbedded {
             }
         }
     }
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/21187b55/eagle-core/eagle-alert-parent/eagle-alert/alert-common/src/test/java/org/apache/eagle/alert/service/TestMetadataServiceClientImpl.java
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-alert-parent/eagle-alert/alert-common/src/test/java/org/apache/eagle/alert/service/TestMetadataServiceClientImpl.java b/eagle-core/eagle-alert-parent/eagle-alert/alert-common/src/test/java/org/apache/eagle/alert/service/TestMetadataServiceClientImpl.java
index 4496b8e..707515e 100644
--- a/eagle-core/eagle-alert-parent/eagle-alert/alert-common/src/test/java/org/apache/eagle/alert/service/TestMetadataServiceClientImpl.java
+++ b/eagle-core/eagle-alert-parent/eagle-alert/alert-common/src/test/java/org/apache/eagle/alert/service/TestMetadataServiceClientImpl.java
@@ -21,8 +21,8 @@ package org.apache.eagle.alert.service;
 
 import java.util.List;
 
+import com.fasterxml.jackson.databind.ObjectMapper;
 import org.apache.eagle.alert.engine.coordinator.PolicyDefinition;
-import org.codehaus.jackson.map.ObjectMapper;
 import org.junit.Ignore;
 import org.junit.Test;
 

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/21187b55/eagle-core/eagle-alert-parent/eagle-alert/alert-common/src/test/java/org/apache/eagle/correlation/meta/JsonTest.java
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-alert-parent/eagle-alert/alert-common/src/test/java/org/apache/eagle/correlation/meta/JsonTest.java b/eagle-core/eagle-alert-parent/eagle-alert/alert-common/src/test/java/org/apache/eagle/correlation/meta/JsonTest.java
index 245b249..ff0a3f9 100644
--- a/eagle-core/eagle-alert-parent/eagle-alert/alert-common/src/test/java/org/apache/eagle/correlation/meta/JsonTest.java
+++ b/eagle-core/eagle-alert-parent/eagle-alert/alert-common/src/test/java/org/apache/eagle/correlation/meta/JsonTest.java
@@ -16,8 +16,8 @@
  */
 package org.apache.eagle.correlation.meta;
 
+import com.fasterxml.jackson.databind.ObjectMapper;
 import org.apache.eagle.alert.engine.coordinator.StreamDefinition;
-import org.codehaus.jackson.map.ObjectMapper;
 import org.junit.Test;
 
 /**

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/21187b55/eagle-core/eagle-alert-parent/eagle-alert/alert-coordinator/pom.xml
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-alert-parent/eagle-alert/alert-coordinator/pom.xml b/eagle-core/eagle-alert-parent/eagle-alert/alert-coordinator/pom.xml
index 38abebd..bd06919 100644
--- a/eagle-core/eagle-alert-parent/eagle-alert/alert-coordinator/pom.xml
+++ b/eagle-core/eagle-alert-parent/eagle-alert/alert-coordinator/pom.xml
@@ -49,18 +49,18 @@
             <groupId>com.sun.jersey.contribs</groupId>
             <artifactId>jersey-multipart</artifactId>
         </dependency>
-        <dependency>
-            <groupId>org.codehaus.jackson</groupId>
-            <artifactId>jackson-mapper-asl</artifactId>
-        </dependency>
-        <dependency>
-            <groupId>org.codehaus.jackson</groupId>
-            <artifactId>jackson-jaxrs</artifactId>
-        </dependency>
-        <dependency>
-            <groupId>org.codehaus.jackson</groupId>
-            <artifactId>jackson-xc</artifactId>
-        </dependency>
+        <!--<dependency>-->
+            <!--<groupId>org.codehaus.jackson</groupId>-->
+            <!--<artifactId>jackson-mapper-asl</artifactId>-->
+        <!--</dependency>-->
+        <!--<dependency>-->
+            <!--<groupId>org.codehaus.jackson</groupId>-->
+            <!--<artifactId>jackson-jaxrs</artifactId>-->
+        <!--</dependency>-->
+        <!--<dependency>-->
+            <!--<groupId>org.codehaus.jackson</groupId>-->
+            <!--<artifactId>jackson-xc</artifactId>-->
+        <!--</dependency>-->
         <dependency>
             <groupId>org.slf4j</groupId>
             <artifactId>slf4j-api</artifactId>

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/21187b55/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/pom.xml
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/pom.xml b/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/pom.xml
index e6244ab..d2f889b 100644
--- a/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/pom.xml
+++ b/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/pom.xml
@@ -46,10 +46,10 @@
             <groupId>com.sun.jersey</groupId>
             <artifactId>jersey-client</artifactId>
         </dependency>
-        <dependency>
-            <groupId>org.codehaus.jackson</groupId>
-            <artifactId>jackson-jaxrs</artifactId>
-        </dependency>
+        <!--<dependency>-->
+            <!--<groupId>org.codehaus.jackson</groupId>-->
+            <!--<artifactId>jackson-jaxrs</artifactId>-->
+        <!--</dependency>-->
         <dependency>
             <groupId>com.netflix.archaius</groupId>
             <artifactId>archaius-core</artifactId>

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/21187b55/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/main/java/org/apache/eagle/alert/engine/utils/SerializableUtils.java
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/main/java/org/apache/eagle/alert/engine/utils/SerializableUtils.java b/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/main/java/org/apache/eagle/alert/engine/utils/SerializableUtils.java
index f4652a3..090d61c 100644
--- a/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/main/java/org/apache/eagle/alert/engine/utils/SerializableUtils.java
+++ b/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/main/java/org/apache/eagle/alert/engine/utils/SerializableUtils.java
@@ -16,6 +16,9 @@
  */
 package org.apache.eagle.alert.engine.utils;
 
+import org.xerial.snappy.SnappyInputStream;
+import org.xerial.snappy.SnappyOutputStream;
+
 import java.io.ByteArrayInputStream;
 import java.io.ByteArrayOutputStream;
 import java.io.IOException;
@@ -23,106 +26,102 @@ import java.io.ObjectInputStream;
 import java.io.ObjectOutputStream;
 import java.io.Serializable;
 
-import org.xerial.snappy.SnappyInputStream;
-import org.xerial.snappy.SnappyOutputStream;
 
 /**
  * Utilities for working with Serializables.
- *
- * Derived from "com.google.cloud.dataflow.sdk.util.SerializableUtils":
- * https://github.com/apache/incubator-beam/blob/master/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/util/SerializableUtils.java
+ * Derived from "com.google.cloud.dataflow.sdk.util.SerializableUtils"
  */
 public class SerializableUtils {
-  /**
-   * Serializes the argument into an array of bytes, and returns it.
-   *
-   * @throws IllegalArgumentException if there are errors when serializing
-   */
-  public static byte[] serializeToCompressedByteArray(Object value) {
-    try {
-      ByteArrayOutputStream buffer = new ByteArrayOutputStream();
-      try (ObjectOutputStream oos = new ObjectOutputStream(new SnappyOutputStream(buffer))) {
-        oos.writeObject(value);
-      }
-      return buffer.toByteArray();
-    } catch (IOException exn) {
-      throw new IllegalArgumentException(
-          "unable to serialize " + value,
-          exn);
+    /**
+     * Serializes the argument into an array of bytes, and returns it.
+     *
+     * @throws IllegalArgumentException if there are errors when serializing
+     */
+    public static byte[] serializeToCompressedByteArray(Object value) {
+        try {
+            ByteArrayOutputStream buffer = new ByteArrayOutputStream();
+            try (ObjectOutputStream oos = new ObjectOutputStream(new SnappyOutputStream(buffer))) {
+                oos.writeObject(value);
+            }
+            return buffer.toByteArray();
+        } catch (IOException exn) {
+            throw new IllegalArgumentException(
+                    "unable to serialize " + value,
+                    exn);
+        }
     }
-  }
 
-  /**
-   * Serializes the argument into an array of bytes, and returns it.
-   *
-   * @throws IllegalArgumentException if there are errors when serializing
-   */
-  public static byte[] serializeToByteArray(Object value) {
-    try {
-      ByteArrayOutputStream buffer = new ByteArrayOutputStream();
-      try (ObjectOutputStream oos = new ObjectOutputStream(buffer)) {
-        oos.writeObject(value);
-      }
-      return buffer.toByteArray();
-    } catch (IOException exn) {
-      throw new IllegalArgumentException("unable to serialize " + value, exn);
+    /**
+     * Serializes the argument into an array of bytes, and returns it.
+     *
+     * @throws IllegalArgumentException if there are errors when serializing
+     */
+    public static byte[] serializeToByteArray(Object value) {
+        try {
+            ByteArrayOutputStream buffer = new ByteArrayOutputStream();
+            try (ObjectOutputStream oos = new ObjectOutputStream(buffer)) {
+                oos.writeObject(value);
+            }
+            return buffer.toByteArray();
+        } catch (IOException exn) {
+            throw new IllegalArgumentException("unable to serialize " + value, exn);
+        }
     }
-  }
 
-  /**
-   * Deserializes an object from the given array of bytes, e.g., as
-   * serialized using {@link #serializeToCompressedByteArray}, and returns it.
-   *
-   * @throws IllegalArgumentException if there are errors when
-   * deserializing, using the provided description to identify what
-   * was being deserialized
-   */
-  public static Object deserializeFromByteArray(byte[] encodedValue,
-                                                          String description) {
-    try {
-      try (ObjectInputStream ois = new ObjectInputStream(new ByteArrayInputStream(encodedValue))) {
-        return ois.readObject();
-      }
-    } catch (IOException | ClassNotFoundException exn) {
-      throw new IllegalArgumentException(
-          "unable to deserialize " + description,
-          exn);
+    /**
+     * Deserializes an object from the given array of bytes, e.g., as
+     * serialized using {@link #serializeToCompressedByteArray}, and returns it.
+     *
+     * @throws IllegalArgumentException if there are errors when
+     *                                  deserializing, using the provided description to identify what
+     *                                  was being deserialized
+     */
+    public static Object deserializeFromByteArray(byte[] encodedValue,
+                                                  String description) {
+        try {
+            try (ObjectInputStream ois = new ObjectInputStream(new ByteArrayInputStream(encodedValue))) {
+                return ois.readObject();
+            }
+        } catch (IOException | ClassNotFoundException exn) {
+            throw new IllegalArgumentException(
+                    "unable to deserialize " + description,
+                    exn);
+        }
     }
-  }
 
-  /**
-   * Deserializes an object from the given array of bytes, e.g., as
-   * serialized using {@link #serializeToCompressedByteArray}, and returns it.
-   *
-   * @throws IllegalArgumentException if there are errors when
-   * deserializing, using the provided description to identify what
-   * was being deserialized
-   */
-  public static Object deserializeFromCompressedByteArray(byte[] encodedValue,
-                                                          String description) {
-    try {
-      try (ObjectInputStream ois = new ObjectInputStream(
-          new SnappyInputStream(new ByteArrayInputStream(encodedValue)))) {
-        return ois.readObject();
-      }
-    } catch (IOException | ClassNotFoundException exn) {
-      throw new IllegalArgumentException(
-          "unable to deserialize " + description,
-          exn);
+    /**
+     * Deserializes an object from the given array of bytes, e.g., as
+     * serialized using {@link #serializeToCompressedByteArray}, and returns it.
+     *
+     * @throws IllegalArgumentException if there are errors when
+     *                                  deserializing, using the provided description to identify what
+     *                                  was being deserialized
+     */
+    public static Object deserializeFromCompressedByteArray(byte[] encodedValue,
+                                                            String description) {
+        try {
+            try (ObjectInputStream ois = new ObjectInputStream(
+                    new SnappyInputStream(new ByteArrayInputStream(encodedValue)))) {
+                return ois.readObject();
+            }
+        } catch (IOException | ClassNotFoundException exn) {
+            throw new IllegalArgumentException(
+                    "unable to deserialize " + description,
+                    exn);
+        }
     }
-  }
 
-  public static <T extends Serializable> T ensureSerializable(T value) {
-    @SuppressWarnings("unchecked")
-    T copy = (T) deserializeFromCompressedByteArray(serializeToCompressedByteArray(value),
-        value.toString());
-    return copy;
-  }
+    public static <T extends Serializable> T ensureSerializable(T value) {
+        @SuppressWarnings("unchecked")
+        T copy = (T) deserializeFromCompressedByteArray(serializeToCompressedByteArray(value),
+                value.toString());
+        return copy;
+    }
 
-  public static <T extends Serializable> T clone(T value) {
-    @SuppressWarnings("unchecked")
-    T copy = (T) deserializeFromCompressedByteArray(serializeToCompressedByteArray(value),
-        value.toString());
-    return copy;
-  }
+    public static <T extends Serializable> T clone(T value) {
+        @SuppressWarnings("unchecked")
+        T copy = (T) deserializeFromCompressedByteArray(serializeToCompressedByteArray(value),
+                value.toString());
+        return copy;
+    }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/21187b55/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/test/java/org/apache/eagle/alert/engine/e2e/CoordinatorClient.java
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/test/java/org/apache/eagle/alert/engine/e2e/CoordinatorClient.java b/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/test/java/org/apache/eagle/alert/engine/e2e/CoordinatorClient.java
index 0cf5115..a85a5cd 100644
--- a/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/test/java/org/apache/eagle/alert/engine/e2e/CoordinatorClient.java
+++ b/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/test/java/org/apache/eagle/alert/engine/e2e/CoordinatorClient.java
@@ -20,8 +20,7 @@ import java.io.Closeable;
 import java.io.IOException;
 
 import javax.ws.rs.core.MediaType;
-
-import org.codehaus.jackson.jaxrs.JacksonJsonProvider;
+import com.fasterxml.jackson.jaxrs.json.JacksonJsonProvider;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/21187b55/eagle-core/eagle-alert-parent/eagle-alert/alert-metadata-parent/alert-metadata-service/pom.xml
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-alert-parent/eagle-alert/alert-metadata-parent/alert-metadata-service/pom.xml b/eagle-core/eagle-alert-parent/eagle-alert/alert-metadata-parent/alert-metadata-service/pom.xml
index 9467f1b..2cbef7c 100644
--- a/eagle-core/eagle-alert-parent/eagle-alert/alert-metadata-parent/alert-metadata-service/pom.xml
+++ b/eagle-core/eagle-alert-parent/eagle-alert/alert-metadata-parent/alert-metadata-service/pom.xml
@@ -64,18 +64,18 @@
             <artifactId>jersey-client</artifactId>
         </dependency>
 
-        <dependency>
-            <groupId>org.codehaus.jackson</groupId>
-            <artifactId>jackson-mapper-asl</artifactId>
-        </dependency>
-        <dependency>
-            <groupId>org.codehaus.jackson</groupId>
-            <artifactId>jackson-jaxrs</artifactId>
-        </dependency>
-        <dependency>
-            <groupId>org.codehaus.jackson</groupId>
-            <artifactId>jackson-xc</artifactId>
-        </dependency>
+        <!--<dependency>-->
+            <!--<groupId>org.codehaus.jackson</groupId>-->
+            <!--<artifactId>jackson-mapper-asl</artifactId>-->
+        <!--</dependency>-->
+        <!--<dependency>-->
+            <!--<groupId>org.codehaus.jackson</groupId>-->
+            <!--<artifactId>jackson-jaxrs</artifactId>-->
+        <!--</dependency>-->
+        <!--<dependency>-->
+            <!--<groupId>org.codehaus.jackson</groupId>-->
+            <!--<artifactId>jackson-xc</artifactId>-->
+        <!--</dependency>-->
         <dependency>
             <groupId>com.fasterxml.jackson.core</groupId>
             <artifactId>jackson-databind</artifactId>

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/21187b55/eagle-core/eagle-app/eagle-app-base/src/main/java/org/apache/eagle/app/environment/impl/StormExecutionRuntime.java
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-app/eagle-app-base/src/main/java/org/apache/eagle/app/environment/impl/StormExecutionRuntime.java b/eagle-core/eagle-app/eagle-app-base/src/main/java/org/apache/eagle/app/environment/impl/StormExecutionRuntime.java
index e37e8f2..99dbed8 100644
--- a/eagle-core/eagle-app/eagle-app-base/src/main/java/org/apache/eagle/app/environment/impl/StormExecutionRuntime.java
+++ b/eagle-core/eagle-app/eagle-app-base/src/main/java/org/apache/eagle/app/environment/impl/StormExecutionRuntime.java
@@ -33,6 +33,8 @@ import org.slf4j.LoggerFactory;
 import scala.Int;
 import storm.trident.spout.RichSpoutBatchExecutor;
 
+import java.util.Objects;
+
 public class StormExecutionRuntime implements ExecutionRuntime<StormEnvironment,StormTopology> {
     private static final Logger LOG = LoggerFactory.getLogger(StormExecutionRuntime.class);
     private static LocalCluster _localCluster;
@@ -62,7 +64,7 @@ public class StormExecutionRuntime implements ExecutionRuntime<StormEnvironment,
     private static final String STORM_NIMBUS_THRIFT_CONF_PATH = "application.storm.nimbusThriftPort";
     private static final String WORKERS = "workers";
 
-    public backtype.storm.Config getStormConfig() {
+    private backtype.storm.Config getStormConfig() {
         backtype.storm.Config conf = new backtype.storm.Config();
         conf.put(RichSpoutBatchExecutor.MAX_BATCH_SIZE_CONF, Int.box(64 * 1024));
         conf.put(backtype.storm.Config.TOPOLOGY_RECEIVER_BUFFER_SIZE, Int.box(8));
@@ -129,7 +131,7 @@ public class StormExecutionRuntime implements ExecutionRuntime<StormEnvironment,
     public void stop(Application<StormEnvironment, StormTopology> executor, com.typesafe.config.Config config) {
         String appId = config.getString("appId");
         LOG.info("Stopping topology {} ..." + appId);
-        if (config.getString("mode") == ApplicationEntity.Mode.CLUSTER.name()) {
+        if (Objects.equals(config.getString("mode"), ApplicationEntity.Mode.CLUSTER.name())) {
             Nimbus.Client stormClient = NimbusClient.getConfiguredClient(getStormConfig()).getClient();
             try {
                 stormClient.killTopology(appId);

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/21187b55/eagle-core/eagle-app/eagle-app-base/src/main/java/org/apache/eagle/app/service/impl/ApplicationProviderSPILoader.java
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-app/eagle-app-base/src/main/java/org/apache/eagle/app/service/impl/ApplicationProviderSPILoader.java b/eagle-core/eagle-app/eagle-app-base/src/main/java/org/apache/eagle/app/service/impl/ApplicationProviderSPILoader.java
index 1e1dd28..c7bc4f6 100644
--- a/eagle-core/eagle-app/eagle-app-base/src/main/java/org/apache/eagle/app/service/impl/ApplicationProviderSPILoader.java
+++ b/eagle-core/eagle-app/eagle-app-base/src/main/java/org/apache/eagle/app/service/impl/ApplicationProviderSPILoader.java
@@ -39,11 +39,10 @@ public class ApplicationProviderSPILoader extends ApplicationProviderLoader{
         super(config);
         if(config.hasPath(APPLICATIONS_DIR_PROPS_KEY)) {
             this.appProviderExtDir = config.getString(APPLICATIONS_DIR_PROPS_KEY);
+            LOG.warn("Using {}: {}",APPLICATIONS_DIR_PROPS_KEY,this.appProviderExtDir);
         }else{
             this.appProviderExtDir = null;
         }
-
-        LOG.warn("Using {}: {}",APPLICATIONS_DIR_PROPS_KEY,this.appProviderExtDir);
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/21187b55/eagle-core/eagle-common/src/main/java/org/apache/eagle/common/EagleExceptionWrapper.java
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-common/src/main/java/org/apache/eagle/common/EagleExceptionWrapper.java b/eagle-core/eagle-common/src/main/java/org/apache/eagle/common/EagleExceptionWrapper.java
index 1fc4e85..4ab2b78 100644
--- a/eagle-core/eagle-common/src/main/java/org/apache/eagle/common/EagleExceptionWrapper.java
+++ b/eagle-core/eagle-common/src/main/java/org/apache/eagle/common/EagleExceptionWrapper.java
@@ -14,29 +14,35 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+
 package org.apache.eagle.common;
 
 public class EagleExceptionWrapper {
-	private final static int MAX_DEPTH = 10;
-	
-	public static String wrap(Exception ex){
-		return wrap(ex, EagleExceptionWrapper.MAX_DEPTH);
-	}
-	
-	public static String wrap(Exception ex, int maxdepth){
-		int d = maxdepth;
-		if(d <= 0)
-			d = EagleExceptionWrapper.MAX_DEPTH;
-		int index = 0;
-		StringBuffer sb = new StringBuffer();
-		sb.append(ex);
-		sb.append(System.getProperty("line.separator"));
-		for(StackTraceElement element : ex.getStackTrace()){
-			sb.append(element.toString());
-			sb.append(System.getProperty("line.separator"));
-			if(++index >= d)
-				break;
-		}
-		return sb.toString();
-	}
+    private static final int MAX_DEPTH = 10;
+
+    public static String wrap(Exception ex) {
+        return wrap(ex, EagleExceptionWrapper.MAX_DEPTH);
+    }
+
+    public static String wrap(Exception ex, int maxdepth) {
+        if(ex == null) {
+            return null;
+        }
+        int d = maxdepth;
+        if (d <= 0) {
+            d = EagleExceptionWrapper.MAX_DEPTH;
+        }
+        int index = 0;
+        StringBuffer sb = new StringBuffer();
+        sb.append(ex);
+        sb.append(System.getProperty("line.separator"));
+        for (StackTraceElement element : ex.getStackTrace()) {
+            sb.append(element.toString());
+            sb.append(System.getProperty("line.separator"));
+            if (++index >= d) {
+                break;
+            }
+        }
+        return sb.toString();
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/21187b55/eagle-core/eagle-common/src/main/java/org/apache/eagle/common/config/EagleConfigConstants.java
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-common/src/main/java/org/apache/eagle/common/config/EagleConfigConstants.java b/eagle-core/eagle-common/src/main/java/org/apache/eagle/common/config/EagleConfigConstants.java
index 26d7b49..7829728 100644
--- a/eagle-core/eagle-common/src/main/java/org/apache/eagle/common/config/EagleConfigConstants.java
+++ b/eagle-core/eagle-common/src/main/java/org/apache/eagle/common/config/EagleConfigConstants.java
@@ -17,21 +17,21 @@
 package org.apache.eagle.common.config;
 
 public final class EagleConfigConstants {
-    public final static String SERVICE_ENV = "eagle.service.env";
-    public final static String SERVICE_HOST = "eagle.service.host";
-    public final static String SERVICE_PORT = "eagle.service.port";
-    public final static String SERVICE_HBASE_ZOOKEEPER_QUORUM = "eagle.service.hbase-zookeeper-quorum";
-    public final static String SERVICE_HBASE_ZOOKEEPER_PROPERTY_CLIENTPORT = "eagle.service.hbase-zookeeper-property-clientPort";
-    public final static String SERVICE_ZOOKEEPER_ZNODE_PARENT = "eagle.service.zookeeper-znode-parent";
-    public final static String SERVICE_HBASE_CLIENT_IPC_POOL_SIZE = "eagle.service.hbase-client-ipc-pool-size";
-    public final static String SERVICE_STORAGE_TYPE = "eagle.service.storage-type";
-    public final static String SERVICE_COPROCESSOR_ENABLED = "eagle.service.coprocessor-enabled";
-    public final static String SERVICE_TABLE_NAME_PREFIXED_WITH_ENVIRONMENT = "eagle.service.table-name-prefixed-with-environment";
-    public final static String SERVICE_HBASE_CLIENT_SCAN_CACHE_SIZE = "eagle.service.hbase-client-scan-cache-size";
-    public final static String SERVICE_THREADPOOL_CORE_SIZE = "eagle.service.threadpool-core-size";
-    public final static String SERVICE_THREADPOOL_MAX_SIZE = "eagle.service.threadpool-max-size";
-    public final static String SERVICE_THREADPOOL_SHRINK_SIZE = "eagle.service.threadpool-shrink-size";
-    public final static String SERVICE_AUDITING_ENABLED = "eagle.service.audit-enabled";
+    public final static String SERVICE_ENV = "service.env";
+    public final static String SERVICE_HOST = "service.host";
+    public final static String SERVICE_PORT = "service.port";
+    public final static String SERVICE_HBASE_ZOOKEEPER_QUORUM = "storage.hbase.hbase-zookeeper-quorum";
+    public final static String SERVICE_HBASE_ZOOKEEPER_PROPERTY_CLIENTPORT = "storage.hbase-zookeeper-property-clientPort";
+    public final static String SERVICE_ZOOKEEPER_ZNODE_PARENT = "storage.zookeeper-znode-parent";
+    public final static String SERVICE_HBASE_CLIENT_IPC_POOL_SIZE = "storage.hbase-client-ipc-pool-size";
+    public final static String SERVICE_STORAGE_TYPE = "storage.storage-type";
+    public final static String SERVICE_COPROCESSOR_ENABLED = "storage.coprocessor-enabled";
+    public final static String SERVICE_TABLE_NAME_PREFIXED_WITH_ENVIRONMENT = "storage.table-name-prefixed-with-environment";
+    public final static String SERVICE_HBASE_CLIENT_SCAN_CACHE_SIZE = "storage.hbase-client-scan-cache-size";
+    public final static String SERVICE_THREADPOOL_CORE_SIZE = "storage.threadpool-core-size";
+    public final static String SERVICE_THREADPOOL_MAX_SIZE = "storage.threadpool-max-size";
+    public final static String SERVICE_THREADPOOL_SHRINK_SIZE = "storage.threadpool-shrink-size";
+    public final static String SERVICE_AUDITING_ENABLED = "storage.audit-enabled";
 
     public final static String EAGLE_TIME_ZONE = "eagle.timezone";
     public final static String DEFAULT_EAGLE_TIME_ZONE = "UTC";

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/21187b55/eagle-core/eagle-common/src/test/resources/test-service-config.conf
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-common/src/test/resources/test-service-config.conf b/eagle-core/eagle-common/src/test/resources/test-service-config.conf
index 848391d..c4fcffe 100644
--- a/eagle-core/eagle-common/src/test/resources/test-service-config.conf
+++ b/eagle-core/eagle-common/src/test/resources/test-service-config.conf
@@ -13,18 +13,19 @@
 # See the License for the specific language governing permissions and
 # limitations under the License.
 
-eagle {
-	timezone = "UTC"
 
-	service {
-		env = "test"
-		host = "localhost"
-		port = 9090
-		storage-type = "hbase"
-		table-name-prefixed-with-environment = true
-		hbase-zookeeper-quorum = "localhost"
-		hbase-zookeeper-property-clientPort = 2181
-		zookeeper-znode-parent = "/hbase-unsecure"
-		coprocessor-enabled = false
-	}
+service {
+	timezone = "UTC"
+	env = "test"
+	host = "localhost"
+	port = 9090
 }
+
+storage{
+	storage-type = "hbase"
+	table-name-prefixed-with-environment = true
+	hbase-zookeeper-quorum = "localhost"
+	hbase-zookeeper-property-clientPort = 2181
+	zookeeper-znode-parent = "/hbase-unsecure"
+	coprocessor-enabled = false
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/21187b55/eagle-core/eagle-query/eagle-antlr/src/main/java/org/apache/eagle/query/parser/TokenType.java
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-query/eagle-antlr/src/main/java/org/apache/eagle/query/parser/TokenType.java b/eagle-core/eagle-query/eagle-antlr/src/main/java/org/apache/eagle/query/parser/TokenType.java
index 2707c60..a16e69d 100755
--- a/eagle-core/eagle-query/eagle-antlr/src/main/java/org/apache/eagle/query/parser/TokenType.java
+++ b/eagle-core/eagle-query/eagle-antlr/src/main/java/org/apache/eagle/query/parser/TokenType.java
@@ -17,28 +17,28 @@
 package org.apache.eagle.query.parser;
 
 public enum TokenType {
-	EXP, NUMBER, STRING, LIST, NULL,ID;
+    EXP, NUMBER, STRING, LIST, NULL, ID;
 
-	public static TokenType locate(String tokenText){
-		String _value = tokenText.trim();
-		if (TokenConstant.EXP_PATTERN.matcher(_value).matches()) {
-			// e.q. "value" with EXP{...}
-			return EXP;
-		}else if(TokenConstant.STRING_PATTERN.matcher(_value).matches()){
-			// e.q. "value" with quotes
-			return STRING;
-		}else if(TokenConstant.ARRAY_PATTERN.matcher(_value).matches()){
-			// e.q. (item1,item2,..)
-			return LIST;
-		}else if(TokenConstant.NUMBER_PATTERN.matcher(_value).matches()){
-			// e.q. 1.32 without quotes
-			return NUMBER;
-		}else if(TokenConstant.NULL_PATTERN.matcher(_value).matches()){
-			// e.q. null or NULL without quotes
-			return NULL;
-		} else if(TokenConstant.ID_PATTERN.matcher(_value).matches()){
-			return ID;
-		}
-		throw new IllegalArgumentException("Can't locate type of value text: "+tokenText);
-	}
+    public static TokenType locate(String tokenText) {
+        String _value = tokenText.trim();
+        if (TokenConstant.EXP_PATTERN.matcher(_value).matches()) {
+            // e.q. "value" with EXP{...}
+            return EXP;
+        } else if (TokenConstant.STRING_PATTERN.matcher(_value).matches()) {
+            // e.q. "value" with quotes
+            return STRING;
+        } else if (TokenConstant.ARRAY_PATTERN.matcher(_value).matches()) {
+            // e.q. (item1,item2,..)
+            return LIST;
+        } else if (TokenConstant.NUMBER_PATTERN.matcher(_value).matches()) {
+            // e.q. 1.32 without quotes
+            return NUMBER;
+        } else if (TokenConstant.NULL_PATTERN.matcher(_value).matches()) {
+            // e.q. null or NULL without quotes
+            return NULL;
+        } else if (TokenConstant.ID_PATTERN.matcher(_value).matches()) {
+            return ID;
+        }
+        throw new IllegalArgumentException("Can't locate type of value text: " + tokenText);
+    }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/21187b55/eagle-core/eagle-query/eagle-client-base/pom.xml
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-query/eagle-client-base/pom.xml b/eagle-core/eagle-query/eagle-client-base/pom.xml
index 9b5d177..41a02c5 100644
--- a/eagle-core/eagle-query/eagle-client-base/pom.xml
+++ b/eagle-core/eagle-query/eagle-client-base/pom.xml
@@ -50,16 +50,16 @@
             <scope>test</scope>
         </dependency>
         <dependency>
-            <groupId>org.codehaus.jackson</groupId>
-            <artifactId>jackson-jaxrs</artifactId>
+            <groupId>com.fasterxml.jackson.core</groupId>
+            <artifactId>jackson-databind</artifactId>
         </dependency>
         <dependency>
-            <groupId>org.codehaus.jackson</groupId>
-            <artifactId>jackson-core-asl</artifactId>
+            <groupId>com.fasterxml.jackson.core</groupId>
+            <artifactId>jackson-core</artifactId>
         </dependency>
         <dependency>
-            <groupId>org.codehaus.jackson</groupId>
-            <artifactId>jackson-xc</artifactId>
+            <groupId>com.fasterxml.jackson.jaxrs</groupId>
+            <artifactId>jackson-jaxrs-json-provider</artifactId>
         </dependency>
         <dependency>
             <groupId>com.sun.jersey</groupId>

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/21187b55/eagle-core/eagle-query/eagle-client-base/src/main/java/org/apache/eagle/service/client/impl/EagleServiceBaseClient.java
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-query/eagle-client-base/src/main/java/org/apache/eagle/service/client/impl/EagleServiceBaseClient.java b/eagle-core/eagle-query/eagle-client-base/src/main/java/org/apache/eagle/service/client/impl/EagleServiceBaseClient.java
index 70e92bf..ea5d9a5 100644
--- a/eagle-core/eagle-query/eagle-client-base/src/main/java/org/apache/eagle/service/client/impl/EagleServiceBaseClient.java
+++ b/eagle-core/eagle-query/eagle-client-base/src/main/java/org/apache/eagle/service/client/impl/EagleServiceBaseClient.java
@@ -16,12 +16,11 @@
  */
 package org.apache.eagle.service.client.impl;
 
-import com.sun.jersey.api.client.AsyncWebResource;
-import com.sun.jersey.api.client.Client;
-import com.sun.jersey.api.client.WebResource;
-import com.sun.jersey.api.client.config.ClientConfig;
-import com.sun.jersey.api.client.config.DefaultClientConfig;
-import com.sun.jersey.client.urlconnection.URLConnectionClientHandler;
+import com.fasterxml.jackson.core.JsonFactory;
+import com.fasterxml.jackson.core.JsonGenerationException;
+import com.fasterxml.jackson.databind.JsonMappingException;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.jaxrs.json.JacksonJsonProvider;
 import org.apache.eagle.common.Base64;
 import org.apache.eagle.log.base.taggedlog.TaggedLogAPIEntity;
 import org.apache.eagle.log.entity.GenericServiceAPIResponseEntity;
@@ -31,11 +30,13 @@ import org.apache.eagle.service.client.EagleServiceAsyncClient;
 import org.apache.eagle.service.client.EagleServiceClientException;
 import org.apache.eagle.service.client.IEagleServiceClient;
 import org.apache.eagle.service.client.security.SecurityConstants;
-import org.codehaus.jackson.JsonFactory;
-import org.codehaus.jackson.JsonGenerationException;
-import org.codehaus.jackson.jaxrs.JacksonJsonProvider;
-import org.codehaus.jackson.map.JsonMappingException;
-import org.codehaus.jackson.map.ObjectMapper;
+
+import com.sun.jersey.api.client.AsyncWebResource;
+import com.sun.jersey.api.client.Client;
+import com.sun.jersey.api.client.WebResource;
+import com.sun.jersey.api.client.config.ClientConfig;
+import com.sun.jersey.api.client.config.DefaultClientConfig;
+import com.sun.jersey.client.urlconnection.URLConnectionClientHandler;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -66,7 +67,7 @@ public abstract class EagleServiceBaseClient implements IEagleServiceClient {
 
     private final static Logger LOG = LoggerFactory.getLogger(EagleServiceBaseClient.class);
 
-    protected static final String DEFAULT_BASE_PATH = "/eagle-service/rest";
+    protected static final String DEFAULT_BASE_PATH = "/rest";
     protected static final MediaType DEFAULT_MEDIA_TYPE = MediaType.APPLICATION_JSON_TYPE;
     protected static final String DEFAULT_HTTP_HEADER_CONTENT_TYPE = "application/json";
     protected static final String CONTENT_TYPE = "Content-Type";
@@ -74,7 +75,7 @@ public abstract class EagleServiceBaseClient implements IEagleServiceClient {
     protected final static String GENERIC_ENTITY_PATH = "/entities";
     protected final static String GENERIC_ENTITY_DELETE_PATH = GENERIC_ENTITY_PATH+"/delete";
     private final Client client;
-    private final List<Closeable> closeables = new LinkedList<Closeable>();
+    private final List<Closeable> closeables = new LinkedList<>();
 
     private volatile boolean isStopped = false;
 
@@ -93,7 +94,6 @@ public abstract class EagleServiceBaseClient implements IEagleServiceClient {
         cc.getProperties().put(URLConnectionClientHandler.PROPERTY_HTTP_URL_CONNECTION_SET_METHOD_WORKAROUND, true);
         this.client = Client.create(cc);
         client.addFilter(new com.sun.jersey.api.client.filter.GZIPContentEncodingFilter());
-        //        Runtime.getRuntime().addShutdownHook(new EagleServiceClientShutdownHook(this));
     }
 
     public EagleServiceBaseClient(String host, int port, String basePath){
@@ -104,23 +104,6 @@ public abstract class EagleServiceBaseClient implements IEagleServiceClient {
         this(host, port, DEFAULT_BASE_PATH, username, password);
     }
 
-//    private class EagleServiceClientShutdownHook extends Thread{
-//        final IEagleServiceClient client;
-//        EagleServiceClientShutdownHook(IEagleServiceClient client){
-//            this.client = client;
-//        }
-//
-//        @Override
-//        public void run() {
-//            LOG.info("Client shutdown hook");
-//            try {
-//                this.client.close();
-//            } catch (IOException e) {
-//                LOG.error(e.getMessage(),e);
-//            }
-//        }
-//    }
-
     public Client getJerseyClient(){
         return client;
     }

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/21187b55/eagle-core/eagle-query/eagle-client-base/src/test/java/org/apache/eagle/service/client/TestEagleServiceClientImpl.java
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-query/eagle-client-base/src/test/java/org/apache/eagle/service/client/TestEagleServiceClientImpl.java b/eagle-core/eagle-query/eagle-client-base/src/test/java/org/apache/eagle/service/client/TestEagleServiceClientImpl.java
index 9051ca7..eab33a7 100644
--- a/eagle-core/eagle-query/eagle-client-base/src/test/java/org/apache/eagle/service/client/TestEagleServiceClientImpl.java
+++ b/eagle-core/eagle-query/eagle-client-base/src/test/java/org/apache/eagle/service/client/TestEagleServiceClientImpl.java
@@ -21,29 +21,19 @@ import org.apache.eagle.log.entity.GenericServiceAPIResponseEntity;
 import org.apache.eagle.log.entity.test.TestTimeSeriesAPIEntity;
 import org.apache.eagle.service.client.impl.EagleServiceClientImpl;
 import org.apache.eagle.service.client.impl.ConcurrentSender;
-import org.junit.Assert;
-import org.junit.Test;
+import org.junit.*;
 
 import java.io.IOException;
 import java.util.*;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.Future;
 
+@Ignore("Depends on eagle server, should be used for integration test")
 public class TestEagleServiceClientImpl extends ClientTestBase {
 
-    IEagleServiceClient client = new EagleServiceClientImpl("localhost",38080);
+    private IEagleServiceClient client = new EagleServiceClientImpl("localhost",5050);
 
-    //@Before
-    public void setUp() {
-        hbase.createTable("unittest", "f");
-    }
-
-    //After
-    public void cleanUp() {
-        hbase.deleteTable("unittest");
-    }
-
-    //@Test
+    @Test
     public void testCreateAndSearch() throws IOException, EagleServiceClientException, IllegalAccessException, InstantiationException {
         List<TestTimeSeriesAPIEntity> entities = new ArrayList<TestTimeSeriesAPIEntity>();
 
@@ -101,7 +91,7 @@ public class TestEagleServiceClientImpl extends ClientTestBase {
         return entity;
     }
 
-    //@Test
+    @Test
     public void testUpdate() throws IOException, EagleServiceClientException, IllegalAccessException, InstantiationException {
         List<TestTimeSeriesAPIEntity> entities = new ArrayList<TestTimeSeriesAPIEntity>();
         for(int i=0;i<100;i++){
@@ -139,7 +129,7 @@ public class TestEagleServiceClientImpl extends ClientTestBase {
         assert response.getObj().size() > 0;
     }
 
-    //@Test
+    @Test
     public void testDelete() throws IOException, EagleServiceClientException {
         List<TestTimeSeriesAPIEntity> entities = new ArrayList<TestTimeSeriesAPIEntity>();
         for(int i=0;i<100;i++){
@@ -185,7 +175,7 @@ public class TestEagleServiceClientImpl extends ClientTestBase {
         assert response.isSuccess();
     }
 
-    //@Test
+    @Test
     public void testMetricsSender() throws IOException, EagleServiceClientException {
         List<GenericMetricEntity> entities = new ArrayList<GenericMetricEntity>();
 
@@ -238,13 +228,13 @@ public class TestEagleServiceClientImpl extends ClientTestBase {
                 .pageSize(1000)
                 .send();
         List<Map> aggResult = metricAggResponse.getObj();
-        assert aggResult != null;
-        assert metricAggResponse.isSuccess();
-
+        Assert.assertNotNull(aggResult);
+        Assert.assertTrue(metricAggResponse.isSuccess());
+        Assert.assertTrue(aggResult.size() > 0);
         client.close();
     }
 
-    //@Test
+    @Test
     public void testBatchSender() throws IOException, EagleServiceClientException {
         client.batch(2)
                 .send(newEntity())
@@ -253,7 +243,7 @@ public class TestEagleServiceClientImpl extends ClientTestBase {
         client.close();
     }
 
-    //@Test
+    @Test
     public void testAsyncSender() throws IOException, EagleServiceClientException, ExecutionException, InterruptedException {
         EagleServiceAsyncClient asyncClient = client.async();
 
@@ -281,7 +271,7 @@ public class TestEagleServiceClientImpl extends ClientTestBase {
         client.close();
     }
 
-    //@Test
+    @Test
     public void testParallelSender() throws IOException, EagleServiceClientException, InterruptedException {
         // Case #1:
         ConcurrentSender concurrentSender = client
@@ -310,9 +300,8 @@ public class TestEagleServiceClientImpl extends ClientTestBase {
         client.close();
     }
 
-    //@Test
+    @Test
     public void testSearch() throws EagleServiceClientException, IOException {
-        hbase.createTable("eagle_metric", "f");
 
         GenericServiceAPIResponseEntity<TestTimeSeriesAPIEntity> response =
                 client.search("TestTimeSeriesAPIEntity[]{*}").startTime(0).endTime(System.currentTimeMillis()+1000).pageSize(1000).send();
@@ -328,7 +317,6 @@ public class TestEagleServiceClientImpl extends ClientTestBase {
                 client.search("GenericMetricService[@cluster = \"cluster4ut\" AND @datacenter = \"datacenter4ut\"]{*}").metricName("unit.test.metrics").startTime(0).endTime(System.currentTimeMillis()+1000).pageSize(1000).send();
 
         Assert.assertTrue(response3.isSuccess());
-        hbase.deleteTable("eagle_metric");
     }
 
     @Test

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/21187b55/eagle-core/eagle-query/eagle-entity-base/pom.xml
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-query/eagle-entity-base/pom.xml b/eagle-core/eagle-query/eagle-entity-base/pom.xml
index 3d272e3..017375c 100755
--- a/eagle-core/eagle-query/eagle-entity-base/pom.xml
+++ b/eagle-core/eagle-query/eagle-entity-base/pom.xml
@@ -67,6 +67,14 @@
             </exclusions>
         </dependency>
         <dependency>
+            <groupId>com.fasterxml.jackson.core</groupId>
+            <artifactId>jackson-annotations</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>com.fasterxml.jackson.core</groupId>
+            <artifactId>jackson-databind</artifactId>
+        </dependency>
+        <dependency>
             <groupId>org.apache.eagle</groupId>
             <artifactId>eagle-embed-hbase</artifactId>
             <version>${project.version}</version>

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/21187b55/eagle-core/eagle-query/eagle-entity-base/src/main/java/org/apache/eagle/log/base/taggedlog/EntityJsonModule.java
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-query/eagle-entity-base/src/main/java/org/apache/eagle/log/base/taggedlog/EntityJsonModule.java b/eagle-core/eagle-query/eagle-entity-base/src/main/java/org/apache/eagle/log/base/taggedlog/EntityJsonModule.java
new file mode 100644
index 0000000..fb86fa6
--- /dev/null
+++ b/eagle-core/eagle-query/eagle-entity-base/src/main/java/org/apache/eagle/log/base/taggedlog/EntityJsonModule.java
@@ -0,0 +1,27 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.eagle.log.base.taggedlog;
+
+import com.fasterxml.jackson.databind.module.SimpleModule;
+
+import java.util.Map;
+
+public class EntityJsonModule extends SimpleModule {
+    public EntityJsonModule(){
+        addSerializer(Map.Entry.class,new MapEntrySerializer());
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/21187b55/eagle-core/eagle-query/eagle-entity-base/src/main/java/org/apache/eagle/log/base/taggedlog/MapEntrySerializer.java
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-query/eagle-entity-base/src/main/java/org/apache/eagle/log/base/taggedlog/MapEntrySerializer.java b/eagle-core/eagle-query/eagle-entity-base/src/main/java/org/apache/eagle/log/base/taggedlog/MapEntrySerializer.java
new file mode 100644
index 0000000..4cebbf6
--- /dev/null
+++ b/eagle-core/eagle-query/eagle-entity-base/src/main/java/org/apache/eagle/log/base/taggedlog/MapEntrySerializer.java
@@ -0,0 +1,38 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.eagle.log.base.taggedlog;
+
+import com.fasterxml.jackson.core.JsonGenerator;
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.JsonSerializer;
+import com.fasterxml.jackson.databind.SerializerProvider;
+
+import java.io.IOException;
+import java.util.Map;
+
+public class MapEntrySerializer extends JsonSerializer<Map.Entry> {
+    private static final String KEY_FIELD = "key";
+    private static final String VALUE_FIELD = "value";
+
+    @Override
+    public void serialize(Map.Entry entry, JsonGenerator jsonGenerator, SerializerProvider serializerProvider) throws IOException, JsonProcessingException {
+        jsonGenerator.writeStartObject();
+        jsonGenerator.writeObjectField(KEY_FIELD,entry.getKey());
+        jsonGenerator.writeObjectField(VALUE_FIELD,entry.getValue());
+        jsonGenerator.writeEndObject();
+    }
+}


[05/52] [abbrv] incubator-eagle git commit: [EAGLE-496] fix code style of jpm

Posted by yo...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/0b852cbc/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/JobCounterItem.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/JobCounterItem.java b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/JobCounterItem.java
index 76969a4..001c5ff 100644
--- a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/JobCounterItem.java
+++ b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/JobCounterItem.java
@@ -15,12 +15,12 @@
  * limitations under the License.
  */
 
-package org.apache.eagle.jpm.util.resourceFetch.model;
+package org.apache.eagle.jpm.util.resourcefetch.model;
 
 import org.codehaus.jackson.annotate.JsonIgnoreProperties;
 import org.codehaus.jackson.map.annotate.JsonSerialize;
 
-@JsonSerialize(include=JsonSerialize.Inclusion.NON_NULL)
+@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
 @JsonIgnoreProperties(ignoreUnknown = true)
 public class JobCounterItem {
     private String name;
@@ -31,24 +31,31 @@ public class JobCounterItem {
     public String getName() {
         return name;
     }
+
     public void setName(String name) {
         this.name = name;
     }
+
     public long getTotalCounterValue() {
         return totalCounterValue;
     }
+
     public void setTotalCounterValue(long totalCounterValue) {
         this.totalCounterValue = totalCounterValue;
     }
+
     public long getMapCounterValue() {
         return mapCounterValue;
     }
+
     public void setMapCounterValue(long mapCounterValue) {
         this.mapCounterValue = mapCounterValue;
     }
+
     public long getReduceCounterValue() {
         return reduceCounterValue;
     }
+
     public void setReduceCounterValue(long reduceCounterValue) {
         this.reduceCounterValue = reduceCounterValue;
     }

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/0b852cbc/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/JobCounters.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/JobCounters.java b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/JobCounters.java
index e212ea0..cc0709f 100644
--- a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/JobCounters.java
+++ b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/JobCounters.java
@@ -15,13 +15,14 @@
  * limitations under the License.
  */
 
-package org.apache.eagle.jpm.util.resourceFetch.model;
+package org.apache.eagle.jpm.util.resourcefetch.model;
 
 import org.codehaus.jackson.annotate.JsonIgnoreProperties;
 import org.codehaus.jackson.map.annotate.JsonSerialize;
 
 import java.util.List;
-@JsonSerialize(include=JsonSerialize.Inclusion.NON_NULL)
+
+@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
 @JsonIgnoreProperties(ignoreUnknown = true)
 public class JobCounters {
     private String id;
@@ -30,12 +31,15 @@ public class JobCounters {
     public String getId() {
         return id;
     }
+
     public void setId(String id) {
         this.id = id;
     }
+
     public List<JobCounterGroup> getCounterGroup() {
         return counterGroup;
     }
+
     public void setCounterGroup(List<JobCounterGroup> counterGroup) {
         this.counterGroup = counterGroup;
     }

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/0b852cbc/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/JobCountersWrapper.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/JobCountersWrapper.java b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/JobCountersWrapper.java
index 1178f52..f6bb29c 100644
--- a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/JobCountersWrapper.java
+++ b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/JobCountersWrapper.java
@@ -15,12 +15,12 @@
  * limitations under the License.
  */
 
-package org.apache.eagle.jpm.util.resourceFetch.model;
+package org.apache.eagle.jpm.util.resourcefetch.model;
 
 import org.codehaus.jackson.annotate.JsonIgnoreProperties;
 import org.codehaus.jackson.map.annotate.JsonSerialize;
 
-@JsonSerialize(include=JsonSerialize.Inclusion.NON_NULL)
+@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
 @JsonIgnoreProperties(ignoreUnknown = true)
 public class JobCountersWrapper {
     public JobCounters getJobCounters() {

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/0b852cbc/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/MRJob.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/MRJob.java b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/MRJob.java
index b175152..f12a5d7 100644
--- a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/MRJob.java
+++ b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/MRJob.java
@@ -17,12 +17,12 @@
  *
  */
 
-package org.apache.eagle.jpm.util.resourceFetch.model;
+package org.apache.eagle.jpm.util.resourcefetch.model;
 
 import org.codehaus.jackson.annotate.JsonIgnoreProperties;
 import org.codehaus.jackson.map.annotate.JsonSerialize;
 
-@JsonSerialize(include=JsonSerialize.Inclusion.NON_NULL)
+@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
 @JsonIgnoreProperties(ignoreUnknown = true)
 public class MRJob {
     private long startTime;

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/0b852cbc/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/MRJobsWrapper.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/MRJobsWrapper.java b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/MRJobsWrapper.java
index ec16506..acd316f 100644
--- a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/MRJobsWrapper.java
+++ b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/MRJobsWrapper.java
@@ -17,12 +17,12 @@
  *
  */
 
-package org.apache.eagle.jpm.util.resourceFetch.model;
+package org.apache.eagle.jpm.util.resourcefetch.model;
 
 import org.codehaus.jackson.annotate.JsonIgnoreProperties;
 import org.codehaus.jackson.map.annotate.JsonSerialize;
 
-@JsonSerialize(include=JsonSerialize.Inclusion.NON_NULL)
+@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
 @JsonIgnoreProperties(ignoreUnknown = true)
 public class MRJobsWrapper {
     public MrJobs getJobs() {

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/0b852cbc/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/MRTask.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/MRTask.java b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/MRTask.java
index d6e34ca..4a549de 100644
--- a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/MRTask.java
+++ b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/MRTask.java
@@ -17,12 +17,12 @@
  *
  */
 
-package org.apache.eagle.jpm.util.resourceFetch.model;
+package org.apache.eagle.jpm.util.resourcefetch.model;
 
 import org.codehaus.jackson.annotate.JsonIgnoreProperties;
 import org.codehaus.jackson.map.annotate.JsonSerialize;
 
-@JsonSerialize(include=JsonSerialize.Inclusion.NON_NULL)
+@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
 @JsonIgnoreProperties(ignoreUnknown = true)
 public class MRTask {
     private long startTime;

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/0b852cbc/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/MRTaskAttempt.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/MRTaskAttempt.java b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/MRTaskAttempt.java
index b29e312..218d021 100644
--- a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/MRTaskAttempt.java
+++ b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/MRTaskAttempt.java
@@ -17,12 +17,12 @@
  *
  */
 
-package org.apache.eagle.jpm.util.resourceFetch.model;
+package org.apache.eagle.jpm.util.resourcefetch.model;
 
 import org.codehaus.jackson.annotate.JsonIgnoreProperties;
 import org.codehaus.jackson.map.annotate.JsonSerialize;
 
-@JsonSerialize(include=JsonSerialize.Inclusion.NON_NULL)
+@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
 @JsonIgnoreProperties(ignoreUnknown = true)
 public class MRTaskAttempt {
     private long startTime;

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/0b852cbc/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/MRTaskAttemptWrapper.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/MRTaskAttemptWrapper.java b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/MRTaskAttemptWrapper.java
index fbf3f21..005b73a 100644
--- a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/MRTaskAttemptWrapper.java
+++ b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/MRTaskAttemptWrapper.java
@@ -17,12 +17,12 @@
  *
  */
 
-package org.apache.eagle.jpm.util.resourceFetch.model;
+package org.apache.eagle.jpm.util.resourcefetch.model;
 
 import org.codehaus.jackson.annotate.JsonIgnoreProperties;
 import org.codehaus.jackson.map.annotate.JsonSerialize;
 
-@JsonSerialize(include=JsonSerialize.Inclusion.NON_NULL)
+@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
 @JsonIgnoreProperties(ignoreUnknown = true)
 public class MRTaskAttemptWrapper {
     public MRTaskAttempts getTaskAttempts() {

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/0b852cbc/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/MRTaskAttempts.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/MRTaskAttempts.java b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/MRTaskAttempts.java
index 606ee05..03af0d7 100644
--- a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/MRTaskAttempts.java
+++ b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/MRTaskAttempts.java
@@ -17,14 +17,14 @@
  *
  */
 
-package org.apache.eagle.jpm.util.resourceFetch.model;
+package org.apache.eagle.jpm.util.resourcefetch.model;
 
 import org.codehaus.jackson.annotate.JsonIgnoreProperties;
 import org.codehaus.jackson.map.annotate.JsonSerialize;
 
 import java.util.List;
 
-@JsonSerialize(include=JsonSerialize.Inclusion.NON_NULL)
+@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
 @JsonIgnoreProperties(ignoreUnknown = true)
 public class MRTaskAttempts {
     public List<MRTaskAttempt> getTaskAttempt() {

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/0b852cbc/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/MRTasks.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/MRTasks.java b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/MRTasks.java
index faf46fc..b23b23d 100644
--- a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/MRTasks.java
+++ b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/MRTasks.java
@@ -17,14 +17,14 @@
  *
  */
 
-package org.apache.eagle.jpm.util.resourceFetch.model;
+package org.apache.eagle.jpm.util.resourcefetch.model;
 
 import org.codehaus.jackson.annotate.JsonIgnoreProperties;
 import org.codehaus.jackson.map.annotate.JsonSerialize;
 
 import java.util.List;
 
-@JsonSerialize(include=JsonSerialize.Inclusion.NON_NULL)
+@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
 @JsonIgnoreProperties(ignoreUnknown = true)
 public class MRTasks {
     public List<MRTask> getTask() {

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/0b852cbc/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/MRTasksWrapper.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/MRTasksWrapper.java b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/MRTasksWrapper.java
index c541e89..f92234a 100644
--- a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/MRTasksWrapper.java
+++ b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/MRTasksWrapper.java
@@ -17,12 +17,12 @@
  *
  */
 
-package org.apache.eagle.jpm.util.resourceFetch.model;
+package org.apache.eagle.jpm.util.resourcefetch.model;
 
 import org.codehaus.jackson.annotate.JsonIgnoreProperties;
 import org.codehaus.jackson.map.annotate.JsonSerialize;
 
-@JsonSerialize(include=JsonSerialize.Inclusion.NON_NULL)
+@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
 @JsonIgnoreProperties(ignoreUnknown = true)
 public class MRTasksWrapper {
     public MRTasks getTasks() {

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/0b852cbc/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/MrJobs.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/MrJobs.java b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/MrJobs.java
index 9165550..1d4871c 100644
--- a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/MrJobs.java
+++ b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/MrJobs.java
@@ -17,14 +17,14 @@
  *
  */
 
-package org.apache.eagle.jpm.util.resourceFetch.model;
+package org.apache.eagle.jpm.util.resourcefetch.model;
 
 import org.codehaus.jackson.annotate.JsonIgnoreProperties;
 import org.codehaus.jackson.map.annotate.JsonSerialize;
 
 import java.util.List;
 
-@JsonSerialize(include=JsonSerialize.Inclusion.NON_NULL)
+@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
 @JsonIgnoreProperties(ignoreUnknown = true)
 public class MrJobs {
     public List<MRJob> getJob() {

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/0b852cbc/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/SparkApplication.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/SparkApplication.java b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/SparkApplication.java
index 5d25d84..299934e 100644
--- a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/SparkApplication.java
+++ b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/SparkApplication.java
@@ -17,14 +17,14 @@
  *
  */
 
-package org.apache.eagle.jpm.util.resourceFetch.model;
+package org.apache.eagle.jpm.util.resourcefetch.model;
 
 import org.codehaus.jackson.annotate.JsonIgnoreProperties;
 import org.codehaus.jackson.map.annotate.JsonSerialize;
 
 import java.util.List;
 
-@JsonSerialize(include=JsonSerialize.Inclusion.NON_NULL)
+@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
 @JsonIgnoreProperties(ignoreUnknown = true)
 public class SparkApplication {
     String id;

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/0b852cbc/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/SparkApplicationAttempt.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/SparkApplicationAttempt.java b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/SparkApplicationAttempt.java
index 6e91c03..b437cc2 100644
--- a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/SparkApplicationAttempt.java
+++ b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/SparkApplicationAttempt.java
@@ -17,12 +17,12 @@
  *
  */
 
-package org.apache.eagle.jpm.util.resourceFetch.model;
+package org.apache.eagle.jpm.util.resourcefetch.model;
 
 import org.codehaus.jackson.annotate.JsonIgnoreProperties;
 import org.codehaus.jackson.map.annotate.JsonSerialize;
 
-@JsonSerialize(include=JsonSerialize.Inclusion.NON_NULL)
+@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
 @JsonIgnoreProperties(ignoreUnknown = true)
 public class SparkApplicationAttempt {
     String attemptId;

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/0b852cbc/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/SparkApplicationWrapper.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/SparkApplicationWrapper.java b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/SparkApplicationWrapper.java
index 5508863..0b04f31 100644
--- a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/SparkApplicationWrapper.java
+++ b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/SparkApplicationWrapper.java
@@ -17,12 +17,12 @@
  *
  */
 
-package org.apache.eagle.jpm.util.resourceFetch.model;
+package org.apache.eagle.jpm.util.resourcefetch.model;
 
 import org.codehaus.jackson.annotate.JsonIgnoreProperties;
 import org.codehaus.jackson.map.annotate.JsonSerialize;
 
-@JsonSerialize(include=JsonSerialize.Inclusion.NON_NULL)
+@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
 @JsonIgnoreProperties(ignoreUnknown = true)
 public class SparkApplicationWrapper {
 

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/0b852cbc/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/SparkExecutor.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/SparkExecutor.java b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/SparkExecutor.java
index db84fa1..c2a2115 100644
--- a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/SparkExecutor.java
+++ b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/SparkExecutor.java
@@ -17,12 +17,12 @@
  *
  */
 
-package org.apache.eagle.jpm.util.resourceFetch.model;
+package org.apache.eagle.jpm.util.resourcefetch.model;
 
 import org.codehaus.jackson.annotate.JsonIgnoreProperties;
 import org.codehaus.jackson.map.annotate.JsonSerialize;
 
-@JsonSerialize(include=JsonSerialize.Inclusion.NON_NULL)
+@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
 @JsonIgnoreProperties(ignoreUnknown = true)
 public class SparkExecutor {
     private String id;

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/0b852cbc/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/SparkJob.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/SparkJob.java b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/SparkJob.java
index 018185a..280cf67 100644
--- a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/SparkJob.java
+++ b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/SparkJob.java
@@ -17,14 +17,14 @@
  *
  */
 
-package org.apache.eagle.jpm.util.resourceFetch.model;
+package org.apache.eagle.jpm.util.resourcefetch.model;
 
 import org.codehaus.jackson.annotate.JsonIgnoreProperties;
 import org.codehaus.jackson.map.annotate.JsonSerialize;
 
 import java.util.List;
 
-@JsonSerialize(include=JsonSerialize.Inclusion.NON_NULL)
+@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
 @JsonIgnoreProperties(ignoreUnknown = true)
 public class SparkJob {
     private int jobId;

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/0b852cbc/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/SparkStage.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/SparkStage.java b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/SparkStage.java
index 4b49e9a..14edeee 100644
--- a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/SparkStage.java
+++ b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/SparkStage.java
@@ -17,14 +17,14 @@
  *
  */
 
-package org.apache.eagle.jpm.util.resourceFetch.model;
+package org.apache.eagle.jpm.util.resourcefetch.model;
 
 import org.codehaus.jackson.annotate.JsonIgnoreProperties;
 import org.codehaus.jackson.map.annotate.JsonSerialize;
 
 import java.util.Map;
 
-@JsonSerialize(include=JsonSerialize.Inclusion.NON_NULL)
+@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
 @JsonIgnoreProperties(ignoreUnknown = true)
 public class SparkStage {
     private String status;

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/0b852cbc/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/SparkTask.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/SparkTask.java b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/SparkTask.java
index 6ee665e..a5dd08f 100644
--- a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/SparkTask.java
+++ b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/SparkTask.java
@@ -17,13 +17,13 @@
  *
  */
 
-package org.apache.eagle.jpm.util.resourceFetch.model;
+package org.apache.eagle.jpm.util.resourcefetch.model;
 
 
 import org.codehaus.jackson.annotate.JsonIgnoreProperties;
 import org.codehaus.jackson.map.annotate.JsonSerialize;
 
-@JsonSerialize(include=JsonSerialize.Inclusion.NON_NULL)
+@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
 @JsonIgnoreProperties(ignoreUnknown = true)
 public class SparkTask {
     private int taskId;

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/0b852cbc/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/SparkTaskInputMetrics.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/SparkTaskInputMetrics.java b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/SparkTaskInputMetrics.java
index 6fcb132..d079694 100644
--- a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/SparkTaskInputMetrics.java
+++ b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/SparkTaskInputMetrics.java
@@ -17,12 +17,12 @@
  *
  */
 
-package org.apache.eagle.jpm.util.resourceFetch.model;
+package org.apache.eagle.jpm.util.resourcefetch.model;
 
 import org.codehaus.jackson.annotate.JsonIgnoreProperties;
 import org.codehaus.jackson.map.annotate.JsonSerialize;
 
-@JsonSerialize(include=JsonSerialize.Inclusion.NON_NULL)
+@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
 @JsonIgnoreProperties(ignoreUnknown = true)
 public class SparkTaskInputMetrics {
     private long bytesRead;

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/0b852cbc/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/SparkTaskMetrics.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/SparkTaskMetrics.java b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/SparkTaskMetrics.java
index 4cd7934..cdb913d 100644
--- a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/SparkTaskMetrics.java
+++ b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/SparkTaskMetrics.java
@@ -17,12 +17,12 @@
  *
  */
 
-package org.apache.eagle.jpm.util.resourceFetch.model;
+package org.apache.eagle.jpm.util.resourcefetch.model;
 
 import org.codehaus.jackson.annotate.JsonIgnoreProperties;
 import org.codehaus.jackson.map.annotate.JsonSerialize;
 
-@JsonSerialize(include=JsonSerialize.Inclusion.NON_NULL)
+@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
 @JsonIgnoreProperties(ignoreUnknown = true)
 public class SparkTaskMetrics {
     private long executorDeserializeTime;

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/0b852cbc/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/SparkTaskShuffleReadMetrics.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/SparkTaskShuffleReadMetrics.java b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/SparkTaskShuffleReadMetrics.java
index 457898b..3aa94bb 100644
--- a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/SparkTaskShuffleReadMetrics.java
+++ b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/SparkTaskShuffleReadMetrics.java
@@ -17,12 +17,12 @@
  *
  */
 
-package org.apache.eagle.jpm.util.resourceFetch.model;
+package org.apache.eagle.jpm.util.resourcefetch.model;
 
 import org.codehaus.jackson.annotate.JsonIgnoreProperties;
 import org.codehaus.jackson.map.annotate.JsonSerialize;
 
-@JsonSerialize(include=JsonSerialize.Inclusion.NON_NULL)
+@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
 @JsonIgnoreProperties(ignoreUnknown = true)
 public class SparkTaskShuffleReadMetrics {
     private long remoteBlocksFetched;

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/0b852cbc/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/SparkTaskShuffleWriteMetrics.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/SparkTaskShuffleWriteMetrics.java b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/SparkTaskShuffleWriteMetrics.java
index 00de09e..3d8f025 100644
--- a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/SparkTaskShuffleWriteMetrics.java
+++ b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/SparkTaskShuffleWriteMetrics.java
@@ -17,12 +17,12 @@
  *
  */
 
-package org.apache.eagle.jpm.util.resourceFetch.model;
+package org.apache.eagle.jpm.util.resourcefetch.model;
 
 import org.codehaus.jackson.annotate.JsonIgnoreProperties;
 import org.codehaus.jackson.map.annotate.JsonSerialize;
 
-@JsonSerialize(include=JsonSerialize.Inclusion.NON_NULL)
+@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
 @JsonIgnoreProperties(ignoreUnknown = true)
 public class SparkTaskShuffleWriteMetrics {
     private long bytesWritten;

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/0b852cbc/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/TaskCounterGroup.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/TaskCounterGroup.java b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/TaskCounterGroup.java
index 9561d56..9803d11 100644
--- a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/TaskCounterGroup.java
+++ b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/TaskCounterGroup.java
@@ -15,13 +15,14 @@
  * limitations under the License.
  */
 
-package org.apache.eagle.jpm.util.resourceFetch.model;
+package org.apache.eagle.jpm.util.resourcefetch.model;
 
 import org.codehaus.jackson.annotate.JsonIgnoreProperties;
 import org.codehaus.jackson.map.annotate.JsonSerialize;
 
 import java.util.List;
-@JsonSerialize(include=JsonSerialize.Inclusion.NON_NULL)
+
+@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
 @JsonIgnoreProperties(ignoreUnknown = true)
 public class TaskCounterGroup {
     public String getCounterGroupName() {

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/0b852cbc/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/TaskCounterItem.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/TaskCounterItem.java b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/TaskCounterItem.java
index dee038a..f8ea22c 100644
--- a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/TaskCounterItem.java
+++ b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/TaskCounterItem.java
@@ -15,12 +15,12 @@
  * limitations under the License.
  */
 
-package org.apache.eagle.jpm.util.resourceFetch.model;
+package org.apache.eagle.jpm.util.resourcefetch.model;
 
 import org.codehaus.jackson.annotate.JsonIgnoreProperties;
 import org.codehaus.jackson.map.annotate.JsonSerialize;
 
-@JsonSerialize(include=JsonSerialize.Inclusion.NON_NULL)
+@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
 @JsonIgnoreProperties(ignoreUnknown = true)
 public class TaskCounterItem {
     public String getName() {

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/0b852cbc/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/TaskCounters.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/TaskCounters.java b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/TaskCounters.java
index eade729..ef99547 100644
--- a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/TaskCounters.java
+++ b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/TaskCounters.java
@@ -15,13 +15,14 @@
  * limitations under the License.
  */
 
-package org.apache.eagle.jpm.util.resourceFetch.model;
+package org.apache.eagle.jpm.util.resourcefetch.model;
 
 import org.codehaus.jackson.annotate.JsonIgnoreProperties;
 import org.codehaus.jackson.map.annotate.JsonSerialize;
 
 import java.util.List;
-@JsonSerialize(include=JsonSerialize.Inclusion.NON_NULL)
+
+@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
 @JsonIgnoreProperties(ignoreUnknown = true)
 public class TaskCounters {
     public String getId() {

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/0b852cbc/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/TaskCountersWrapper.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/TaskCountersWrapper.java b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/TaskCountersWrapper.java
index 3624b73..80808a3 100644
--- a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/TaskCountersWrapper.java
+++ b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/model/TaskCountersWrapper.java
@@ -15,12 +15,12 @@
  * limitations under the License.
  */
 
-package org.apache.eagle.jpm.util.resourceFetch.model;
+package org.apache.eagle.jpm.util.resourcefetch.model;
 
 import org.codehaus.jackson.annotate.JsonIgnoreProperties;
 import org.codehaus.jackson.map.annotate.JsonSerialize;
 
-@JsonSerialize(include=JsonSerialize.Inclusion.NON_NULL)
+@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
 @JsonIgnoreProperties(ignoreUnknown = true)
 public class TaskCountersWrapper {
     public TaskCounters getJobTaskCounters() {

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/0b852cbc/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/url/JobListServiceURLBuilderImpl.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/url/JobListServiceURLBuilderImpl.java b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/url/JobListServiceURLBuilderImpl.java
index e3beb9f..5513771 100644
--- a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/url/JobListServiceURLBuilderImpl.java
+++ b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/url/JobListServiceURLBuilderImpl.java
@@ -14,39 +14,39 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.eagle.jpm.util.resourceFetch.url;
+package org.apache.eagle.jpm.util.resourcefetch.url;
 
 import org.apache.eagle.jpm.util.Constants;
 
 public class JobListServiceURLBuilderImpl implements ServiceURLBuilder {
-	
-	public String build(String ... parameters) {
-		/**
-		 * {rmUrl}/ws/v1/cluster/apps?state=RUNNING
-		 * We need to remove tailing slashes to avoid "url//ws/v1"
-		 * because it would not be found and would be redirected to
-		 * history server ui.
-		 */
-		String rmUrl = URLUtil.removeTrailingSlash(parameters[0]);
 
-		String restApi = null;
-		String jobState = parameters[1];
+    public String build(String... parameters) {
+        /**
+         * {rmUrl}/ws/v1/cluster/apps?state=RUNNING.
+         * We need to remove tailing slashes to avoid "url//ws/v1"
+         * because it would not be found and would be redirected to
+         * history server ui.
+         */
+        String rmUrl = URLUtil.removeTrailingSlash(parameters[0]);
 
-		if (jobState.equals(Constants.JobState.RUNNING.name())) {
-			restApi = Constants.V2_APPS_RUNNING_URL;
-		} else if (jobState.equals(Constants.JobState.FINISHED.name())) {
-			restApi = Constants.V2_APPS_COMPLETED_URL;
-		} else if (jobState.equals(Constants.JobState.ALL.name())) {
-			restApi = Constants.V2_APPS_URL;
-		}
-		if (restApi == null) {
-			return null;
-		}
-		// "/ws/v1/cluster/apps?state=RUNNING"
-		StringBuilder sb = new StringBuilder();
-		sb.append(rmUrl).append("/").append(restApi);
-		sb.append("&").append(Constants.ANONYMOUS_PARAMETER);
+        String restApi = null;
+        String jobState = parameters[1];
 
-		return sb.toString();
-	}
+        if (jobState.equals(Constants.JobState.RUNNING.name())) {
+            restApi = Constants.V2_APPS_RUNNING_URL;
+        } else if (jobState.equals(Constants.JobState.FINISHED.name())) {
+            restApi = Constants.V2_APPS_COMPLETED_URL;
+        } else if (jobState.equals(Constants.JobState.ALL.name())) {
+            restApi = Constants.V2_APPS_URL;
+        }
+        if (restApi == null) {
+            return null;
+        }
+        // "/ws/v1/cluster/apps?state=RUNNING"
+        StringBuilder sb = new StringBuilder();
+        sb.append(rmUrl).append("/").append(restApi);
+        sb.append("&").append(Constants.ANONYMOUS_PARAMETER);
+
+        return sb.toString();
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/0b852cbc/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/url/ServiceURLBuilder.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/url/ServiceURLBuilder.java b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/url/ServiceURLBuilder.java
index 597e359..09fea2f 100644
--- a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/url/ServiceURLBuilder.java
+++ b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/url/ServiceURLBuilder.java
@@ -14,8 +14,8 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.eagle.jpm.util.resourceFetch.url;
+package org.apache.eagle.jpm.util.resourcefetch.url;
 
 public interface ServiceURLBuilder {
-	String build(String... parameters);
+    String build(String... parameters);
 }

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/0b852cbc/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/url/SparkCompleteJobServiceURLBuilderImpl.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/url/SparkCompleteJobServiceURLBuilderImpl.java b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/url/SparkCompleteJobServiceURLBuilderImpl.java
index 6339d14..8d959b7 100644
--- a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/url/SparkCompleteJobServiceURLBuilderImpl.java
+++ b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/url/SparkCompleteJobServiceURLBuilderImpl.java
@@ -17,7 +17,7 @@
  *
  */
 
-package org.apache.eagle.jpm.util.resourceFetch.url;
+package org.apache.eagle.jpm.util.resourcefetch.url;
 
 import org.apache.eagle.jpm.util.Constants;
 

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/0b852cbc/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/url/SparkJobServiceURLBuilderImpl.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/url/SparkJobServiceURLBuilderImpl.java b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/url/SparkJobServiceURLBuilderImpl.java
index 009731f..c5ec67a 100644
--- a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/url/SparkJobServiceURLBuilderImpl.java
+++ b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/url/SparkJobServiceURLBuilderImpl.java
@@ -17,7 +17,7 @@
  *
  */
 
-package org.apache.eagle.jpm.util.resourceFetch.url;
+package org.apache.eagle.jpm.util.resourcefetch.url;
 
 import org.apache.eagle.jpm.util.Constants;
 

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/0b852cbc/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/url/URLUtil.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/url/URLUtil.java b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/url/URLUtil.java
index 5ae2e8d..11cde80 100644
--- a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/url/URLUtil.java
+++ b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/url/URLUtil.java
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package org.apache.eagle.jpm.util.resourceFetch.url;
+package org.apache.eagle.jpm.util.resourcefetch.url;
 
 /**
  * URL utils.

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/0b852cbc/eagle-security/eagle-security-hive/src/main/java/org/apache/eagle/security/hive/jobrunning/HiveJobFetchSpout.java
----------------------------------------------------------------------
diff --git a/eagle-security/eagle-security-hive/src/main/java/org/apache/eagle/security/hive/jobrunning/HiveJobFetchSpout.java b/eagle-security/eagle-security-hive/src/main/java/org/apache/eagle/security/hive/jobrunning/HiveJobFetchSpout.java
index 812540e..c0673b3 100644
--- a/eagle-security/eagle-security-hive/src/main/java/org/apache/eagle/security/hive/jobrunning/HiveJobFetchSpout.java
+++ b/eagle-security/eagle-security-hive/src/main/java/org/apache/eagle/security/hive/jobrunning/HiveJobFetchSpout.java
@@ -25,12 +25,12 @@ import backtype.storm.tuple.Fields;
 import org.apache.eagle.dataproc.impl.storm.ValuesArray;
 import org.apache.eagle.jpm.util.*;
 import org.apache.eagle.jpm.util.jobrecover.RunningJobManager;
-import org.apache.eagle.jpm.util.resourceFetch.RMResourceFetcher;
-import org.apache.eagle.jpm.util.resourceFetch.connection.InputStreamUtils;
-import org.apache.eagle.jpm.util.resourceFetch.connection.URLConnectionUtils;
-import org.apache.eagle.jpm.util.resourceFetch.model.AppInfo;
-import org.apache.eagle.jpm.util.resourceFetch.model.MRJob;
-import org.apache.eagle.jpm.util.resourceFetch.model.MRJobsWrapper;
+import org.apache.eagle.jpm.util.resourcefetch.RMResourceFetcher;
+import org.apache.eagle.jpm.util.resourcefetch.connection.InputStreamUtils;
+import org.apache.eagle.jpm.util.resourcefetch.connection.URLConnectionUtils;
+import org.apache.eagle.jpm.util.resourcefetch.model.AppInfo;
+import org.apache.eagle.jpm.util.resourcefetch.model.MRJob;
+import org.apache.eagle.jpm.util.resourcefetch.model.MRJobsWrapper;
 import org.apache.eagle.security.hive.config.RunningJobCrawlConfig;
 import org.codehaus.jackson.JsonParser;
 import org.codehaus.jackson.map.ObjectMapper;



[43/52] [abbrv] incubator-eagle git commit: [Fix] New add file miss apache license header which brake apache:rat build

Posted by yo...@apache.org.
[Fix] New add file miss apache license header which brake apache:rat build

Author: ralphsu


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

Branch: refs/heads/master
Commit: 0a2ce9ed33e0291ad5e187df72756d8c57e2c754
Parents: f81ae06
Author: Ralph, Su <su...@gmail.com>
Authored: Wed Sep 7 15:56:19 2016 +0800
Committer: Ralph, Su <su...@gmail.com>
Committed: Wed Sep 7 15:56:19 2016 +0800

----------------------------------------------------------------------
 .../alert/engine/evaluator/PoilcyExtendedTest.java  | 16 ++++++++++++++++
 1 file changed, 16 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/0a2ce9ed/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/test/java/org/apache/eagle/alert/engine/evaluator/PoilcyExtendedTest.java
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/test/java/org/apache/eagle/alert/engine/evaluator/PoilcyExtendedTest.java b/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/test/java/org/apache/eagle/alert/engine/evaluator/PoilcyExtendedTest.java
index dd75efb..3f653d4 100644
--- a/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/test/java/org/apache/eagle/alert/engine/evaluator/PoilcyExtendedTest.java
+++ b/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/test/java/org/apache/eagle/alert/engine/evaluator/PoilcyExtendedTest.java
@@ -1,3 +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
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
 package org.apache.eagle.alert.engine.evaluator;
 
 import org.apache.eagle.alert.engine.coordinator.PolicyDefinition;


[40/52] [abbrv] incubator-eagle git commit: Update spark history job feeder config & refactor the code

Posted by yo...@apache.org.
Update spark history job feeder config & refactor the code

Author: Qingwen Zhao <qi...@gmail.com>

Closes #416 from qingwen220/sparkHist.


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

Branch: refs/heads/master
Commit: 3110c72e47f697f5c59b5f7d6559d527cf25db3a
Parents: 8774b85
Author: Qingwen Zhao <qi...@gmail.com>
Authored: Wed Sep 7 10:36:24 2016 +0800
Committer: Qingwen Zhao <qi...@gmail.com>
Committed: Wed Sep 7 10:36:24 2016 +0800

----------------------------------------------------------------------
 .../environment/impl/StormExecutionRuntime.java |  32 +-
 .../apache/eagle/jpm/spark/crawl/EventType.java |  24 -
 .../jpm/spark/crawl/JHFInputStreamReader.java   |  24 -
 .../eagle/jpm/spark/crawl/JHFParserBase.java    |  29 -
 .../jpm/spark/crawl/JHFSparkEventReader.java    | 713 -------------------
 .../eagle/jpm/spark/crawl/JHFSparkParser.java   |  73 --
 .../jpm/spark/crawl/SparkApplicationInfo.java   |  69 --
 .../SparkFilesystemInputStreamReaderImpl.java   |  53 --
 .../running/entities/JPMEntityRepository.java   |  33 +
 .../jpm/spark/running/entities/JobConfig.java   |  26 +
 .../spark/running/entities/SparkAppEntity.java  | 476 +++++++++++++
 .../running/entities/SparkExecutorEntity.java   | 233 ++++++
 .../spark/running/entities/SparkJobEntity.java  | 191 +++++
 .../running/entities/SparkStageEntity.java      | 299 ++++++++
 .../spark/running/entities/SparkTaskEntity.java | 290 ++++++++
 .../spark/history/SparkHistoryJobAppConfig.java |   4 -
 .../history/crawl/JHFInputStreamReader.java     |  24 +
 .../jpm/spark/history/crawl/JHFParserBase.java  |  29 +
 .../history/crawl/JHFSparkEventReader.java      | 713 +++++++++++++++++++
 .../jpm/spark/history/crawl/JHFSparkParser.java |  73 ++
 .../history/crawl/SparkApplicationInfo.java     |  69 ++
 .../SparkFilesystemInputStreamReaderImpl.java   |  53 ++
 .../status/JobHistoryZKStateManager.java        |   7 +-
 .../history/storm/SparkHistoryJobParseBolt.java |  13 +-
 .../history/storm/SparkHistoryJobSpout.java     |   5 +-
 ...spark.history.SparkHistoryJobAppProvider.xml |  18 -
 .../src/main/resources/application.conf         |   9 +-
 .../running/entities/JPMEntityRepository.java   |  30 -
 .../jpm/spark/running/entities/JobConfig.java   |  25 -
 .../spark/running/entities/SparkAppEntity.java  | 475 ------------
 .../running/entities/SparkExecutorEntity.java   | 232 ------
 .../spark/running/entities/SparkJobEntity.java  | 190 -----
 .../running/entities/SparkStageEntity.java      | 298 --------
 .../spark/running/entities/SparkTaskEntity.java | 289 --------
 .../running/parser/SparkApplicationParser.java  |   8 +-
 .../src/main/resources/application.conf         |   6 +-
 .../apache/eagle/jpm/util/SparkEventType.java   |  25 +
 .../util/resourcefetch/RMResourceFetcher.java   |   2 +-
 38 files changed, 2575 insertions(+), 2587 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/3110c72e/eagle-core/eagle-app/eagle-app-base/src/main/java/org/apache/eagle/app/environment/impl/StormExecutionRuntime.java
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-app/eagle-app-base/src/main/java/org/apache/eagle/app/environment/impl/StormExecutionRuntime.java b/eagle-core/eagle-app/eagle-app-base/src/main/java/org/apache/eagle/app/environment/impl/StormExecutionRuntime.java
index 04cc19b..e37e8f2 100644
--- a/eagle-core/eagle-app/eagle-app-base/src/main/java/org/apache/eagle/app/environment/impl/StormExecutionRuntime.java
+++ b/eagle-core/eagle-app/eagle-app-base/src/main/java/org/apache/eagle/app/environment/impl/StormExecutionRuntime.java
@@ -34,13 +34,13 @@ import scala.Int;
 import storm.trident.spout.RichSpoutBatchExecutor;
 
 public class StormExecutionRuntime implements ExecutionRuntime<StormEnvironment,StormTopology> {
-    private final static Logger LOG = LoggerFactory.getLogger(StormExecutionRuntime.class);
+    private static final Logger LOG = LoggerFactory.getLogger(StormExecutionRuntime.class);
     private static LocalCluster _localCluster;
 
     private StormEnvironment environment;
 
-    private static LocalCluster getLocalCluster(){
-        if(_localCluster == null){
+    private static LocalCluster getLocalCluster() {
+        if (_localCluster == null) {
             _localCluster = new LocalCluster();
         }
         return _localCluster;
@@ -56,13 +56,13 @@ public class StormExecutionRuntime implements ExecutionRuntime<StormEnvironment,
         return this.environment;
     }
 
-    private final static String STORM_NIMBUS_HOST_CONF_PATH = "application.storm.nimbusHost";
-    private final static String STORM_NIMBUS_HOST_DEFAULT = "localhost";
-    private final static Integer STORM_NIMBUS_THRIFT_DEFAULT = 6627;
-    private final static String STORM_NIMBUS_THRIFT_CONF_PATH = "application.storm.nimbusThriftPort";
+    private static final String STORM_NIMBUS_HOST_CONF_PATH = "application.storm.nimbusHost";
+    private static final String STORM_NIMBUS_HOST_DEFAULT = "localhost";
+    private static final Integer STORM_NIMBUS_THRIFT_DEFAULT = 6627;
+    private static final String STORM_NIMBUS_THRIFT_CONF_PATH = "application.storm.nimbusThriftPort";
     private static final String WORKERS = "workers";
 
-    public backtype.storm.Config getStormConfig(){
+    public backtype.storm.Config getStormConfig() {
         backtype.storm.Config conf = new backtype.storm.Config();
         conf.put(RichSpoutBatchExecutor.MAX_BATCH_SIZE_CONF, Int.box(64 * 1024));
         conf.put(backtype.storm.Config.TOPOLOGY_RECEIVER_BUFFER_SIZE, Int.box(8));
@@ -71,14 +71,14 @@ public class StormExecutionRuntime implements ExecutionRuntime<StormEnvironment,
         conf.put(backtype.storm.Config.TOPOLOGY_EXECUTOR_SEND_BUFFER_SIZE, Int.box(16384));
         conf.put(backtype.storm.Config.NIMBUS_THRIFT_MAX_BUFFER_SIZE, Int.box(20480000));
         String nimbusHost = STORM_NIMBUS_HOST_DEFAULT;
-        if(environment.config().hasPath(STORM_NIMBUS_HOST_CONF_PATH)) {
+        if (environment.config().hasPath(STORM_NIMBUS_HOST_CONF_PATH)) {
             nimbusHost = environment.config().getString(STORM_NIMBUS_HOST_CONF_PATH);
             LOG.info("Overriding {} = {}",STORM_NIMBUS_HOST_CONF_PATH,nimbusHost);
         } else {
             LOG.info("Using default {} = {}",STORM_NIMBUS_HOST_CONF_PATH,STORM_NIMBUS_HOST_DEFAULT);
         }
         Integer nimbusThriftPort =  STORM_NIMBUS_THRIFT_DEFAULT;
-        if(environment.config().hasPath(STORM_NIMBUS_THRIFT_CONF_PATH)) {
+        if (environment.config().hasPath(STORM_NIMBUS_THRIFT_CONF_PATH)) {
             nimbusThriftPort = environment.config().getInt(STORM_NIMBUS_THRIFT_CONF_PATH);
             LOG.info("Overriding {} = {}",STORM_NIMBUS_THRIFT_CONF_PATH,nimbusThriftPort);
         } else {
@@ -94,15 +94,15 @@ public class StormExecutionRuntime implements ExecutionRuntime<StormEnvironment,
     }
 
     @Override
-    public void start(Application<StormEnvironment, StormTopology> executor, com.typesafe.config.Config config){
+    public void start(Application<StormEnvironment, StormTopology> executor, com.typesafe.config.Config config) {
         String topologyName = config.getString("appId");
-        Preconditions.checkNotNull(topologyName,"[appId] is required by null for "+executor.getClass().getCanonicalName());
+        Preconditions.checkNotNull(topologyName,"[appId] is required by null for " + executor.getClass().getCanonicalName());
         StormTopology topology = executor.execute(config, environment);
-        LOG.info("Starting {} ({}), mode: {}",topologyName,executor.getClass().getCanonicalName(), config.getString("mode"));
+        LOG.info("Starting {} ({}), mode: {}",topologyName, executor.getClass().getCanonicalName(), config.getString("mode"));
         Config conf = getStormConfig();
-        if(ApplicationEntity.Mode.CLUSTER.name().equalsIgnoreCase(config.getString("mode"))){
+        if (ApplicationEntity.Mode.CLUSTER.name().equalsIgnoreCase(config.getString("mode"))) {
             String jarFile = config.hasPath("jarPath") ? config.getString("jarPath") : null;
-            if(jarFile == null){
+            if (jarFile == null) {
                 jarFile = DynamicJarPathFinder.findPath(executor.getClass());
             }
             synchronized (StormExecutionRuntime.class) {
@@ -129,7 +129,7 @@ public class StormExecutionRuntime implements ExecutionRuntime<StormEnvironment,
     public void stop(Application<StormEnvironment, StormTopology> executor, com.typesafe.config.Config config) {
         String appId = config.getString("appId");
         LOG.info("Stopping topology {} ..." + appId);
-        if(config.getString("mode") == ApplicationEntity.Mode.CLUSTER.name()){
+        if (config.getString("mode") == ApplicationEntity.Mode.CLUSTER.name()) {
             Nimbus.Client stormClient = NimbusClient.getConfiguredClient(getStormConfig()).getClient();
             try {
                 stormClient.killTopology(appId);

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/3110c72e/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/spark/crawl/EventType.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/spark/crawl/EventType.java b/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/spark/crawl/EventType.java
deleted file mode 100644
index 1ba15b7..0000000
--- a/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/spark/crawl/EventType.java
+++ /dev/null
@@ -1,24 +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.eagle.jpm.spark.crawl;
-
-public enum EventType {
-    SparkListenerBlockManagerAdded, SparkListenerEnvironmentUpdate, SparkListenerApplicationStart,
-    SparkListenerExecutorAdded, SparkListenerJobStart,SparkListenerStageSubmitted, SparkListenerTaskStart,SparkListenerBlockManagerRemoved,
-    SparkListenerTaskEnd, SparkListenerStageCompleted, SparkListenerJobEnd, SparkListenerApplicationEnd,SparkListenerExecutorRemoved
-}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/3110c72e/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/spark/crawl/JHFInputStreamReader.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/spark/crawl/JHFInputStreamReader.java b/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/spark/crawl/JHFInputStreamReader.java
deleted file mode 100644
index 8a8d0db..0000000
--- a/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/spark/crawl/JHFInputStreamReader.java
+++ /dev/null
@@ -1,24 +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.eagle.jpm.spark.crawl;
-
-import java.io.InputStream;
-
-public interface JHFInputStreamReader {
-    void read(InputStream is) throws Exception;
-}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/3110c72e/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/spark/crawl/JHFParserBase.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/spark/crawl/JHFParserBase.java b/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/spark/crawl/JHFParserBase.java
deleted file mode 100644
index 62ba7d9..0000000
--- a/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/spark/crawl/JHFParserBase.java
+++ /dev/null
@@ -1,29 +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.eagle.jpm.spark.crawl;
-
-import java.io.InputStream;
-
-public interface JHFParserBase {
-    /**
-     * this method will ensure to close the inputStream.
-     * @param is
-     * @throws Exception
-     */
-    void parse(InputStream is) throws Exception;
-}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/3110c72e/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/spark/crawl/JHFSparkEventReader.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/spark/crawl/JHFSparkEventReader.java b/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/spark/crawl/JHFSparkEventReader.java
deleted file mode 100644
index 22b715a..0000000
--- a/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/spark/crawl/JHFSparkEventReader.java
+++ /dev/null
@@ -1,713 +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.eagle.jpm.spark.crawl;
-
-import com.typesafe.config.Config;
-import com.typesafe.config.ConfigFactory;
-import org.apache.commons.lang.ArrayUtils;
-import org.apache.eagle.jpm.spark.entity.*;
-import org.apache.eagle.jpm.util.*;
-import org.apache.eagle.log.base.taggedlog.TaggedLogAPIEntity;
-import org.apache.eagle.service.client.EagleServiceClientException;
-import org.apache.eagle.service.client.impl.EagleServiceBaseClient;
-import org.apache.eagle.service.client.impl.EagleServiceClientImpl;
-import org.json.simple.JSONArray;
-import org.json.simple.JSONObject;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-import java.util.*;
-
-public class JHFSparkEventReader {
-    private static final Logger LOG = LoggerFactory.getLogger(JHFSparkEventReader.class);
-
-    private static final int FLUSH_LIMIT = 500;
-    private long firstTaskLaunchTime;
-    private long lastEventTime;
-
-    private Map<String, SparkExecutor> executors;
-    private SparkApp app;
-    private Map<Integer, SparkJob> jobs;
-    private Map<String, SparkStage> stages;
-    private Map<Integer, Set<String>> jobStageMap;
-    private Map<Long, SparkTask> tasks;
-    private EagleServiceClientImpl client;
-    private Map<String, Map<Integer, Boolean>> stageTaskStatusMap;
-
-    private List<TaggedLogAPIEntity> createEntities;
-
-    private Config conf;
-
-    public JHFSparkEventReader(Map<String, String> baseTags, SparkApplicationInfo info) {
-        app = new SparkApp();
-        app.setTags(new HashMap<String, String>(baseTags));
-        app.setYarnState(info.getState());
-        app.setYarnStatus(info.getFinalStatus());
-        createEntities = new ArrayList<>();
-        jobs = new HashMap<Integer, SparkJob>();
-        stages = new HashMap<String, SparkStage>();
-        jobStageMap = new HashMap<Integer, Set<String>>();
-        tasks = new HashMap<Long, SparkTask>();
-        executors = new HashMap<String, SparkExecutor>();
-        stageTaskStatusMap = new HashMap<>();
-        conf = ConfigFactory.load();
-        this.initiateClient();
-    }
-
-    public SparkApp getApp() {
-        return this.app;
-    }
-
-    public void read(JSONObject eventObj) {
-        String eventType = (String) eventObj.get("Event");
-        if (eventType.equalsIgnoreCase(EventType.SparkListenerApplicationStart.toString())) {
-            handleAppStarted(eventObj);
-        } else if (eventType.equalsIgnoreCase(EventType.SparkListenerEnvironmentUpdate.toString())) {
-            handleEnvironmentSet(eventObj);
-        } else if (eventType.equalsIgnoreCase(EventType.SparkListenerExecutorAdded.toString())) {
-            handleExecutorAdd(eventObj);
-        } else if (eventType.equalsIgnoreCase(EventType.SparkListenerBlockManagerAdded.toString())) {
-            handleBlockManagerAdd(eventObj);
-        } else if (eventType.equalsIgnoreCase(EventType.SparkListenerJobStart.toString())) {
-            handleJobStart(eventObj);
-        } else if (eventType.equalsIgnoreCase(EventType.SparkListenerStageSubmitted.toString())) {
-            handleStageSubmit(eventObj);
-        } else if (eventType.equalsIgnoreCase(EventType.SparkListenerTaskStart.toString())) {
-            handleTaskStart(eventObj);
-        } else if (eventType.equalsIgnoreCase(EventType.SparkListenerTaskEnd.toString())) {
-            handleTaskEnd(eventObj);
-        } else if (eventType.equalsIgnoreCase(EventType.SparkListenerStageCompleted.toString())) {
-            handleStageComplete(eventObj);
-        } else if (eventType.equalsIgnoreCase(EventType.SparkListenerJobEnd.toString())) {
-            handleJobEnd(eventObj);
-        } else if (eventType.equalsIgnoreCase(EventType.SparkListenerExecutorRemoved.toString())) {
-            handleExecutorRemoved(eventObj);
-        } else if (eventType.equalsIgnoreCase(EventType.SparkListenerApplicationEnd.toString())) {
-            handleAppEnd(eventObj);
-        } else if (eventType.equalsIgnoreCase(EventType.SparkListenerBlockManagerRemoved.toString())) {
-            //nothing to do now
-        } else {
-            LOG.info("Not registered event type:" + eventType);
-        }
-
-    }
-
-    private void handleEnvironmentSet(JSONObject event) {
-        app.setConfig(new JobConfig());
-        JSONObject sparkProps = (JSONObject) event.get("Spark Properties");
-
-        String[] additionalJobConf = conf.getString("basic.jobConf.additional.info").split(",\\s*");
-        String[] props = {"spark.yarn.app.id", "spark.executor.memory", "spark.driver.host", "spark.driver.port",
-            "spark.driver.memory", "spark.scheduler.pool", "spark.executor.cores", "spark.yarn.am.memory",
-            "spark.yarn.am.cores", "spark.yarn.executor.memoryOverhead", "spark.yarn.driver.memoryOverhead", "spark.yarn.am.memoryOverhead", "spark.master"};
-        String[] jobConf = (String[])ArrayUtils.addAll(additionalJobConf, props);
-        for (String prop : jobConf) {
-            if (sparkProps.containsKey(prop)) {
-                app.getConfig().getConfig().put(prop, (String) sparkProps.get(prop));
-            }
-        }
-    }
-
-    private Object getConfigVal(JobConfig config, String configName, String type) {
-        if (config.getConfig().containsKey(configName)) {
-            Object val = config.getConfig().get(configName);
-            if (type.equalsIgnoreCase(Integer.class.getName())) {
-                return Integer.parseInt((String) val);
-            } else {
-                return val;
-            }
-        } else {
-            if (type.equalsIgnoreCase(Integer.class.getName())) {
-                return conf.getInt("spark.defaultVal." + configName);
-            } else {
-                return conf.getString("spark.defaultVal." + configName);
-            }
-        }
-    }
-
-    private boolean isClientMode(JobConfig config) {
-        return config.getConfig().get("spark.master").equalsIgnoreCase("yarn-client");
-    }
-
-    private void handleAppStarted(JSONObject event) {
-        //need update all entities tag before app start
-        List<TaggedLogAPIEntity> entities = new ArrayList<TaggedLogAPIEntity>();
-        entities.addAll(this.executors.values());
-        entities.add(this.app);
-
-        long appStartTime = JSONUtils.getLong(event, "Timestamp", lastEventTime);
-        for (TaggedLogAPIEntity entity : entities) {
-            entity.getTags().put(SparkJobTagName.SPARK_APP_ID.toString(), JSONUtils.getString(event, "App ID"));
-            entity.getTags().put(SparkJobTagName.SPARK_APP_NAME.toString(), JSONUtils.getString(event, "App Name"));
-            // In yarn-client mode, attemptId is not available in the log, so we set attemptId = 1.
-            String attemptId = isClientMode(this.app.getConfig()) ? "1" : JSONUtils.getString(event, "App Attempt ID");
-            entity.getTags().put(SparkJobTagName.SPARK_APP_ATTEMPT_ID.toString(), attemptId);
-            // the second argument of getNormalizeName() is changed to null because the original code contains sensitive text
-            // original second argument looks like: this.app.getConfig().getConfig().get("xxx"), "xxx" is the sensitive text
-            entity.getTags().put(SparkJobTagName.SPARK_APP_NORM_NAME.toString(), this.getNormalizedName(JSONUtils.getString(event, "App Name"), null));
-            entity.getTags().put(SparkJobTagName.SPARK_USER.toString(), JSONUtils.getString(event, "User"));
-
-            entity.setTimestamp(appStartTime);
-        }
-
-        this.app.setStartTime(appStartTime);
-        this.lastEventTime = appStartTime;
-    }
-
-    private void handleExecutorAdd(JSONObject event) {
-        String executorID = (String) event.get("Executor ID");
-        long executorAddTime = JSONUtils.getLong(event, "Timestamp", lastEventTime);
-        this.lastEventTime = executorAddTime;
-        SparkExecutor executor = this.initiateExecutor(executorID, executorAddTime);
-
-        JSONObject executorInfo = JSONUtils.getJSONObject(event, "Executor Info");
-
-    }
-
-    private void handleBlockManagerAdd(JSONObject event) {
-        long maxMemory = JSONUtils.getLong(event, "Maximum Memory");
-        long timestamp = JSONUtils.getLong(event, "Timestamp", lastEventTime);
-        this.lastEventTime = timestamp;
-        JSONObject blockInfo = JSONUtils.getJSONObject(event, "Block Manager ID");
-        String executorID = JSONUtils.getString(blockInfo, "Executor ID");
-        String hostAndPort = JSONUtils.getString(blockInfo, "Host") + ":" + JSONUtils.getLong(blockInfo, "Port");
-
-        SparkExecutor executor = this.initiateExecutor(executorID, timestamp);
-        executor.setMaxMemory(maxMemory);
-        executor.setHostPort(hostAndPort);
-    }
-
-    private void handleTaskStart(JSONObject event) {
-        this.initializeTask(event);
-    }
-
-    private void handleTaskEnd(JSONObject event) {
-        JSONObject taskInfo = JSONUtils.getJSONObject(event, "Task Info");
-        long taskId = JSONUtils.getLong(taskInfo, "Task ID");
-        SparkTask task = tasks.get(taskId);
-        if (task == null) {
-            return;
-        }
-
-        task.setFailed(JSONUtils.getBoolean(taskInfo, "Failed"));
-        JSONObject taskMetrics = JSONUtils.getJSONObject(event, "Task Metrics");
-        if (null != taskMetrics) {
-            task.setExecutorDeserializeTime(JSONUtils.getLong(taskMetrics, "Executor Deserialize Time", lastEventTime));
-            task.setExecutorRunTime(JSONUtils.getLong(taskMetrics, "Executor Run Time", lastEventTime));
-            task.setJvmGcTime(JSONUtils.getLong(taskMetrics, "JVM GC Time", lastEventTime));
-            task.setResultSize(JSONUtils.getLong(taskMetrics, "Result Size"));
-            task.setResultSerializationTime(JSONUtils.getLong(taskMetrics, "Result Serialization Time", lastEventTime));
-            task.setMemoryBytesSpilled(JSONUtils.getLong(taskMetrics, "Memory Bytes Spilled"));
-            task.setDiskBytesSpilled(JSONUtils.getLong(taskMetrics, "Disk Bytes Spilled"));
-
-            JSONObject inputMetrics = JSONUtils.getJSONObject(taskMetrics, "Input Metrics");
-            if (null != inputMetrics) {
-                task.setInputBytes(JSONUtils.getLong(inputMetrics, "Bytes Read"));
-                task.setInputRecords(JSONUtils.getLong(inputMetrics, "Records Read"));
-            }
-
-            JSONObject outputMetrics = JSONUtils.getJSONObject(taskMetrics, "Output Metrics");
-            if (null != outputMetrics) {
-                task.setOutputBytes(JSONUtils.getLong(outputMetrics, "Bytes Written"));
-                task.setOutputRecords(JSONUtils.getLong(outputMetrics, "Records Written"));
-            }
-
-            JSONObject shuffleWriteMetrics = JSONUtils.getJSONObject(taskMetrics, "Shuffle Write Metrics");
-            if (null != shuffleWriteMetrics) {
-                task.setShuffleWriteBytes(JSONUtils.getLong(shuffleWriteMetrics, "Shuffle Bytes Written"));
-                task.setShuffleWriteRecords(JSONUtils.getLong(shuffleWriteMetrics, "Shuffle Records Written"));
-            }
-
-            JSONObject shuffleReadMetrics = JSONUtils.getJSONObject(taskMetrics, "Shuffle Read Metrics");
-            if (null != shuffleReadMetrics) {
-                task.setShuffleReadLocalBytes(JSONUtils.getLong(shuffleReadMetrics, "Local Bytes Read"));
-                task.setShuffleReadRemoteBytes(JSONUtils.getLong(shuffleReadMetrics, "Remote Bytes Read"));
-                task.setShuffleReadRecords(JSONUtils.getLong(shuffleReadMetrics, "Total Records Read"));
-            }
-        } else {
-            //for tasks success without task metrics, save in the end if no other information
-            if (!task.isFailed()) {
-                return;
-            }
-        }
-
-        aggregateToStage(task);
-        aggregateToExecutor(task);
-        tasks.remove(taskId);
-        this.flushEntities(task, false);
-    }
-
-
-    private SparkTask initializeTask(JSONObject event) {
-        SparkTask task = new SparkTask();
-        task.setTags(new HashMap<>(this.app.getTags()));
-        task.setTimestamp(app.getTimestamp());
-
-        task.getTags().put(SparkJobTagName.SPARK_SATGE_ID.toString(), Long.toString(JSONUtils.getLong(event, "Stage ID")));
-        task.getTags().put(SparkJobTagName.SPARK_STAGE_ATTEMPT_ID.toString(), Long.toString(JSONUtils.getLong(event, "Stage Attempt ID")));
-
-        JSONObject taskInfo = JSONUtils.getJSONObject(event, "Task Info");
-        long taskId = JSONUtils.getLong(taskInfo, "Task ID");
-        task.setTaskId(taskId);
-
-        task.getTags().put(SparkJobTagName.SPARK_TASK_INDEX.toString(), Long.toString(JSONUtils.getLong(taskInfo, "Index")));
-        task.getTags().put(SparkJobTagName.SPARK_TASK_ATTEMPT_ID.toString(), Integer.toString(JSONUtils.getInt(taskInfo, "Attempt")));
-        long launchTime = JSONUtils.getLong(taskInfo, "Launch Time", lastEventTime);
-        this.lastEventTime = launchTime;
-        if (taskId == 0) {
-            this.setFirstTaskLaunchTime(launchTime);
-        }
-        task.setLaunchTime(launchTime);
-        task.setExecutorId(JSONUtils.getString(taskInfo, "Executor ID"));
-        task.setHost(JSONUtils.getString(taskInfo, "Host"));
-        task.setTaskLocality(JSONUtils.getString(taskInfo, "Locality"));
-        task.setSpeculative(JSONUtils.getBoolean(taskInfo, "Speculative"));
-
-        tasks.put(task.getTaskId(), task);
-        return task;
-    }
-
-    private void setFirstTaskLaunchTime(long launchTime) {
-        this.firstTaskLaunchTime = launchTime;
-    }
-
-    private void handleJobStart(JSONObject event) {
-        SparkJob job = new SparkJob();
-        job.setTags(new HashMap<>(this.app.getTags()));
-        job.setTimestamp(app.getTimestamp());
-
-        int jobId = JSONUtils.getInt(event, "Job ID");
-        job.getTags().put(SparkJobTagName.SPARK_JOB_ID.toString(), Integer.toString(jobId));
-        long submissionTime = JSONUtils.getLong(event, "Submission Time", lastEventTime);
-        job.setSubmissionTime(submissionTime);
-        this.lastEventTime = submissionTime;
-
-        //for complete application, no active stages/tasks
-        job.setNumActiveStages(0);
-        job.setNumActiveTasks(0);
-
-        this.jobs.put(jobId, job);
-        this.jobStageMap.put(jobId, new HashSet<String>());
-
-        JSONArray stages = JSONUtils.getJSONArray(event, "Stage Infos");
-        int stagesSize = (stages == null ? 0 : stages.size());
-        job.setNumStages(stagesSize);
-        for (int i = 0; i < stagesSize; i++) {
-            JSONObject stageInfo = (JSONObject) stages.get(i);
-            int stageId = JSONUtils.getInt(stageInfo, "Stage ID");
-            int stageAttemptId = JSONUtils.getInt(stageInfo, "Stage Attempt ID");
-            String stageName = JSONUtils.getString(stageInfo, "Stage Name");
-            int numTasks = JSONUtils.getInt(stageInfo, "Number of Tasks");
-            this.initiateStage(jobId, stageId, stageAttemptId, stageName, numTasks);
-        }
-    }
-
-    private void handleStageSubmit(JSONObject event) {
-        JSONObject stageInfo = JSONUtils.getJSONObject(event, "Stage Info");
-        int stageId = JSONUtils.getInt(stageInfo, "Stage ID");
-        int stageAttemptId = JSONUtils.getInt(stageInfo, "Stage Attempt ID");
-        String key = this.generateStageKey(Integer.toString(stageId), Integer.toString(stageAttemptId));
-        stageTaskStatusMap.put(key, new HashMap<Integer, Boolean>());
-
-        if (!stages.containsKey(key)) {
-            //may be further attempt for one stage
-            String baseAttempt = this.generateStageKey(Integer.toString(stageId), "0");
-            if (stages.containsKey(baseAttempt)) {
-                SparkStage stage = stages.get(baseAttempt);
-                String jobId = stage.getTags().get(SparkJobTagName.SPARK_JOB_ID.toString());
-
-                String stageName = JSONUtils.getString(event, "Stage Name");
-                int numTasks = JSONUtils.getInt(stageInfo, "Number of Tasks");
-                this.initiateStage(Integer.parseInt(jobId), stageId, stageAttemptId, stageName, numTasks);
-            }
-        }
-    }
-
-    private void handleStageComplete(JSONObject event) {
-        JSONObject stageInfo = JSONUtils.getJSONObject(event, "Stage Info");
-        int stageId = JSONUtils.getInt(stageInfo, "Stage ID");
-        int stageAttemptId = JSONUtils.getInt(stageInfo, "Stage Attempt ID");
-        String key = this.generateStageKey(Integer.toString(stageId), Integer.toString(stageAttemptId));
-        SparkStage stage = stages.get(key);
-
-        // If "Submission Time" is not available, use the "Launch Time" of "Task ID" = 0.
-        Long submissionTime = JSONUtils.getLong(stageInfo, "Submission Time", firstTaskLaunchTime);
-
-        stage.setSubmitTime(submissionTime);
-
-        long completeTime = JSONUtils.getLong(stageInfo, "Completion Time", lastEventTime);
-        stage.setCompleteTime(completeTime);
-        this.lastEventTime = completeTime;
-
-        if (stageInfo != null && stageInfo.containsKey("Failure Reason")) {
-            stage.setStatus(SparkEntityConstant.SparkStageStatus.FAILED.toString());
-        } else {
-            stage.setStatus(SparkEntityConstant.SparkStageStatus.COMPLETE.toString());
-        }
-    }
-
-    private void handleExecutorRemoved(JSONObject event) {
-        String executorID = JSONUtils.getString(event, "Executor ID");
-        SparkExecutor executor = executors.get(executorID);
-        long removedTime = JSONUtils.getLong(event, "Timestamp", lastEventTime);
-        executor.setEndTime(removedTime);
-        this.lastEventTime = removedTime;
-    }
-
-    private void handleJobEnd(JSONObject event) {
-        int jobId = JSONUtils.getInt(event, "Job ID");
-        SparkJob job = jobs.get(jobId);
-
-        long completionTime = JSONUtils.getLong(event, "Completion Time", lastEventTime);
-        job.setCompletionTime(completionTime);
-        this.lastEventTime = completionTime;
-
-        JSONObject jobResult = JSONUtils.getJSONObject(event, "Job Result");
-        String result = JSONUtils.getString(jobResult, "Result");
-        if (result.equalsIgnoreCase("JobSucceeded")) {
-            job.setStatus(SparkEntityConstant.SparkJobStatus.SUCCEEDED.toString());
-        } else {
-            job.setStatus(SparkEntityConstant.SparkJobStatus.FAILED.toString());
-        }
-    }
-
-    private void handleAppEnd(JSONObject event) {
-        long endTime = JSONUtils.getLong(event, "Timestamp", lastEventTime);
-        app.setEndTime(endTime);
-        this.lastEventTime = endTime;
-    }
-
-    public void clearReader() throws Exception {
-        //clear tasks
-        for (SparkTask task : tasks.values()) {
-            LOG.info("Task {} does not have result or no task metrics.", task.getTaskId());
-            task.setFailed(true);
-            aggregateToStage(task);
-            aggregateToExecutor(task);
-            this.flushEntities(task, false);
-        }
-
-        List<SparkStage> needStoreStages = new ArrayList<>();
-        for (SparkStage stage : this.stages.values()) {
-            int jobId = Integer.parseInt(stage.getTags().get(SparkJobTagName.SPARK_JOB_ID.toString()));
-            if (stage.getSubmitTime() == 0 || stage.getCompleteTime() == 0) {
-                SparkJob job = this.jobs.get(jobId);
-                job.setNumSkippedStages(job.getNumSkippedStages() + 1);
-                job.setNumSkippedTasks(job.getNumSkippedTasks() + stage.getNumTasks());
-            } else {
-                this.aggregateToJob(stage);
-                this.aggregateStageToApp(stage);
-                needStoreStages.add(stage);
-            }
-            String stageId = stage.getTags().get(SparkJobTagName.SPARK_SATGE_ID.toString());
-            String stageAttemptId = stage.getTags().get(SparkJobTagName.SPARK_STAGE_ATTEMPT_ID.toString());
-            this.jobStageMap.get(jobId).remove(this.generateStageKey(stageId, stageAttemptId));
-        }
-
-        this.flushEntities(needStoreStages, false);
-        for (SparkJob job : jobs.values()) {
-            this.aggregateJobToApp(job);
-        }
-        this.flushEntities(jobs.values(), false);
-
-        app.setExecutors(executors.values().size());
-
-        long executorMemory = Utils.parseMemory((String) this.getConfigVal(this.app.getConfig(), "spark.executor.memory", String.class.getName()));
-        long driverMemory = Utils.parseMemory(this.isClientMode(app.getConfig())
-            ? (String) this.getConfigVal(this.app.getConfig(), "spark.yarn.am.memory", String.class.getName())
-            : (String) this.getConfigVal(app.getConfig(), "spark.driver.memory", String.class.getName()));
-
-        int executorCore = (Integer) this.getConfigVal(app.getConfig(), "spark.executor.cores", Integer.class.getName());
-        int driverCore = this.isClientMode(app.getConfig())
-            ? (Integer) this.getConfigVal(app.getConfig(), "spark.yarn.am.cores", Integer.class.getName())
-            : (Integer) this.getConfigVal(app.getConfig(), "spark.driver.cores", Integer.class.getName());
-
-        long executorMemoryOverhead = this.getMemoryOverhead(app.getConfig(), executorMemory, "spark.yarn.executor.memoryOverhead");
-        long driverMemoryOverhead = this.isClientMode(app.getConfig())
-            ? this.getMemoryOverhead(app.getConfig(), driverMemory, "spark.yarn.am.memoryOverhead")
-            : this.getMemoryOverhead(app.getConfig(), driverMemory, "spark.yarn.driver.memoryOverhead");
-
-        app.setExecMemoryBytes(executorMemory);
-        app.setDriveMemoryBytes(driverMemory);
-        app.setExecutorCores(executorCore);
-        app.setDriverCores(driverCore);
-        app.setExecutorMemoryOverhead(executorMemoryOverhead);
-        app.setDriverMemoryOverhead(driverMemoryOverhead);
-
-        for (SparkExecutor executor : executors.values()) {
-            String executorID = executor.getTags().get(SparkJobTagName.SPARK_EXECUTOR_ID.toString());
-            if (executorID.equalsIgnoreCase("driver")) {
-                executor.setExecMemoryBytes(driverMemory);
-                executor.setCores(driverCore);
-                executor.setMemoryOverhead(driverMemoryOverhead);
-            } else {
-                executor.setExecMemoryBytes(executorMemory);
-                executor.setCores(executorCore);
-                executor.setMemoryOverhead(executorMemoryOverhead);
-            }
-            if (app.getEndTime() <= 0L) {
-                app.setEndTime(this.lastEventTime);
-            }
-            if (executor.getEndTime() <= 0L) {
-                executor.setEndTime(app.getEndTime());
-            }
-            this.aggregateExecutorToApp(executor);
-        }
-        this.flushEntities(executors.values(), false);
-        //spark code...tricky
-        app.setSkippedTasks(app.getCompleteTasks());
-        this.flushEntities(app, true);
-    }
-
-    private long getMemoryOverhead(JobConfig config, long executorMemory, String fieldName) {
-        long result = 0L;
-        String fieldValue = config.getConfig().get(fieldName);
-        if (fieldValue != null) {
-            result = Utils.parseMemory(fieldValue + "m");
-            if (result == 0L) {
-               result = Utils.parseMemory(fieldValue);
-            }
-        }
-
-        if (result == 0L) {
-            result = Math.max(
-                    Utils.parseMemory(conf.getString("spark.defaultVal.spark.yarn.overhead.min")),
-                    executorMemory * conf.getInt("spark.defaultVal." + fieldName + ".factor") / 100);
-        }
-        return result;
-    }
-
-    private void aggregateExecutorToApp(SparkExecutor executor) {
-        long totalExecutorTime = app.getTotalExecutorTime() + executor.getEndTime() - executor.getStartTime();
-        if (totalExecutorTime < 0L) {
-            totalExecutorTime = 0L;
-        }
-        app.setTotalExecutorTime(totalExecutorTime);
-    }
-
-    private void aggregateJobToApp(SparkJob job) {
-        //aggregate job level metrics
-        app.setNumJobs(app.getNumJobs() + 1);
-        app.setTotalTasks(app.getTotalTasks() + job.getNumTask());
-        app.setCompleteTasks(app.getCompleteTasks() + job.getNumCompletedTasks());
-        app.setSkippedTasks(app.getSkippedTasks() + job.getNumSkippedTasks());
-        app.setFailedTasks(app.getFailedTasks() + job.getNumFailedTasks());
-        app.setTotalStages(app.getTotalStages() + job.getNumStages());
-        app.setFailedStages(app.getFailedStages() + job.getNumFailedStages());
-        app.setSkippedStages(app.getSkippedStages() + job.getNumSkippedStages());
-    }
-
-    private void aggregateStageToApp(SparkStage stage) {
-        //aggregate task level metrics
-        app.setDiskBytesSpilled(app.getDiskBytesSpilled() + stage.getDiskBytesSpilled());
-        app.setMemoryBytesSpilled(app.getMemoryBytesSpilled() + stage.getMemoryBytesSpilled());
-        app.setExecutorRunTime(app.getExecutorRunTime() + stage.getExecutorRunTime());
-        app.setJvmGcTime(app.getJvmGcTime() + stage.getJvmGcTime());
-        app.setExecutorDeserializeTime(app.getExecutorDeserializeTime() + stage.getExecutorDeserializeTime());
-        app.setResultSerializationTime(app.getResultSerializationTime() + stage.getResultSerializationTime());
-        app.setResultSize(app.getResultSize() + stage.getResultSize());
-        app.setInputRecords(app.getInputRecords() + stage.getInputRecords());
-        app.setInputBytes(app.getInputBytes() + stage.getInputBytes());
-        app.setOutputRecords(app.getOutputRecords() + stage.getOutputRecords());
-        app.setOutputBytes(app.getOutputBytes() + stage.getOutputBytes());
-        app.setShuffleWriteRecords(app.getShuffleWriteRecords() + stage.getShuffleWriteRecords());
-        app.setShuffleWriteBytes(app.getShuffleWriteBytes() + stage.getShuffleWriteBytes());
-        app.setShuffleReadRecords(app.getShuffleReadRecords() + stage.getShuffleReadRecords());
-        app.setShuffleReadBytes(app.getShuffleReadBytes() + stage.getShuffleReadBytes());
-    }
-
-    private void aggregateToStage(SparkTask task) {
-        String stageId = task.getTags().get(SparkJobTagName.SPARK_SATGE_ID.toString());
-        String stageAttemptId = task.getTags().get(SparkJobTagName.SPARK_STAGE_ATTEMPT_ID.toString());
-        String key = this.generateStageKey(stageId, stageAttemptId);
-        SparkStage stage = stages.get(key);
-
-        stage.setDiskBytesSpilled(stage.getDiskBytesSpilled() + task.getDiskBytesSpilled());
-        stage.setMemoryBytesSpilled(stage.getMemoryBytesSpilled() + task.getMemoryBytesSpilled());
-        stage.setExecutorRunTime(stage.getExecutorRunTime() + task.getExecutorRunTime());
-        stage.setJvmGcTime(stage.getJvmGcTime() + task.getJvmGcTime());
-        stage.setExecutorDeserializeTime(stage.getExecutorDeserializeTime() + task.getExecutorDeserializeTime());
-        stage.setResultSerializationTime(stage.getResultSerializationTime() + task.getResultSerializationTime());
-        stage.setResultSize(stage.getResultSize() + task.getResultSize());
-        stage.setInputRecords(stage.getInputRecords() + task.getInputRecords());
-        stage.setInputBytes(stage.getInputBytes() + task.getInputBytes());
-        stage.setOutputRecords(stage.getOutputRecords() + task.getOutputRecords());
-        stage.setOutputBytes(stage.getOutputBytes() + task.getOutputBytes());
-        stage.setShuffleWriteRecords(stage.getShuffleWriteRecords() + task.getShuffleWriteRecords());
-        stage.setShuffleWriteBytes(stage.getShuffleWriteBytes() + task.getShuffleWriteBytes());
-        stage.setShuffleReadRecords(stage.getShuffleReadRecords() + task.getShuffleReadRecords());
-        long taskShuffleReadBytes = task.getShuffleReadLocalBytes() + task.getShuffleReadRemoteBytes();
-        stage.setShuffleReadBytes(stage.getShuffleReadBytes() + taskShuffleReadBytes);
-
-        boolean success = !task.isFailed();
-
-        Integer taskIndex = Integer.parseInt(task.getTags().get(SparkJobTagName.SPARK_TASK_INDEX.toString()));
-        if (stageTaskStatusMap.get(key).containsKey(taskIndex)) {
-            //has previous task attempt, retrieved from task index in one stage
-            boolean previousResult = stageTaskStatusMap.get(key).get(taskIndex);
-            success = previousResult || success;
-            if (previousResult != success) {
-                stage.setNumFailedTasks(stage.getNumFailedTasks() - 1);
-                stage.setNumCompletedTasks(stage.getNumCompletedTasks() + 1);
-                stageTaskStatusMap.get(key).put(taskIndex, success);
-            }
-        } else {
-            if (success) {
-                stage.setNumCompletedTasks(stage.getNumCompletedTasks() + 1);
-            } else {
-                stage.setNumFailedTasks(stage.getNumFailedTasks() + 1);
-            }
-            stageTaskStatusMap.get(key).put(taskIndex, success);
-        }
-
-    }
-
-    private void aggregateToExecutor(SparkTask task) {
-        String executorId = task.getExecutorId();
-        SparkExecutor executor = executors.get(executorId);
-
-        if (null != executor) {
-            executor.setTotalTasks(executor.getTotalTasks() + 1);
-            if (task.isFailed()) {
-                executor.setFailedTasks(executor.getFailedTasks() + 1);
-            } else {
-                executor.setCompletedTasks(executor.getCompletedTasks() + 1);
-            }
-            long taskShuffleReadBytes = task.getShuffleReadLocalBytes() + task.getShuffleReadRemoteBytes();
-            executor.setTotalShuffleRead(executor.getTotalShuffleRead() + taskShuffleReadBytes);
-            executor.setTotalDuration(executor.getTotalDuration() + task.getExecutorRunTime());
-            executor.setTotalInputBytes(executor.getTotalInputBytes() + task.getInputBytes());
-            executor.setTotalShuffleWrite(executor.getTotalShuffleWrite() + task.getShuffleWriteBytes());
-            executor.setTotalDuration(executor.getTotalDuration() + task.getExecutorRunTime());
-        }
-
-    }
-
-    private void aggregateToJob(SparkStage stage) {
-        int jobId = Integer.parseInt(stage.getTags().get(SparkJobTagName.SPARK_JOB_ID.toString()));
-        SparkJob job = jobs.get(jobId);
-        job.setNumCompletedTasks(job.getNumCompletedTasks() + stage.getNumCompletedTasks());
-        job.setNumFailedTasks(job.getNumFailedTasks() + stage.getNumFailedTasks());
-        job.setNumTask(job.getNumTask() + stage.getNumTasks());
-
-
-        if (stage.getStatus().equalsIgnoreCase(SparkEntityConstant.SparkStageStatus.COMPLETE.toString())) {
-            //if multiple attempts succeed, just count one
-            if (!hasStagePriorAttemptSuccess(stage)) {
-                job.setNumCompletedStages(job.getNumCompletedStages() + 1);
-            }
-        } else {
-            job.setNumFailedStages(job.getNumFailedStages() + 1);
-        }
-    }
-
-    private boolean hasStagePriorAttemptSuccess(SparkStage stage) {
-        int stageAttemptId = Integer.parseInt(stage.getTags().get(SparkJobTagName.SPARK_STAGE_ATTEMPT_ID.toString()));
-        for (int i = 0; i < stageAttemptId; i++) {
-            SparkStage previousStage = stages.get(this.generateStageKey(
-                    stage.getTags().get(SparkJobTagName.SPARK_SATGE_ID.toString()), Integer.toString(i)));
-            if (previousStage.getStatus().equalsIgnoreCase(SparkEntityConstant.SparkStageStatus.COMPLETE.toString())) {
-                return true;
-            }
-        }
-        return false;
-    }
-
-
-    private String generateStageKey(String stageId, String stageAttemptId) {
-        return stageId + "-" + stageAttemptId;
-    }
-
-    private void initiateStage(int jobId, int stageId, int stageAttemptId, String name, int numTasks) {
-        SparkStage stage = new SparkStage();
-        stage.setTags(new HashMap<>(this.app.getTags()));
-        stage.setTimestamp(app.getTimestamp());
-        stage.getTags().put(SparkJobTagName.SPARK_JOB_ID.toString(), Integer.toString(jobId));
-        stage.getTags().put(SparkJobTagName.SPARK_SATGE_ID.toString(), Integer.toString(stageId));
-        stage.getTags().put(SparkJobTagName.SPARK_STAGE_ATTEMPT_ID.toString(), Integer.toString(stageAttemptId));
-        stage.setName(name);
-        stage.setNumActiveTasks(0);
-        stage.setNumTasks(numTasks);
-        stage.setSchedulingPool(this.app.getConfig().getConfig().get("spark.scheduler.pool") == null ?
-                "default" : this.app.getConfig().getConfig().get("spark.scheduler.pool"));
-
-        String stageKey = this.generateStageKey(Integer.toString(stageId), Integer.toString(stageAttemptId));
-        stages.put(stageKey, stage);
-        this.jobStageMap.get(jobId).add(stageKey);
-    }
-
-
-    private SparkExecutor initiateExecutor(String executorID, long startTime) {
-        if (!executors.containsKey(executorID)) {
-            SparkExecutor executor = new SparkExecutor();
-            executor.setTags(new HashMap<>(this.app.getTags()));
-            executor.getTags().put(SparkJobTagName.SPARK_EXECUTOR_ID.toString(), executorID);
-            executor.setStartTime(startTime);
-            executor.setTimestamp(app.getTimestamp());
-
-            this.executors.put(executorID, executor);
-        }
-
-        return this.executors.get(executorID);
-    }
-
-    private String getNormalizedName(String jobName, String assignedName) {
-        if (null != assignedName) {
-            return assignedName;
-        } else {
-            return JobNameNormalization.getInstance().normalize(jobName);
-        }
-    }
-
-    private void flushEntities(Object entity, boolean forceFlush) {
-        this.flushEntities(Collections.singletonList(entity), forceFlush);
-    }
-
-    private void flushEntities(Collection entities, boolean forceFlush) {
-        this.createEntities.addAll(entities);
-
-        if (forceFlush || this.createEntities.size() >= FLUSH_LIMIT) {
-            try {
-                this.doFlush(this.createEntities);
-                this.createEntities.clear();
-            } catch (Exception e) {
-                LOG.error("Fail to flush entities", e);
-            }
-
-        }
-    }
-
-    private EagleServiceBaseClient initiateClient() {
-        String host = conf.getString("eagleProps.eagle.service.host");
-        int port = conf.getInt("eagleProps.eagle.service.port");
-        String userName = conf.getString("eagleProps.eagle.service.username");
-        String pwd = conf.getString("eagleProps.eagle.service.password");
-        client = new EagleServiceClientImpl(host, port, userName, pwd);
-        int timeout = conf.getInt("eagleProps.eagle.service.read.timeout");
-        client.getJerseyClient().setReadTimeout(timeout * 1000);
-
-        return client;
-    }
-
-    private void doFlush(List entities) throws IOException, EagleServiceClientException {
-        client.create(entities);
-        int size = (entities == null ? 0 : entities.size());
-        LOG.info("finish flushing entities of total number " + size);
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/3110c72e/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/spark/crawl/JHFSparkParser.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/spark/crawl/JHFSparkParser.java b/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/spark/crawl/JHFSparkParser.java
deleted file mode 100644
index 02fc5cf..0000000
--- a/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/spark/crawl/JHFSparkParser.java
+++ /dev/null
@@ -1,73 +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.eagle.jpm.spark.crawl;
-
-
-import org.json.simple.JSONObject;
-import org.json.simple.parser.JSONParser;
-import org.json.simple.parser.ParseException;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.BufferedReader;
-import java.io.InputStream;
-import java.io.InputStreamReader;
-
-public class JHFSparkParser implements JHFParserBase {
-
-    private static final Logger logger = LoggerFactory.getLogger(JHFSparkParser.class);
-
-    private boolean isValidJson;
-
-    private JHFSparkEventReader eventReader;
-
-    public JHFSparkParser(JHFSparkEventReader reader) {
-        this.eventReader = reader;
-    }
-
-    @Override
-    public void parse(InputStream is) throws Exception {
-        try (BufferedReader reader = new BufferedReader(new InputStreamReader(is))) {
-            for (String line = reader.readLine(); line != null; line = reader.readLine()) {
-                isValidJson = true;
-                JSONObject eventObj = parseAndValidateJSON(line);
-                if (isValidJson) {
-                    try {
-                        this.eventReader.read(eventObj);
-                    } catch (Exception e) {
-                        e.printStackTrace();
-                    }
-                }
-            }
-
-            this.eventReader.clearReader();
-        }
-    }
-
-    private JSONObject parseAndValidateJSON(String line) {
-        JSONObject eventObj = null;
-        JSONParser parser = new JSONParser();
-        try {
-            eventObj = (JSONObject) parser.parse(line);
-        } catch (ParseException ex) {
-            isValidJson = false;
-            logger.error(String.format("Invalid json string. Fail to parse %s.", line), ex);
-        }
-        return eventObj;
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/3110c72e/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/spark/crawl/SparkApplicationInfo.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/spark/crawl/SparkApplicationInfo.java b/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/spark/crawl/SparkApplicationInfo.java
deleted file mode 100644
index 423d045..0000000
--- a/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/spark/crawl/SparkApplicationInfo.java
+++ /dev/null
@@ -1,69 +0,0 @@
-/*
- *
- *  * Licensed to the Apache Software Foundation (ASF) under one or more
- *  * contributor license agreements.  See the NOTICE file distributed with
- *  * this work for additional information regarding copyright ownership.
- *  * The ASF licenses this file to You under the Apache License, Version 2.0
- *  * (the "License"); you may not use this file except in compliance with
- *  * the License.  You may obtain a copy of the License at
- *  *
- *  *    http://www.apache.org/licenses/LICENSE-2.0
- *  *
- *  * Unless required by applicable law or agreed to in writing, software
- *  * distributed under the License is distributed on an "AS IS" BASIS,
- *  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- *  * See the License for the specific language governing permissions and
- *  * limitations under the License.
- *
- */
-
-package org.apache.eagle.jpm.spark.crawl;
-
-public class SparkApplicationInfo {
-
-    private String state;
-    private String finalStatus;
-    private String queue;
-    private String name;
-    private String user;
-
-    public String getState() {
-        return state;
-    }
-
-    public void setState(String state) {
-        this.state = state;
-    }
-
-    public String getFinalStatus() {
-        return finalStatus;
-    }
-
-    public void setFinalStatus(String finalStatus) {
-        this.finalStatus = finalStatus;
-    }
-
-    public String getQueue() {
-        return queue;
-    }
-
-    public void setQueue(String queue) {
-        this.queue = queue;
-    }
-
-    public String getName() {
-        return name;
-    }
-
-    public void setName(String name) {
-        this.name = name;
-    }
-
-    public String getUser() {
-        return user;
-    }
-
-    public void setUser(String user) {
-        this.user = user;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/3110c72e/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/spark/crawl/SparkFilesystemInputStreamReaderImpl.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/spark/crawl/SparkFilesystemInputStreamReaderImpl.java b/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/spark/crawl/SparkFilesystemInputStreamReaderImpl.java
deleted file mode 100644
index 3964454..0000000
--- a/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/spark/crawl/SparkFilesystemInputStreamReaderImpl.java
+++ /dev/null
@@ -1,53 +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.eagle.jpm.spark.crawl;
-
-import org.apache.eagle.jpm.util.SparkJobTagName;
-
-import java.io.File;
-import java.io.FileInputStream;
-import java.io.InputStream;
-import java.util.HashMap;
-import java.util.Map;
-
-public class SparkFilesystemInputStreamReaderImpl implements JHFInputStreamReader {
-
-    private String site;
-    private SparkApplicationInfo app;
-
-
-    public SparkFilesystemInputStreamReaderImpl(String site, SparkApplicationInfo app) {
-        this.site = site;
-        this.app = app;
-    }
-
-    @Override
-    public void read(InputStream is) throws Exception {
-        Map<String, String> baseTags = new HashMap<>();
-        baseTags.put(SparkJobTagName.SITE.toString(), site);
-        baseTags.put(SparkJobTagName.SPARK_QUEUE.toString(), app.getQueue());
-        JHFParserBase parser = new JHFSparkParser(new JHFSparkEventReader(baseTags, this.app));
-        parser.parse(is);
-    }
-
-    public static void main(String[] args) throws Exception {
-        SparkFilesystemInputStreamReaderImpl impl = new SparkFilesystemInputStreamReaderImpl("apollo-phx", new SparkApplicationInfo());
-        impl.read(new FileInputStream(new File("E:\\eagle\\application_1459803563374_535667_1")));
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/3110c72e/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/spark/running/entities/JPMEntityRepository.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/spark/running/entities/JPMEntityRepository.java b/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/spark/running/entities/JPMEntityRepository.java
new file mode 100644
index 0000000..81f266b
--- /dev/null
+++ b/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/spark/running/entities/JPMEntityRepository.java
@@ -0,0 +1,33 @@
+/*
+ *  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.eagle.jpm.spark.running.entities;
+
+import org.apache.eagle.jpm.mr.runningentity.JobConfigSerDeser;
+import org.apache.eagle.log.entity.repo.EntityRepository;
+
+public class JPMEntityRepository extends EntityRepository {
+    public JPMEntityRepository() {
+        entitySet.add(SparkAppEntity.class);
+        entitySet.add(SparkJobEntity.class);
+        entitySet.add(SparkStageEntity.class);
+        entitySet.add(SparkTaskEntity.class);
+        entitySet.add(SparkExecutorEntity.class);
+        serDeserMap.put(JobConfig.class, new JobConfigSerDeser());
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/3110c72e/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/spark/running/entities/JobConfig.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/spark/running/entities/JobConfig.java b/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/spark/running/entities/JobConfig.java
new file mode 100644
index 0000000..0d3a86f
--- /dev/null
+++ b/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/spark/running/entities/JobConfig.java
@@ -0,0 +1,26 @@
+/*
+ *  Licensed to the Apache Software Foundation (ASF) under one
+ *  or more contributor license agreements.  See the NOTICE file
+ *  distributed with this work for additional information
+ *  regarding copyright ownership.  The ASF licenses this file
+ *  to you under the Apache License, Version 2.0 (the
+ *  "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package org.apache.eagle.jpm.spark.running.entities;
+
+import java.io.Serializable;
+import java.util.HashMap;
+
+public class JobConfig extends HashMap<String, String> implements Serializable {
+}
+

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/3110c72e/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/spark/running/entities/SparkAppEntity.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/spark/running/entities/SparkAppEntity.java b/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/spark/running/entities/SparkAppEntity.java
new file mode 100644
index 0000000..51c8a50
--- /dev/null
+++ b/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/spark/running/entities/SparkAppEntity.java
@@ -0,0 +1,476 @@
+/*
+ *  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.eagle.jpm.spark.running.entities;
+
+import org.apache.eagle.jpm.util.Constants;
+import org.apache.eagle.jpm.util.resourcefetch.model.AppInfo;
+import org.apache.eagle.log.base.taggedlog.TaggedLogAPIEntity;
+import org.apache.eagle.log.entity.meta.*;
+import org.codehaus.jackson.annotate.JsonIgnoreProperties;
+
+@Table("eagleSparkRunningApps")
+@ColumnFamily("f")
+@Prefix("sparkApp")
+@Service(Constants.RUNNING_SPARK_APP_SERVICE_ENDPOINT_NAME)
+@JsonIgnoreProperties(ignoreUnknown = true)
+@TimeSeries(true)
+@Tags({"site","sparkAppId", "sparkAppAttemptId", "sparkAppName", "user", "queue"})
+@Partition({"site"})
+public class SparkAppEntity extends TaggedLogAPIEntity {
+    @Column("a")
+    private long  startTime;
+    @Column("b")
+    private long endTime;
+    @Column("c")
+    private String yarnState;
+    @Column("d")
+    private String yarnStatus;
+    @Column("e")
+    private JobConfig config;
+    @Column("f")
+    private int numJobs;
+    @Column("g")
+    private int totalStages;
+    @Column("h")
+    private int skippedStages;
+    @Column("i")
+    private int failedStages;
+    @Column("j")
+    private int totalTasks;
+    @Column("k")
+    private int skippedTasks;
+    @Column("l")
+    private int failedTasks;
+    @Column("m")
+    private int executors;
+    @Column("n")
+    private long inputBytes;
+    @Column("o")
+    private long inputRecords;
+    @Column("p")
+    private long outputBytes;
+    @Column("q")
+    private long outputRecords;
+    @Column("r")
+    private long shuffleReadBytes;
+    @Column("s")
+    private long shuffleReadRecords;
+    @Column("t")
+    private long shuffleWriteBytes;
+    @Column("u")
+    private long shuffleWriteRecords;
+    @Column("v")
+    private long executorDeserializeTime;
+    @Column("w")
+    private long executorRunTime;
+    @Column("x")
+    private long resultSize;
+    @Column("y")
+    private long jvmGcTime;
+    @Column("z")
+    private long resultSerializationTime;
+    @Column("ab")
+    private long memoryBytesSpilled;
+    @Column("ac")
+    private long diskBytesSpilled;
+    @Column("ad")
+    private long execMemoryBytes;
+    @Column("ae")
+    private long driveMemoryBytes;
+    @Column("af")
+    private int completeTasks;
+    @Column("ag")
+    private long totalExecutorTime;
+    @Column("ah")
+    private long executorMemoryOverhead;
+    @Column("ai")
+    private long driverMemoryOverhead;
+    @Column("aj")
+    private int executorCores;
+    @Column("ak")
+    private int driverCores;
+    @Column("al")
+    private AppInfo appInfo;
+    @Column("am")
+    private int activeStages;
+    @Column("an")
+    private int completeStages;
+    @Column("ba")
+    private int activeTasks;
+
+    public int getActiveTasks() {
+        return activeTasks;
+    }
+
+    public void setActiveTasks(int activeTasks) {
+        this.activeTasks = activeTasks;
+        valueChanged("activeTasks");
+    }
+
+    public int getCompleteStages() {
+        return completeStages;
+    }
+
+    public void setCompleteStages(int completeStages) {
+        this.completeStages = completeStages;
+        valueChanged("completeStages");
+    }
+
+    public int getActiveStages() {
+        return activeStages;
+    }
+
+    public void setActiveStages(int activeStages) {
+        this.activeStages = activeStages;
+        valueChanged("activeStages");
+    }
+
+    public AppInfo getAppInfo() {
+        return appInfo;
+    }
+
+    public void setAppInfo(AppInfo appInfo) {
+        this.appInfo = appInfo;
+        valueChanged("appInfo");
+    }
+
+    public long getStartTime() {
+        return startTime;
+    }
+
+    public long getEndTime() {
+        return endTime;
+    }
+
+    public String getYarnState() {
+        return yarnState;
+    }
+
+    public String getYarnStatus() {
+        return yarnStatus;
+    }
+
+    public int getNumJobs() {
+        return numJobs;
+    }
+
+    public int getTotalStages() {
+        return totalStages;
+    }
+
+    public int getSkippedStages() {
+        return skippedStages;
+    }
+
+    public int getFailedStages() {
+        return failedStages;
+    }
+
+    public int getTotalTasks() {
+        return totalTasks;
+    }
+
+    public int getSkippedTasks() {
+        return skippedTasks;
+    }
+
+    public int getFailedTasks() {
+        return failedTasks;
+    }
+
+    public int getExecutors() {
+        return executors;
+    }
+
+    public long getInputBytes() {
+        return inputBytes;
+    }
+
+    public long getInputRecords() {
+        return inputRecords;
+    }
+
+    public long getOutputBytes() {
+        return outputBytes;
+    }
+
+    public long getOutputRecords() {
+        return outputRecords;
+    }
+
+    public long getShuffleReadBytes() {
+        return shuffleReadBytes;
+    }
+
+    public long getShuffleReadRecords() {
+        return shuffleReadRecords;
+    }
+
+    public long getShuffleWriteBytes() {
+        return shuffleWriteBytes;
+    }
+
+    public long getShuffleWriteRecords() {
+        return shuffleWriteRecords;
+    }
+
+    public long getExecutorDeserializeTime() {
+        return executorDeserializeTime;
+    }
+
+    public long getExecutorRunTime() {
+        return executorRunTime;
+    }
+
+    public long getResultSize() {
+        return resultSize;
+    }
+
+    public long getJvmGcTime() {
+        return jvmGcTime;
+    }
+
+    public long getResultSerializationTime() {
+        return resultSerializationTime;
+    }
+
+    public long getMemoryBytesSpilled() {
+        return memoryBytesSpilled;
+    }
+
+    public long getDiskBytesSpilled() {
+        return diskBytesSpilled;
+    }
+
+    public long getExecMemoryBytes() {
+        return execMemoryBytes;
+    }
+
+    public long getDriveMemoryBytes() {
+        return driveMemoryBytes;
+    }
+
+    public int getCompleteTasks() {
+        return completeTasks;
+    }
+
+    public JobConfig getConfig() {
+        return config;
+    }
+
+    public void setStartTime(long startTime) {
+        this.startTime = startTime;
+        valueChanged("startTime");
+    }
+
+    public void setEndTime(long endTime) {
+        this.endTime = endTime;
+        valueChanged("endTime");
+    }
+
+    public void setYarnState(String yarnState) {
+        this.yarnState = yarnState;
+        valueChanged("yarnState");
+    }
+
+    public void setYarnStatus(String yarnStatus) {
+        this.yarnStatus = yarnStatus;
+        valueChanged("yarnStatus");
+    }
+
+    public void setConfig(JobConfig config) {
+        this.config = config;
+        valueChanged("config");
+    }
+
+    public void setNumJobs(int numJobs) {
+        this.numJobs = numJobs;
+        valueChanged("numJobs");
+    }
+
+    public void setTotalStages(int totalStages) {
+        this.totalStages = totalStages;
+        valueChanged("totalStages");
+    }
+
+    public void setSkippedStages(int skippedStages) {
+        this.skippedStages = skippedStages;
+        valueChanged("skippedStages");
+    }
+
+    public void setFailedStages(int failedStages) {
+        this.failedStages = failedStages;
+        valueChanged("failedStages");
+    }
+
+    public void setTotalTasks(int totalTasks) {
+        this.totalTasks = totalTasks;
+        valueChanged("totalTasks");
+    }
+
+    public void setSkippedTasks(int skippedTasks) {
+        this.skippedTasks = skippedTasks;
+        valueChanged("skippedTasks");
+    }
+
+    public void setFailedTasks(int failedTasks) {
+        this.failedTasks = failedTasks;
+        valueChanged("failedTasks");
+    }
+
+    public void setExecutors(int executors) {
+        this.executors = executors;
+        valueChanged("executors");
+    }
+
+    public void setInputBytes(long inputBytes) {
+        this.inputBytes = inputBytes;
+        valueChanged("inputBytes");
+    }
+
+    public void setInputRecords(long inputRecords) {
+        this.inputRecords = inputRecords;
+        valueChanged("inputRecords");
+    }
+
+    public void setOutputBytes(long outputBytes) {
+        this.outputBytes = outputBytes;
+        valueChanged("outputBytes");
+    }
+
+    public void setOutputRecords(long outputRecords) {
+        this.outputRecords = outputRecords;
+        valueChanged("outputRecords");
+    }
+
+    public void setShuffleReadBytes(long shuffleReadRemoteBytes) {
+        this.shuffleReadBytes = shuffleReadRemoteBytes;
+        valueChanged("shuffleReadBytes");
+    }
+
+    public void setShuffleReadRecords(long shuffleReadRecords) {
+        this.shuffleReadRecords = shuffleReadRecords;
+        valueChanged("shuffleReadRecords");
+    }
+
+    public void setShuffleWriteBytes(long shuffleWriteBytes) {
+        this.shuffleWriteBytes = shuffleWriteBytes;
+        valueChanged("shuffleWriteBytes");
+    }
+
+    public void setShuffleWriteRecords(long shuffleWriteRecords) {
+        this.shuffleWriteRecords = shuffleWriteRecords;
+        valueChanged("shuffleWriteRecords");
+    }
+
+    public void setExecutorDeserializeTime(long executorDeserializeTime) {
+        this.executorDeserializeTime = executorDeserializeTime;
+        valueChanged("executorDeserializeTime");
+    }
+
+    public void setExecutorRunTime(long executorRunTime) {
+        this.executorRunTime = executorRunTime;
+        valueChanged("executorRunTime");
+    }
+
+    public void setResultSize(long resultSize) {
+        this.resultSize = resultSize;
+        valueChanged("resultSize");
+    }
+
+    public void setJvmGcTime(long jvmGcTime) {
+        this.jvmGcTime = jvmGcTime;
+        valueChanged("jvmGcTime");
+    }
+
+    public void setResultSerializationTime(long resultSerializationTime) {
+        this.resultSerializationTime = resultSerializationTime;
+        valueChanged("resultSerializationTime");
+    }
+
+    public void setMemoryBytesSpilled(long memoryBytesSpilled) {
+        this.memoryBytesSpilled = memoryBytesSpilled;
+        valueChanged("memoryBytesSpilled");
+    }
+
+    public void setDiskBytesSpilled(long diskBytesSpilled) {
+        this.diskBytesSpilled = diskBytesSpilled;
+        valueChanged("diskBytesSpilled");
+    }
+
+    public void setExecMemoryBytes(long execMemoryBytes) {
+        this.execMemoryBytes = execMemoryBytes;
+        valueChanged("execMemoryBytes");
+    }
+
+    public void setDriveMemoryBytes(long driveMemoryBytes) {
+        this.driveMemoryBytes = driveMemoryBytes;
+        valueChanged("driveMemoryBytes");
+    }
+
+    public void setCompleteTasks(int completeTasks) {
+        this.completeTasks = completeTasks;
+        valueChanged("completeTasks");
+    }
+
+    public long getTotalExecutorTime() {
+        return totalExecutorTime;
+    }
+
+    public void setTotalExecutorTime(long totalExecutorTime) {
+        this.totalExecutorTime = totalExecutorTime;
+        valueChanged("totalExecutorTime");
+    }
+
+    public long getExecutorMemoryOverhead() {
+        return executorMemoryOverhead;
+    }
+
+    public void setExecutorMemoryOverhead(long executorMemoryOverhead) {
+        this.executorMemoryOverhead = executorMemoryOverhead;
+        valueChanged("executorMemoryOverhead");
+    }
+
+    public long getDriverMemoryOverhead() {
+        return driverMemoryOverhead;
+    }
+
+    public void setDriverMemoryOverhead(long driverMemoryOverhead) {
+        this.driverMemoryOverhead = driverMemoryOverhead;
+        valueChanged("driverMemoryOverhead");
+    }
+
+    public int getExecutorCores() {
+        return executorCores;
+    }
+
+    public void setExecutorCores(int executorCores) {
+        this.executorCores = executorCores;
+        valueChanged("executorCores");
+    }
+
+    public int getDriverCores() {
+        return driverCores;
+    }
+
+    public void setDriverCores(int driverCores) {
+        this.driverCores = driverCores;
+        valueChanged("driverCores");
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/3110c72e/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/spark/running/entities/SparkExecutorEntity.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/spark/running/entities/SparkExecutorEntity.java b/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/spark/running/entities/SparkExecutorEntity.java
new file mode 100644
index 0000000..6d0441c
--- /dev/null
+++ b/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/spark/running/entities/SparkExecutorEntity.java
@@ -0,0 +1,233 @@
+/*
+ *  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.eagle.jpm.spark.running.entities;
+
+import org.apache.eagle.jpm.util.Constants;
+import org.apache.eagle.log.base.taggedlog.TaggedLogAPIEntity;
+import org.apache.eagle.log.entity.meta.*;
+import org.codehaus.jackson.annotate.JsonIgnoreProperties;
+
+@Table("eagleSparkRunningExecutors")
+@ColumnFamily("f")
+@Prefix("sparkExecutor")
+@Service(Constants.RUNNING_SPARK_EXECUTOR_SERVICE_ENDPOINT_NAME)
+@JsonIgnoreProperties(ignoreUnknown = true)
+@TimeSeries(true)
+@Tags({"site","sparkAppId", "sparkAppAttemptId", "sparkAppName", "executorId","user", "queue"})
+@Partition({"site"})
+public class SparkExecutorEntity extends TaggedLogAPIEntity {
+    @Column("a")
+    private String hostPort;
+    @Column("b")
+    private int rddBlocks;
+    @Column("c")
+    private long memoryUsed;
+    @Column("d")
+    private long diskUsed;
+    @Column("e")
+    private int activeTasks = 0;
+    @Column("f")
+    private int failedTasks = 0;
+    @Column("g")
+    private int completedTasks = 0;
+    @Column("h")
+    private int totalTasks = 0;
+    @Column("i")
+    private long totalDuration = 0;
+    @Column("j")
+    private long totalInputBytes = 0;
+    @Column("k")
+    private long totalShuffleRead = 0;
+    @Column("l")
+    private long totalShuffleWrite = 0;
+    @Column("m")
+    private long maxMemory;
+    @Column("n")
+    private long startTime;
+    @Column("o")
+    private long endTime = 0;
+    @Column("p")
+    private long execMemoryBytes;
+    @Column("q")
+    private int cores;
+    @Column("r")
+    private long memoryOverhead;
+
+    public String getHostPort() {
+        return hostPort;
+    }
+
+    public void setHostPort(String hostPort) {
+        this.hostPort = hostPort;
+        this.valueChanged("hostPort");
+    }
+
+    public int getRddBlocks() {
+        return rddBlocks;
+    }
+
+    public void setRddBlocks(int rddBlocks) {
+        this.rddBlocks = rddBlocks;
+        this.valueChanged("rddBlocks");
+    }
+
+    public long getMemoryUsed() {
+        return memoryUsed;
+    }
+
+    public void setMemoryUsed(long memoryUsed) {
+        this.memoryUsed = memoryUsed;
+        this.valueChanged("memoryUsed");
+    }
+
+    public long getDiskUsed() {
+        return diskUsed;
+    }
+
+    public void setDiskUsed(long diskUsed) {
+        this.diskUsed = diskUsed;
+        this.valueChanged("diskUsed");
+    }
+
+    public int getActiveTasks() {
+        return activeTasks;
+    }
+
+    public void setActiveTasks(int activeTasks) {
+        this.activeTasks = activeTasks;
+        this.valueChanged("activeTasks");
+    }
+
+    public int getFailedTasks() {
+        return failedTasks;
+    }
+
+    public void setFailedTasks(int failedTasks) {
+        this.failedTasks = failedTasks;
+        this.valueChanged("failedTasks");
+    }
+
+    public int getCompletedTasks() {
+        return completedTasks;
+    }
+
+    public void setCompletedTasks(int completedTasks) {
+        this.completedTasks = completedTasks;
+        this.valueChanged("completedTasks");
+    }
+
+    public int getTotalTasks() {
+        return totalTasks;
+    }
+
+    public void setTotalTasks(int totalTasks) {
+        this.totalTasks = totalTasks;
+        this.valueChanged("totalTasks");
+    }
+
+    public long getTotalDuration() {
+        return totalDuration;
+    }
+
+    public void setTotalDuration(long totalDuration) {
+        this.totalDuration = totalDuration;
+        this.valueChanged("totalDuration");
+    }
+
+    public long getTotalInputBytes() {
+        return totalInputBytes;
+    }
+
+    public void setTotalInputBytes(long totalInputBytes) {
+        this.totalInputBytes = totalInputBytes;
+        this.valueChanged("totalInputBytes");
+    }
+
+    public long getTotalShuffleRead() {
+        return totalShuffleRead;
+    }
+
+    public void setTotalShuffleRead(long totalShuffleRead) {
+        this.totalShuffleRead = totalShuffleRead;
+        this.valueChanged("totalShuffleRead");
+    }
+
+    public long getTotalShuffleWrite() {
+        return totalShuffleWrite;
+    }
+
+    public void setTotalShuffleWrite(long totalShuffleWrite) {
+        this.totalShuffleWrite = totalShuffleWrite;
+        this.valueChanged("totalShuffleWrite");
+    }
+
+    public long getMaxMemory() {
+        return maxMemory;
+    }
+
+    public void setMaxMemory(long maxMemory) {
+        this.maxMemory = maxMemory;
+        this.valueChanged("maxMemory");
+    }
+
+    public long getStartTime() {
+        return startTime;
+    }
+
+    public void setStartTime(long startTime) {
+        this.startTime = startTime;
+        valueChanged("startTime");
+    }
+
+    public long getEndTime() {
+        return endTime;
+    }
+
+    public void setEndTime(long endTime) {
+        this.endTime = endTime;
+        this.valueChanged("endTime");
+    }
+
+    public long getExecMemoryBytes() {
+        return execMemoryBytes;
+    }
+
+    public void setExecMemoryBytes(long execMemoryBytes) {
+        this.execMemoryBytes = execMemoryBytes;
+        this.valueChanged("execMemoryBytes");
+    }
+
+    public int getCores() {
+        return cores;
+    }
+
+    public void setCores(int cores) {
+        this.cores = cores;
+        valueChanged("cores");
+    }
+
+    public long getMemoryOverhead() {
+        return memoryOverhead;
+    }
+
+    public void setMemoryOverhead(long memoryOverhead) {
+        this.memoryOverhead = memoryOverhead;
+        valueChanged("memoryOverhead");
+    }
+}


[49/52] [abbrv] incubator-eagle git commit: [EAGLE-520] Fix and decouple co-processor from eagle aggreation query service

Posted by yo...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/21187b55/eagle-core/eagle-query/eagle-storage-hbase/src/main/java/org/apache/eagle/storage/hbase/query/coprocessor/AggregateProtocolEndPoint.java
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-query/eagle-storage-hbase/src/main/java/org/apache/eagle/storage/hbase/query/coprocessor/AggregateProtocolEndPoint.java b/eagle-core/eagle-query/eagle-storage-hbase/src/main/java/org/apache/eagle/storage/hbase/query/coprocessor/AggregateProtocolEndPoint.java
index 53d27de..7ef8b80 100755
--- a/eagle-core/eagle-query/eagle-storage-hbase/src/main/java/org/apache/eagle/storage/hbase/query/coprocessor/AggregateProtocolEndPoint.java
+++ b/eagle-core/eagle-query/eagle-storage-hbase/src/main/java/org/apache/eagle/storage/hbase/query/coprocessor/AggregateProtocolEndPoint.java
@@ -16,6 +16,7 @@
  */
 package org.apache.eagle.storage.hbase.query.coprocessor;
 
+import org.apache.eagle.common.DateTimeUtil;
 import org.apache.eagle.log.base.taggedlog.TaggedLogAPIEntity;
 import org.apache.eagle.log.entity.*;
 import org.apache.eagle.log.entity.meta.EntityDefinition;
@@ -25,7 +26,7 @@ import org.apache.eagle.query.aggregate.raw.GroupbyKeyValue;
 import org.apache.eagle.query.aggregate.raw.RawAggregator;
 import org.apache.eagle.query.aggregate.timeseries.TimeSeriesAggregator;
 import org.apache.eagle.storage.hbase.query.coprocessor.generated.AggregateProtos;
-import org.apache.eagle.common.DateTimeUtil;
+
 import com.google.protobuf.RpcCallback;
 import com.google.protobuf.RpcController;
 import com.google.protobuf.Service;
@@ -49,8 +50,8 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 
-//public abstract class AbstractAggregateEndPoint extends BaseEndpointCoprocessor{
 public class AggregateProtocolEndPoint extends AggregateProtos.AggregateProtocol implements AggregateProtocol, Coprocessor, CoprocessorService {
+    private static final Logger LOG = LoggerFactory.getLogger(AggregateProtocolEndPoint.class);
 
     private RegionCoprocessorEnvironment env;
 
@@ -59,16 +60,19 @@ public class AggregateProtocolEndPoint extends AggregateProtos.AggregateProtocol
         return this;
     }
 
-    public AggregateProtocolEndPoint() {}
+    public AggregateProtocolEndPoint() {
+    }
 
-    protected void checkNotNull(Object obj,String name) {
-		if(obj==null) throw new NullPointerException(name+" is null");
-	}
+    protected void checkNotNull(Object obj, String name) {
+        if (obj == null) {
+            throw new NullPointerException(name + " is null");
+        }
+    }
 
     @Override
     public void start(CoprocessorEnvironment env) throws IOException {
         if (env instanceof RegionCoprocessorEnvironment) {
-            this.env = (RegionCoprocessorEnvironment)env;
+            this.env = (RegionCoprocessorEnvironment) env;
         } else {
             throw new CoprocessorException("Must be loaded on a table region!");
         }
@@ -79,302 +83,296 @@ public class AggregateProtocolEndPoint extends AggregateProtos.AggregateProtocol
         // do nothing
     }
 
-//    @Override
-//	public ProtocolSignature getProtocolSignature(String protocol, long version, int clientMethodsHashCode) throws IOException {
-//		if (AggregateProtocol.class.getName().equals(protocol)) {
-////			return new ProtocolSignature(AggregateProtocol.VERSION, null);
-//			return new ProtocolSignature(98l, null);
-//		}
-//		throw new IOException("Unknown protocol: " + protocol);
-//	}
-
-	protected HRegion getCurrentRegion(){
-		return this.env.getRegion();
-	}
-
-	/**
-	 * <pre>
-	 * Region-unittest,\x82\xB4\x85\xC2\x7F\xFF\xFE\xB6\xC9jNG\xEE!\x5C3\xBB\xAE\xA1:\x05\xA5\xA9x\xB0\xA1"8\x05\xFB(\xD2VY\xDB\x9A\x06\x09\xA9\x98\xC2\xE3\x8D=,1413960230654.aaf2a6c9f2c87c196f43497243bb2424.
-	 * RegionID-unittest,1413960230654
-	 * </pre>
-	 */
-	protected String getLogHeader(){
-		HRegion region = this.getCurrentRegion();
-		return LOG.isDebugEnabled() ? String.format("Region-%s",region.getRegionNameAsString()):
-				String.format("Region-%s,%d",region.getTableDesc().getNameAsString(),region.getRegionId());
-	}
-
-	protected class InternalReadReport {
-		public InternalReadReport(long counter,long startTimestamp,long stopTimestamp){
-			this.counter = counter;
-			this.startTimestamp = startTimestamp;
-			this.stopTimestamp = stopTimestamp;
-		}
-		public long getCounter() {
-			return counter;
-		}
-		public void setCounter(long counter) {
-			this.counter = counter;
-		}
-
-		public long getStartTimestamp() {
-			return startTimestamp;
-		}
-
-		public void setStartTimestamp(long startTimestamp) {
-			this.startTimestamp = startTimestamp;
-		}
-
-		public long getStopTimestamp() {
-			return stopTimestamp;
-		}
-
-		public void setStopTimestamp(long stopTimestamp) {
-			this.stopTimestamp = stopTimestamp;
-		}
-
-		private long counter;
-		private long startTimestamp;
-		private long stopTimestamp;
-	}
-
-	/**
-	 * Asynchronous HBase scan read as entity
-	 *
-	 * @param scan
-	 * @throws java.io.IOException
-	 */
-	protected InternalReadReport asyncStreamRead(EntityDefinition ed, Scan scan, EntityCreationListener listener) throws IOException {
-//		_init();
-		long counter = 0;
-		long startTimestamp = 0;
-		long stopTimestamp = 0;
-
-		InternalScanner scanner = this.getCurrentRegion().getScanner(scan);
-		List<Cell> results = new ArrayList<Cell>();
-		try{
-			boolean hasMoreRows;
-			GenericMetricShadowEntity singleMetricEntity = null;
-			do{
-				hasMoreRows = scanner.next(results);
-				Map<String, byte[]> kvMap = new HashMap<String, byte[]>();
-				if(!results.isEmpty()){
-					counter ++;
-					byte[] row = results.get(0).getRow();
-					long timestamp = RowkeyBuilder.getTimestamp(row, ed);
-					
-					// Min
-					if(startTimestamp == 0 || startTimestamp > timestamp ){
-						startTimestamp = timestamp;
-					}
-					
-					// Max
-					if(stopTimestamp == 0 || stopTimestamp < timestamp ){
-						stopTimestamp = timestamp;
-					}
-					
-					for(Cell kv:results){
-						String qualifierName = Bytes.toString(kv.getQualifier());
-//						Qualifier qualifier = null;
-//						if(!ed.isTag(qualifierName)){
-//							qualifier = ed.getQualifierNameMap().get(qualifierName);
-//							if(qualifier == null){
-//								LOG.error("qualifier for   " + qualifierName + " not exist");
-//								throw new NullPointerException("qualifier for field "+qualifierName+" not exist");
-//							}
-//						}
-						if(kv.getValue()!=null) kvMap.put(qualifierName ,kv.getValue());
-					}
-					
-					// LOG.info("DEBUG: timestamp="+timestamp+", keys=["+StringUtils.join(kvMap.keySet(),",")+"]");
-					
-					InternalLog internalLog = HBaseInternalLogHelper.buildObject(ed, row, timestamp, kvMap);
-					if(internalLog!=null){
-						TaggedLogAPIEntity logAPIEntity = null;
-						try {
-							logAPIEntity = HBaseInternalLogHelper.buildEntity(internalLog, ed);
-							if(logAPIEntity instanceof GenericMetricEntity){
-								if(singleMetricEntity == null) singleMetricEntity = new GenericMetricShadowEntity();
-								GenericMetricEntity e = (GenericMetricEntity)logAPIEntity;
-								if(e.getValue()!=null) {
-									int count = e.getValue().length;
-									@SuppressWarnings("unused")
-									Class<?> cls = ed.getMetricDefinition().getSingleTimestampEntityClass();
-									for (int i = 0; i < count; i++) {
-										long ts = logAPIEntity.getTimestamp() + i * ed.getMetricDefinition().getInterval();
-										// exclude those entity which is not within the time range in search condition. [start, end)
-										singleMetricEntity.setTimestamp(ts);
-										singleMetricEntity.setTags(e.getTags());
-										singleMetricEntity.setValue(e.getValue()[i]);
-										// Min
-										if (startTimestamp == 0 || startTimestamp > ts) startTimestamp = ts;
-										// Max
-										if (stopTimestamp == 0 || stopTimestamp < ts) stopTimestamp = ts;
-										listener.entityCreated(singleMetricEntity);
-									}
-								}
-							}else {
-								// LOG.info("DEBUG: rowKey="+logAPIEntity.getEncodedRowkey());
-								listener.entityCreated(logAPIEntity);
-							}
-						} catch (Exception e) {
-							if(internalLog!=null) {
-								LOG.error("Got exception to handle " + internalLog.toString() + ": " + e.getMessage(), e);
-							}
-							throw new IOException(e);
-						}
-					}else{
-						LOG.error("Got null to parse internal log for row: " + row.length + " with fields: " + kvMap);
-					}
-					results.clear();
-				}else{
-					if(LOG.isDebugEnabled()) LOG.warn("Empty batch of KeyValue");
-				}
-			} while(hasMoreRows);
-		}catch(IOException ex){
-			LOG.error(ex.getMessage(),ex);
-			throw ex;
-		} finally {
-            if(scanner != null) {
+    protected HRegion getCurrentRegion() {
+        return this.env.getRegion();
+    }
+
+    /**
+     * <pre>
+     * Region-unittest
+     *  ,\x82\xB4\x85\xC2\x7F\xFF\xFE\xB6\xC9jNG\xEE!\x5C3\xBB\xAE\xA1
+     *  :\x05\xA5\xA9x\xB0\xA1"8\x05\xFB(\xD2VY\xDB\x9A\x06\x09\xA9\x98\xC2\xE3\x8D=,1413960230654.aaf2a6c9f2c87c196f43497243bb2424.
+     *
+     * RegionID-unittest,1413960230654
+     * </pre>
+     */
+    protected String getLogHeader() {
+        HRegion region = this.getCurrentRegion();
+        return LOG.isDebugEnabled() ? String.format("Region-%s", region.getRegionNameAsString()) :
+            String.format("Region-%s,%d", region.getTableDesc().getNameAsString(), region.getRegionId());
+    }
+
+    protected class InternalReadReport {
+        public InternalReadReport(long counter, long startTimestamp, long stopTimestamp) {
+            this.counter = counter;
+            this.startTimestamp = startTimestamp;
+            this.stopTimestamp = stopTimestamp;
+        }
+
+        public long getCounter() {
+            return counter;
+        }
+
+        public void setCounter(long counter) {
+            this.counter = counter;
+        }
+
+        public long getStartTimestamp() {
+            return startTimestamp;
+        }
+
+        public void setStartTimestamp(long startTimestamp) {
+            this.startTimestamp = startTimestamp;
+        }
+
+        public long getStopTimestamp() {
+            return stopTimestamp;
+        }
+
+        public void setStopTimestamp(long stopTimestamp) {
+            this.stopTimestamp = stopTimestamp;
+        }
+
+        private long counter;
+        private long startTimestamp;
+        private long stopTimestamp;
+    }
+
+    /**
+     * Asynchronous HBase scan read as entity.
+     */
+    protected InternalReadReport asyncStreamRead(EntityDefinition ed, Scan scan, EntityCreationListener listener) throws IOException {
+        long counter = 0;
+        long startTimestamp = 0;
+        long stopTimestamp = 0;
+
+        InternalScanner scanner = this.getCurrentRegion().getScanner(scan);
+        List<Cell> results = new ArrayList<Cell>();
+        try {
+            boolean hasMoreRows;
+            GenericMetricShadowEntity singleMetricEntity = null;
+            do {
+                hasMoreRows = scanner.next(results);
+                Map<String, byte[]> kvMap = new HashMap<String, byte[]>();
+                if (!results.isEmpty()) {
+                    counter++;
+                    byte[] row = results.get(0).getRow();
+                    long timestamp = RowkeyBuilder.getTimestamp(row, ed);
+
+                    // Min
+                    if (startTimestamp == 0 || startTimestamp > timestamp) {
+                        startTimestamp = timestamp;
+                    }
+
+                    // Max
+                    if (stopTimestamp == 0 || stopTimestamp < timestamp) {
+                        stopTimestamp = timestamp;
+                    }
+
+                    for (Cell kv : results) {
+                        String qualifierName = Bytes.toString(kv.getQualifier());
+                        // Qualifier qualifier = null;
+                        // if(!ed.isTag(qualifierName)){
+                        //  qualifier = ed.getQualifierNameMap().get(qualifierName);
+                        //  if(qualifier == null){
+                        //      LOG.error("qualifier for   " + qualifierName + " not exist");
+                        //      throw new NullPointerException("qualifier for field "+qualifierName+" not exist");
+                        //  }
+                        // }
+                        if (kv.getValue() != null) {
+                            kvMap.put(qualifierName, kv.getValue());
+                        }
+                    }
+
+                    // LOG.info("DEBUG: timestamp="+timestamp+", keys=["+StringUtils.join(kvMap.keySet(),",")+"]");
+
+                    InternalLog internalLog = HBaseInternalLogHelper.buildObject(ed, row, timestamp, kvMap);
+                    if (internalLog != null) {
+                        TaggedLogAPIEntity logAPIEntity = null;
+                        try {
+                            logAPIEntity = HBaseInternalLogHelper.buildEntity(internalLog, ed);
+                            if (logAPIEntity instanceof GenericMetricEntity) {
+                                if (singleMetricEntity == null) {
+                                    singleMetricEntity = new GenericMetricShadowEntity();
+                                }
+                                GenericMetricEntity e = (GenericMetricEntity) logAPIEntity;
+                                if (e.getValue() != null) {
+                                    int count = e.getValue().length;
+                                    @SuppressWarnings("unused")
+                                    Class<?> cls = ed.getMetricDefinition().getSingleTimestampEntityClass();
+                                    for (int i = 0; i < count; i++) {
+                                        long ts = logAPIEntity.getTimestamp() + i * ed.getMetricDefinition().getInterval();
+                                        // exclude those entity which is not within the time range in search condition. [start, end)
+                                        singleMetricEntity.setTimestamp(ts);
+                                        singleMetricEntity.setTags(e.getTags());
+                                        singleMetricEntity.setValue(e.getValue()[i]);
+                                        // Min
+                                        if (startTimestamp == 0 || startTimestamp > ts) {
+                                            startTimestamp = ts;
+                                        }
+                                        // Max
+                                        if (stopTimestamp == 0 || stopTimestamp < ts) {
+                                            stopTimestamp = ts;
+                                        }
+                                        listener.entityCreated(singleMetricEntity);
+                                    }
+                                }
+                            } else {
+                                // LOG.info("DEBUG: rowKey="+logAPIEntity.getEncodedRowkey());
+                                listener.entityCreated(logAPIEntity);
+                            }
+                        } catch (Exception e) {
+                            if (internalLog != null) {
+                                LOG.error("Got exception to handle " + internalLog.toString() + ": " + e.getMessage(), e);
+                            }
+                            throw new IOException(e);
+                        }
+                    } else {
+                        LOG.error("Got null to parse internal log for row: " + row.length + " with fields: " + kvMap);
+                    }
+                    results.clear();
+                } else {
+                    if (LOG.isDebugEnabled()) {
+                        LOG.warn("Empty batch of KeyValue");
+                    }
+                }
+            }
+            while (hasMoreRows);
+        } catch (IOException ex) {
+            LOG.error(ex.getMessage(), ex);
+            throw ex;
+        } finally {
+            if (scanner != null) {
                 scanner.close();
             }
-		}
-		return new InternalReadReport(counter,startTimestamp,stopTimestamp);
-	}
-
-	/**
-	 * Asynchronous HBase scan read as RAW qualifier
-	 *
-	 * @param scan
-	 * @param listener
-	 * @throws Exception
-	 */
-	protected InternalReadReport asyncStreamRead(EntityDefinition ed, Scan scan, QualifierCreationListener listener) throws IOException {
-//		_init();
-		long counter = 0;
-		long startTimestamp = 0;
-		long stopTimestamp = 0;
-		InternalScanner scanner = this.getCurrentRegion().getScanner(scan);
-		List<Cell> results = new ArrayList<Cell>();
-		try{
-			boolean hasMoreRows;//false by default
-			do{
-				hasMoreRows = scanner.next(results);
-				Map<String, byte[]> kvMap = new HashMap<String, byte[]>();
-				if(!results.isEmpty()){
-					counter ++;
-					byte[] row = results.get(0).getRow();
-//					if(ed.isTimeSeries()){
-					long timestamp = RowkeyBuilder.getTimestamp(row,ed);
-					// Min
-					if(startTimestamp == 0 || startTimestamp > timestamp ){
-						startTimestamp = timestamp;
-					}
-					// Max
-					if(stopTimestamp == 0 || stopTimestamp < timestamp ){
-						stopTimestamp = timestamp;
-					}
-//					}
-					
-					for(Cell kv:results){
-						String qualifierName = Bytes.toString(kv.getQualifier());
-						Qualifier qualifier = null;
-						if(!ed.isTag(qualifierName)){
-							qualifier = ed.getQualifierNameMap().get(qualifierName);
-							if(qualifier == null){
-								LOG.error("qualifier for field " + qualifierName + " not exist");
-								throw new IOException(new NullPointerException("qualifier for field "+qualifierName+" is null"));
-							}
-							qualifierName = qualifier.getDisplayName();
-						}
-						if(kv.getValue()!=null) kvMap.put(qualifierName,kv.getValue());
-					}
-					
-//					LOG.info("DEBUG: timestamp="+timestamp+", keys=["+StringUtils.join(kvMap.keySet(),",")+"]");
-
-					if(!kvMap.isEmpty()) listener.qualifierCreated(kvMap);
-					results.clear();
-				}else{
-					if(LOG.isDebugEnabled()) LOG.warn("Empty batch of KeyValue");
-				}
-			} while(hasMoreRows);
-		} catch(IOException ex){
-			LOG.error(ex.getMessage(),ex);
-			throw ex;
-		} finally {
-            if(scanner != null) {
+        }
+        return new InternalReadReport(counter, startTimestamp, stopTimestamp);
+    }
+
+    /**
+     * Asynchronous HBase scan read as RAW qualifier.
+     */
+    protected InternalReadReport asyncStreamRead(EntityDefinition ed, Scan scan, QualifierCreationListener listener) throws IOException {
+        long counter = 0;
+        long startTimestamp = 0;
+        long stopTimestamp = 0;
+        InternalScanner scanner = this.getCurrentRegion().getScanner(scan);
+        List<Cell> results = new ArrayList<Cell>();
+        try {
+            boolean hasMoreRows;//false by default
+            do {
+                hasMoreRows = scanner.next(results);
+                Map<String, byte[]> kvMap = new HashMap<>();
+                if (!results.isEmpty()) {
+                    counter++;
+                    byte[] row = results.get(0).getRow();
+                    long timestamp = RowkeyBuilder.getTimestamp(row, ed);
+                    // Min
+                    if (startTimestamp == 0 || startTimestamp > timestamp) {
+                        startTimestamp = timestamp;
+                    }
+                    // Max
+                    if (stopTimestamp == 0 || stopTimestamp < timestamp) {
+                        stopTimestamp = timestamp;
+                    }
+
+                    for (Cell kv : results) {
+                        String qualifierName = Bytes.toString(kv.getQualifier());
+                        Qualifier qualifier = null;
+                        if (!ed.isTag(qualifierName)) {
+                            qualifier = ed.getQualifierNameMap().get(qualifierName);
+                            if (qualifier == null) {
+                                LOG.error("qualifier for field " + qualifierName + " not exist");
+                                throw new IOException(new NullPointerException("qualifier for field " + qualifierName + " is null"));
+                            }
+                            qualifierName = qualifier.getDisplayName();
+                        }
+                        if (kv.getValue() != null) {
+                            kvMap.put(qualifierName, kv.getValue());
+                        }
+                    }
+
+                    if (!kvMap.isEmpty()) {
+                        listener.qualifierCreated(kvMap);
+                    }
+                    results.clear();
+                } else {
+                    if (LOG.isDebugEnabled()) {
+                        LOG.warn("Empty batch of KeyValue");
+                    }
+                }
+            }
+            while (hasMoreRows);
+        } catch (IOException ex) {
+            LOG.error(ex.getMessage(), ex);
+            throw ex;
+        } finally {
+            if (scanner != null) {
                 scanner.close();
             }
-		}
+        }
 
-		return new InternalReadReport(counter,startTimestamp,stopTimestamp);
-	}
+        return new InternalReadReport(counter, startTimestamp, stopTimestamp);
+    }
 
     @Override
-    public void aggregate(RpcController controller, AggregateProtos.AggregateRequest request, RpcCallback<AggregateProtos.AggregateResult> done) {
+    public void timeseriesAggregate(RpcController controller, AggregateProtos.TimeSeriesAggregateRequest request, RpcCallback<AggregateProtos.AggregateResult> done) {
         AggregateResult result = null;
         try {
             result = this.aggregate(ProtoBufConverter.fromPBEntityDefinition(request.getEntityDefinition()),
-                    ProtoBufConverter.fromPBScan(request.getScan()),
-                    ProtoBufConverter.fromPBStringList(request.getGroupbyFieldsList()),
-                    ProtoBufConverter.fromPBByteArrayList(request.getAggregateFuncTypesList()),
-                    ProtoBufConverter.fromPBStringList(request.getAggregatedFieldsList())
+                ProtoBufConverter.fromPBScan(request.getScan()),
+                ProtoBufConverter.fromPBStringList(request.getGroupbyFieldsList()),
+                ProtoBufConverter.fromPBByteArrayList(request.getAggregateFuncTypesList()),
+                ProtoBufConverter.fromPBStringList(request.getAggregatedFieldsList()),
+                request.getStartTime(),
+                request.getEndTime(),
+                request.getIntervalMin()
             );
         } catch (IOException e) {
+            LOG.error("Failed to convert result to PB-based message", e);
             ResponseConverter.setControllerException(controller, e);
         }
         try {
             done.run(ProtoBufConverter.toPBAggregateResult(result));
         } catch (IOException e) {
-            throw new RuntimeException("Failed to convert result to PB-based message",e);
+            LOG.error("Failed to convert result to PB-based message", e);
+            ResponseConverter.setControllerException(controller, e);
         }
     }
 
     @Override
-    public void timeseriesAggregate(RpcController controller, AggregateProtos.TimeSeriesAggregateRequest request, RpcCallback<AggregateProtos.AggregateResult> done) {
+    public void aggregate(RpcController controller, AggregateProtos.AggregateRequest request, RpcCallback<AggregateProtos.AggregateResult> done) {
         AggregateResult result = null;
         try {
             result = this.aggregate(ProtoBufConverter.fromPBEntityDefinition(request.getEntityDefinition()),
-                    ProtoBufConverter.fromPBScan(request.getScan()),
-                    ProtoBufConverter.fromPBStringList(request.getGroupbyFieldsList()),
-                    ProtoBufConverter.fromPBByteArrayList(request.getAggregateFuncTypesList()),
-                    ProtoBufConverter.fromPBStringList(request.getAggregatedFieldsList()),
-                    request.getStartTime(),
-                    request.getEndTime(),
-                    request.getIntervalMin()
+                ProtoBufConverter.fromPBScan(request.getScan()),
+                ProtoBufConverter.fromPBStringList(request.getGroupbyFieldsList()),
+                ProtoBufConverter.fromPBByteArrayList(request.getAggregateFuncTypesList()),
+                ProtoBufConverter.fromPBStringList(request.getAggregatedFieldsList())
             );
         } catch (IOException e) {
-            LOG.error("Failed to convert result to PB-based message",e);
             ResponseConverter.setControllerException(controller, e);
         }
         try {
             done.run(ProtoBufConverter.toPBAggregateResult(result));
         } catch (IOException e) {
-            LOG.error("Failed to convert result to PB-based message",e);
-            ResponseConverter.setControllerException(controller, e);
+            throw new RuntimeException("Failed to convert result to PB-based message", e);
         }
     }
 
-    private final static Logger LOG = LoggerFactory.getLogger(AggregateProtocolEndPoint.class);
-    /**
-     *
-     * @param entityDefinition
-     * @param scan
-     * @param groupbyFields
-     * @param aggregateFuncTypes
-     * @param aggregatedFields
-     * @return
-     * @throws Exception
-     */
     @Override
-    public AggregateResult aggregate(EntityDefinition entityDefinition, Scan scan, List<String> groupbyFields, List<byte[]> aggregateFuncTypes, List<String> aggregatedFields) throws IOException {
-//		LOG.info("Using coprocessor instance: "+this);
+    public AggregateResult aggregate(EntityDefinition entityDefinition, Scan scan, List<String> groupbyFields,
+                                     List<byte[]> aggregateFuncTypes, List<String> aggregatedFields) throws IOException {
         checkNotNull(entityDefinition, "entityDefinition");
         String serviceName = entityDefinition.getService();
-        LOG.info(this.getLogHeader() +" raw group aggregate on service: " + serviceName + " by: " + groupbyFields + " func: " + AggregateFunctionType.fromBytesList(aggregateFuncTypes) + " fields: " + aggregatedFields);
-        if(LOG.isDebugEnabled()) LOG.debug("SCAN: "+scan.toJSON());
-        long _start = System.currentTimeMillis();
-        final RawAggregator aggregator = new RawAggregator(groupbyFields,AggregateFunctionType.fromBytesList(aggregateFuncTypes),aggregatedFields,entityDefinition);
+        LOG.info(this.getLogHeader() + " raw group aggregate on service: " + serviceName
+            + " by: " + groupbyFields + " func: " + AggregateFunctionType.fromBytesList(aggregateFuncTypes) + " fields: " + aggregatedFields);
+        if (LOG.isDebugEnabled()) {
+            LOG.debug("SCAN: " + scan.toJSON());
+        }
+        final long startTimestamp = System.currentTimeMillis();
+        final RawAggregator aggregator = new RawAggregator(groupbyFields,
+            AggregateFunctionType.fromBytesList(aggregateFuncTypes), aggregatedFields, entityDefinition);
         InternalReadReport report = this.asyncStreamRead(entityDefinition, scan, aggregator);
 
         List<GroupbyKeyValue> keyValues = aggregator.getGroupbyKeyValues();
@@ -384,34 +382,31 @@ public class AggregateProtocolEndPoint extends AggregateProtos.AggregateProtocol
         result.setStopTimestamp(report.getStopTimestamp());
 
         long _stop = System.currentTimeMillis();
-        LOG.info(String.format("%s: scan = %d rows, group = %d keys, startTime = %d, endTime = %d, spend = %d ms", this.getLogHeader(),report.getCounter(),keyValues.size(),report.getStartTimestamp(),report.getStopTimestamp(),(_stop - _start)));
+        LOG.info(String.format("%s: scan = %d rows, group = %d keys, startTime = %d, endTime = %d, spend = %d ms",
+            this.getLogHeader(), report.getCounter(), keyValues.size(), report.getStartTimestamp(),
+            report.getStopTimestamp(), (_stop - startTimestamp)));
 
         return result;
     }
 
-    /**
-     * TODO: refactor time series aggregator to remove dependency of business logic entity class
-     *
-     * @param entityDefinition
-     * @param scan
-     * @param groupbyFields
-     * @param aggregateFuncTypes
-     * @param aggregatedFields
-     * @param intervalMin
-     * @return
-     * @throws Exception
-     */
     @Override
-    public AggregateResult aggregate(EntityDefinition entityDefinition, Scan scan, List<String> groupbyFields, List<byte[]> aggregateFuncTypes, List<String> aggregatedFields, long startTime,long endTime,long intervalMin) throws IOException {
-//		LOG.info("Using coprocessor instance: "+this);
+    public AggregateResult aggregate(EntityDefinition entityDefinition, Scan scan, List<String> groupbyFields,
+                                     List<byte[]> aggregateFuncTypes, List<String> aggregatedFields, long startTime, long endTime, long intervalMin) throws IOException {
         checkNotNull(entityDefinition, "entityDefinition");
         String serviceName = entityDefinition.getService();
-        LOG.info(this.getLogHeader() + " time series group aggregate on service: " + serviceName + " by: " + groupbyFields + " func: " + AggregateFunctionType.fromBytesList(aggregateFuncTypes) + " fields: " + aggregatedFields + " intervalMin: " + intervalMin +
-                " from: " + DateTimeUtil.millisecondsToHumanDateWithMilliseconds(startTime) + " to: " + DateTimeUtil.millisecondsToHumanDateWithMilliseconds(endTime));
-        if(LOG.isDebugEnabled()) LOG.debug("SCAN: "+scan.toJSON());
-        long _start = System.currentTimeMillis();
-        final TimeSeriesAggregator aggregator = new TimeSeriesAggregator(groupbyFields,AggregateFunctionType.fromBytesList(aggregateFuncTypes),aggregatedFields,startTime,endTime,intervalMin);
-        InternalReadReport report = this.asyncStreamRead(entityDefinition, scan,aggregator);
+        LOG.info(this.getLogHeader() + " time series group aggregate on service: " + serviceName
+            + " by: " + groupbyFields + " func: " + AggregateFunctionType.fromBytesList(aggregateFuncTypes)
+            + " fields: " + aggregatedFields + " intervalMin: " + intervalMin
+            + " from: " + DateTimeUtil.millisecondsToHumanDateWithMilliseconds(startTime)
+            + " to: " + DateTimeUtil.millisecondsToHumanDateWithMilliseconds(endTime));
+        if (LOG.isDebugEnabled()) {
+            LOG.debug("SCAN: " + scan.toJSON());
+        }
+
+        final long startTimestamp = System.currentTimeMillis();
+        final TimeSeriesAggregator aggregator = new TimeSeriesAggregator(groupbyFields,
+            AggregateFunctionType.fromBytesList(aggregateFuncTypes), aggregatedFields, startTime, endTime, intervalMin);
+        InternalReadReport report = this.asyncStreamRead(entityDefinition, scan, aggregator);
         List<GroupbyKeyValue> keyValues = aggregator.getGroupbyKeyValues();
 
         AggregateResult result = new AggregateResult();
@@ -420,28 +415,9 @@ public class AggregateProtocolEndPoint extends AggregateProtos.AggregateProtocol
         result.setStopTimestamp(report.getStopTimestamp());
 
         long _stop = System.currentTimeMillis();
-        LOG.info(String.format("%s: scan = %d rows, group = %d keys, startTime = %d, endTime = %d, spend = %d ms", this.getLogHeader(),report.getCounter(),keyValues.size(),report.getStartTimestamp(),report.getStopTimestamp(),(_stop - _start)));
+        LOG.info(String.format("%s: scan = %d rows, group = %d keys, startTime = %d, endTime = %d, spend = %d ms",
+            this.getLogHeader(), report.getCounter(), keyValues.size(), report.getStartTimestamp(), report.getStopTimestamp(), (_stop - startTimestamp)));
 
         return result;
     }
-
-//	/**
-//	 * Initialization per aggregate RPC call
-//	 */
-//	private void _init(){
-//		this.startTimestamp = 0;
-//		this.stopTimestamp = 0;
-//	}
-//
-//	// Min
-//	private long startTimestamp;
-//	// Max
-//	private long stopTimestamp;
-//
-//	public long getStartTimestamp() {
-//		return startTimestamp;
-//	}
-//	public long getStopTimestamp() {
-//		return stopTimestamp;
-//	}
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/21187b55/eagle-core/eagle-query/eagle-storage-hbase/src/main/java/org/apache/eagle/storage/hbase/query/coprocessor/AggregateResult.java
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-query/eagle-storage-hbase/src/main/java/org/apache/eagle/storage/hbase/query/coprocessor/AggregateResult.java b/eagle-core/eagle-query/eagle-storage-hbase/src/main/java/org/apache/eagle/storage/hbase/query/coprocessor/AggregateResult.java
index 84380eb..a49ad57 100755
--- a/eagle-core/eagle-query/eagle-storage-hbase/src/main/java/org/apache/eagle/storage/hbase/query/coprocessor/AggregateResult.java
+++ b/eagle-core/eagle-query/eagle-storage-hbase/src/main/java/org/apache/eagle/storage/hbase/query/coprocessor/AggregateResult.java
@@ -31,8 +31,7 @@ import java.io.Serializable;
 import java.util.List;
 
 /**
- * Aggregated writable result consist of group-by key-values list and additional meta information
- * 
+ * Aggregated writable result consist of group-by key-values list and additional meta information.
  * <h2>Schema</h2>
  * <pre>
  * {
@@ -42,88 +41,88 @@ import java.util.List;
  * }
  * </pre>
  */
-public class AggregateResult implements Writable,Serializable{
-
-	private final static Logger LOG = LoggerFactory.getLogger(AggregateResult.class);
-
-	/**
-	 * Automatically generated default serialVersionUID
-	 */
-	private static final long serialVersionUID = 1L;
-	
-	private final WritableList<GroupbyKeyValue> keyValues;
-	
-	private long startTimestamp = 0;
-	
-	public long getStartTimestamp() {
-		return startTimestamp;
-	}
-
-	public void setStartTimestamp(long startTimestamp) {
-		this.startTimestamp = startTimestamp;
-	}
-
-	public long getStopTimestamp() {
-		return stopTimestamp;
-	}
-
-	public void setStopTimestamp(long stopTimestamp) {
-		this.stopTimestamp = stopTimestamp;
-	}
-
-	public WritableList<GroupbyKeyValue> getKeyValues() {
-		return keyValues;
-	}
-	
-	public void setKeyValues(List<GroupbyKeyValue> keyValues){
-		this.keyValues.addAll(keyValues);
-	}
-	
-	private long stopTimestamp;
-	
-	public AggregateResult(){
-		this.keyValues = new WritableList<GroupbyKeyValue>(GroupbyKeyValue.class);
-	}
-	
-	@Override
-	public void readFields(DataInput in) throws IOException {
-		this.startTimestamp = in.readLong();
-		this.stopTimestamp = in.readLong();
-		keyValues.readFields(in);
-	}
-	
-	@Override
-	public void write(DataOutput out) throws IOException {
-		out.writeLong(this.startTimestamp);
-		out.writeLong(this.stopTimestamp);
-		keyValues.write(out);
-	}
-
-
-	public static AggregateResult build(List<String[]> keys,List<double[]> values,List<Integer> counts,long startTimestamp,long stopTimestamp){
-		if(keys.size() > values.size()){
-			throw new IllegalArgumentException("keys' size: "+keys.size()+" not equal with values' size: "+values.size());
-		}
-		AggregateResult result = new AggregateResult();
-		result.setStartTimestamp(startTimestamp);
-		result.setStopTimestamp(stopTimestamp);
-		WritableList<GroupbyKeyValue> keyValues = new WritableList<GroupbyKeyValue>(GroupbyKeyValue.class,keys.size());
-
-		for(int i=0;i<keys.size();i++) {
-			String[] key  = keys.get(i);
-			GroupbyKey gkey = new GroupbyKey();
-			for (String k : key) {
-				gkey.addValue(k.getBytes());
-			}
-			GroupbyValue gvalue = new GroupbyValue();
-			double[] value = values.get(i);
-			for(double v:value){
-				gvalue.add(v);
-				gvalue.addMeta(counts.get(i));
-			}
-			keyValues.add(new GroupbyKeyValue(gkey, gvalue));
-		}
-		result.setKeyValues(keyValues);
-		return result;
-	}
+public class AggregateResult implements Writable, Serializable {
+
+    private static final Logger LOG = LoggerFactory.getLogger(AggregateResult.class);
+
+    /**
+     * Automatically generated default serialVersionUID.
+     */
+    private static final long serialVersionUID = 1L;
+
+    private final WritableList<GroupbyKeyValue> keyValues;
+
+    private long startTimestamp = 0;
+
+    public long getStartTimestamp() {
+        return startTimestamp;
+    }
+
+    public void setStartTimestamp(long startTimestamp) {
+        this.startTimestamp = startTimestamp;
+    }
+
+    public long getStopTimestamp() {
+        return stopTimestamp;
+    }
+
+    public void setStopTimestamp(long stopTimestamp) {
+        this.stopTimestamp = stopTimestamp;
+    }
+
+    public WritableList<GroupbyKeyValue> getKeyValues() {
+        return keyValues;
+    }
+
+    public void setKeyValues(List<GroupbyKeyValue> keyValues) {
+        this.keyValues.addAll(keyValues);
+    }
+
+    private long stopTimestamp;
+
+    public AggregateResult() {
+        this.keyValues = new WritableList<GroupbyKeyValue>(GroupbyKeyValue.class);
+    }
+
+    @Override
+    public void readFields(DataInput in) throws IOException {
+        this.startTimestamp = in.readLong();
+        this.stopTimestamp = in.readLong();
+        keyValues.readFields(in);
+    }
+
+    @Override
+    public void write(DataOutput out) throws IOException {
+        out.writeLong(this.startTimestamp);
+        out.writeLong(this.stopTimestamp);
+        keyValues.write(out);
+    }
+
+
+    public static AggregateResult build(List<String[]> keys, List<double[]> values, List<Integer> counts, long startTimestamp, long stopTimestamp) {
+        if (keys.size() > values.size()) {
+            throw new IllegalArgumentException("keys' size: " + keys.size() + " not equal with values' size: " + values.size());
+        }
+        AggregateResult result = new AggregateResult();
+        result.setStartTimestamp(startTimestamp);
+        result.setStopTimestamp(stopTimestamp);
+        WritableList<GroupbyKeyValue> keyValues = new WritableList<GroupbyKeyValue>(GroupbyKeyValue.class, keys.size());
+
+        for (int i = 0; i < keys.size(); i++) {
+            String[] key = keys.get(i);
+            GroupbyKey gkey = new GroupbyKey();
+            for (String k : key) {
+                gkey.addValue(k.getBytes());
+            }
+            GroupbyValue gvalue = new GroupbyValue();
+            double[] value = values.get(i);
+            for (double v : value) {
+                gvalue.add(v);
+                gvalue.addMeta(counts.get(i));
+            }
+            keyValues.add(new GroupbyKeyValue(gkey, gvalue));
+        }
+        result.setKeyValues(keyValues);
+        return result;
+    }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/21187b55/eagle-core/eagle-query/eagle-storage-hbase/src/main/java/org/apache/eagle/storage/hbase/query/coprocessor/AggregateResultCallback.java
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-query/eagle-storage-hbase/src/main/java/org/apache/eagle/storage/hbase/query/coprocessor/AggregateResultCallback.java b/eagle-core/eagle-query/eagle-storage-hbase/src/main/java/org/apache/eagle/storage/hbase/query/coprocessor/AggregateResultCallback.java
index a68a592..306a6d1 100755
--- a/eagle-core/eagle-query/eagle-storage-hbase/src/main/java/org/apache/eagle/storage/hbase/query/coprocessor/AggregateResultCallback.java
+++ b/eagle-core/eagle-query/eagle-storage-hbase/src/main/java/org/apache/eagle/storage/hbase/query/coprocessor/AggregateResultCallback.java
@@ -21,28 +21,20 @@ import org.apache.hadoop.hbase.client.coprocessor.Batch;
 
 /**
  * <h1>AggregateResultCallback Interface</h1>
- *
  * Merge coprocessor results from different regions and generate final aggregate result
  * <br/>
- *
- * @see org.apache.hadoop.hbase.client.HTableInterface
- * 		coprocessorExec(Class<T> protocol, byte[] startKey, byte[] endKey, Batch.Call<T,R> callable) throws IOException, Throwable;
- *
+ * @see org.apache.hadoop.hbase.client.HTableInterface#coprocessorService(Class, byte[], byte[], Batch.Call, Batch.Callback)
  */
-public interface AggregateResultCallback extends Batch.Callback<AggregateProtos.AggregateResult>{
-	/**
-	 * Generate final result after callback from region servers
-	 *
-	 * @return AggregateResult
-	 */
+public interface AggregateResultCallback extends Batch.Callback<AggregateProtos.AggregateResult> {
+    /**
+     * Generate final result after callback from region servers.
+     *
+     * @return AggregateResult
+     */
     AggregateResult result();
 
     /**
-     * Compatible for older callback interface in 0.94 or older
-     *
-     * @param region
-     * @param row
-     * @param result
+     * Compatible for older callback interface in 0.94 or older.
      */
     void update(byte[] region, byte[] row, AggregateResult result);
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/21187b55/eagle-core/eagle-query/eagle-storage-hbase/src/main/java/org/apache/eagle/storage/hbase/query/coprocessor/ProtoBufConverter.java
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-query/eagle-storage-hbase/src/main/java/org/apache/eagle/storage/hbase/query/coprocessor/ProtoBufConverter.java b/eagle-core/eagle-query/eagle-storage-hbase/src/main/java/org/apache/eagle/storage/hbase/query/coprocessor/ProtoBufConverter.java
index 060d3ba..c3c57ed 100644
--- a/eagle-core/eagle-query/eagle-storage-hbase/src/main/java/org/apache/eagle/storage/hbase/query/coprocessor/ProtoBufConverter.java
+++ b/eagle-core/eagle-query/eagle-storage-hbase/src/main/java/org/apache/eagle/storage/hbase/query/coprocessor/ProtoBufConverter.java
@@ -32,44 +32,63 @@ import java.util.ArrayList;
 import java.util.List;
 
 /**
- * The protocol adapter for migrating from hbase-0.94 to hbase-0.96+
- *
- * @since 6/3/15
+ * The protocol adapter for <code>hbase-0.98</code> and <code>protobuffer-2.5</code>
  */
 public final class ProtoBufConverter {
     public static AggregateResult fromPBResult(AggregateProtos.AggregateResult pbResult) throws IOException {
-        ByteArrayDataInput byteArrayDataInput = ByteStreams.newDataInput(pbResult.getByteArray().toByteArray());;
+        ByteArrayDataInput byteArrayDataInput = ByteStreams.newDataInput(pbResult.getByteArray().toByteArray());
+        ;
         AggregateResult result = new AggregateResult();
         result.readFields(byteArrayDataInput);
         return result;
     }
 
-    public static AggregateProtos.AggregateRequest toPBRequest(EntityDefinition entityDefinition, Scan scan, List<String> groupbyFields, List<byte[]> aggregateFuncTypesBytes, List<String> aggregatedFields) throws IOException {
+    public static AggregateProtos.AggregateRequest toPBRequest(
+            EntityDefinition entityDefinition, Scan scan, List<String> groupbyFields,
+            List<byte[]> aggregateFuncTypesBytes, List<String> aggregatedFields) throws IOException {
         AggregateProtos.AggregateRequest.Builder builder = AggregateProtos.AggregateRequest.newBuilder()
                 .setEntityDefinition(AggregateProtos.EntityDefinition.newBuilder().setByteArray(writableToByteString(entityDefinition)))
                 .setScan(toPBScan(scan));
 
-        for(String groupbyField:groupbyFields) builder.addGroupbyFields(groupbyField);
-        for(byte[] funcTypeBytes:aggregateFuncTypesBytes) builder.addAggregateFuncTypes(ByteString.copyFrom(funcTypeBytes));
-        for(String aggField:aggregatedFields) builder.addAggregatedFields(aggField);
+        for (String groupbyField : groupbyFields) {
+            builder.addGroupbyFields(groupbyField);
+        }
+
+        for (byte[] funcTypeBytes : aggregateFuncTypesBytes) {
+            builder.addAggregateFuncTypes(ByteString.copyFrom(funcTypeBytes));
+        }
+
+        for (String aggField : aggregatedFields) {
+            builder.addAggregatedFields(aggField);
+        }
 
         return builder.build();
     }
 
     public static ByteString writableToByteString(Writable writable) throws IOException {
-        ByteArrayDataOutput dataOutput = ByteStreams.newDataOutput();;
+        ByteArrayDataOutput dataOutput = ByteStreams.newDataOutput();
         writable.write(dataOutput);
         return ByteString.copyFrom(dataOutput.toByteArray());
     }
 
-    public static AggregateProtos.TimeSeriesAggregateRequest toPBTimeSeriesRequest(EntityDefinition entityDefinition, Scan scan, List<String> groupbyFields, List<byte[]> aggregateFuncTypesBytes, List<String> aggregatedFields, long startTime, long endTime, long intervalMin) throws IOException {
-        AggregateProtos.TimeSeriesAggregateRequest.Builder builder = AggregateProtos.TimeSeriesAggregateRequest.newBuilder()
+    public static AggregateProtos.TimeSeriesAggregateRequest toPBTimeSeriesRequest(
+            EntityDefinition entityDefinition, Scan scan, List<String> groupbyFields,
+            List<byte[]> aggregateFuncTypesBytes, List<String> aggregatedFields,
+            long startTime, long endTime, long intervalMin) throws IOException {
+        AggregateProtos.TimeSeriesAggregateRequest.Builder builder =
+                AggregateProtos.TimeSeriesAggregateRequest.newBuilder()
                 .setEntityDefinition(AggregateProtos.EntityDefinition.newBuilder().setByteArray(writableToByteString(entityDefinition)))
                 .setScan(toPBScan(scan));
 
-        for(String groupbyField:groupbyFields) builder.addGroupbyFields(groupbyField);
-        for(byte[] funcTypeBytes:aggregateFuncTypesBytes) builder.addAggregateFuncTypes(ByteString.copyFrom(funcTypeBytes));
-        for(String aggField:aggregatedFields) builder.addAggregatedFields(aggField);
+        for (String groupbyField : groupbyFields) {
+            builder.addGroupbyFields(groupbyField);
+        }
+        for (byte[] funcTypeBytes : aggregateFuncTypesBytes) {
+            builder.addAggregateFuncTypes(ByteString.copyFrom(funcTypeBytes));
+        }
+        for (String aggField : aggregatedFields) {
+            builder.addAggregatedFields(aggField);
+        }
 
         builder.setStartTime(startTime);
         builder.setEndTime(endTime);
@@ -79,33 +98,36 @@ public final class ProtoBufConverter {
     }
 
     public static EntityDefinition fromPBEntityDefinition(AggregateProtos.EntityDefinition entityDefinition) throws IOException {
-        ByteArrayDataInput byteArrayDataInput = ByteStreams.newDataInput(entityDefinition.getByteArray().toByteArray());;
+        ByteArrayDataInput byteArrayDataInput = ByteStreams.newDataInput(entityDefinition.getByteArray().toByteArray());
+        ;
         EntityDefinition result = new EntityDefinition();
         result.readFields(byteArrayDataInput);
         return result;
     }
 
-    public static List<String> fromPBStringList(com.google.protobuf.ProtocolStringList groupbyFieldsList) {
-        List<String> result = new ArrayList<>(groupbyFieldsList.size());
-        for(ByteString byteString:groupbyFieldsList.asByteStringList()){
-            result.add(byteString.toStringUtf8());
-        }
-        return result;
+    //    /**
+    //     * For protobuffer-2.6
+    //     */
+    //    public static List<String> fromPBStringList(com.google.protobuf.ProtocolStringList groupbyFieldsList) {
+    //        List<String> result = new ArrayList<>(groupbyFieldsList.size());
+    //        for(ByteString byteString:groupbyFieldsList.asByteStringList()){
+    //            result.add(byteString.toStringUtf8());
+    //        }
+    //        return result;
+    //    }
+
+    public static List<String> fromPBStringList(List<String> groupbyFieldsList) {
+        return groupbyFieldsList;
     }
 
     public static List<byte[]> fromPBByteArrayList(List<ByteString> aggregateFuncTypesList) {
         List<byte[]> bytesArrayList = new ArrayList<>(aggregateFuncTypesList.size());
-        for(ByteString byteString:aggregateFuncTypesList){
+        for (ByteString byteString : aggregateFuncTypesList) {
             bytesArrayList.add(byteString.toByteArray());
         }
         return bytesArrayList;
     }
 
-    /**
-     *
-     * @param scan
-     * @return
-     */
     public static Scan fromPBScan(ClientProtos.Scan scan) throws IOException {
         return ProtobufUtil.toScan(scan);
     }

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/21187b55/eagle-core/eagle-query/eagle-storage-hbase/src/main/java/org/apache/eagle/storage/hbase/query/coprocessor/_AggregateProtocolEndPoint.java
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-query/eagle-storage-hbase/src/main/java/org/apache/eagle/storage/hbase/query/coprocessor/_AggregateProtocolEndPoint.java b/eagle-core/eagle-query/eagle-storage-hbase/src/main/java/org/apache/eagle/storage/hbase/query/coprocessor/_AggregateProtocolEndPoint.java
deleted file mode 100755
index af213e9..0000000
--- a/eagle-core/eagle-query/eagle-storage-hbase/src/main/java/org/apache/eagle/storage/hbase/query/coprocessor/_AggregateProtocolEndPoint.java
+++ /dev/null
@@ -1,139 +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 eagle.storage.hbase.query.coprocessor;
-//
-//import eagle.log.entity.meta.EntityDefinition;
-//import eagle.query.aggregate.AggregateFunctionType;
-//import eagle.query.aggregate.raw.GroupbyKeyValue;
-//import eagle.query.aggregate.raw.RawAggregator;
-//import eagle.query.aggregate.timeseries.TimeSeriesAggregator;
-//import eagle.storage.hbase.query.coprocessor.generated.AggregateProtos;
-//import eagle.storage.hbase.query.coprocessor.impl.AbstractAggregateEndPoint;
-//import hadoop.eagle.common.DateTimeUtil;
-//import com.google.protobuf.RpcCallback;
-//import com.google.protobuf.RpcController;
-//import org.apache.hadoop.hbase.client.Scan;
-//import org.apache.hadoop.hbase.protobuf.ResponseConverter;
-//import org.apache.hadoop.hbase.protobuf.generated.ClientProtos;
-//import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
-//import org.slf4j.Logger;
-//import org.slf4j.LoggerFactory;
-//
-//import java.io.IOException;
-//import java.util.List;
-//
-///**
-// * Coprocessor EndPoint of protocol <code>AggregateProtocol</code>
-// *
-// * <br/>
-// * <h2>Deployment:</h2>
-// *
-// * Firstly deploy jar files to cluster on local file system or HDFS.<br/><br/>
-// * Secondly configure in <code>hbase-site.xml</code> as following:
-// * <pre>
-// * &lt;property&gt;
-// *   &lt;name>hbase.coprocessor.region.classes&lt;/name&gt;
-// *   &lt;value>AggregateProtocolEndPoint&lt;/value&gt;
-// * &lt;/property&gt;
-// * </pre>
-// * Or register on related hbase tables
-// * <pre>
-// * hbase(main):005:0>  alter 't1', METHOD => 'table_att', 'coprocessor'=>'hdfs:///foo.jar|AggregateProtocolEndPoint|1001|'
-// * </pre>
-// *
-// * <h2>Reference:</h2>
-// * <a href="https://blogs.apache.org/hbase/entry/coprocessor_introduction">
-// * 	Coprocessor Introduction
-// * 	(Authors: Trend Micro Hadoop Group: Mingjie Lai, Eugene Koontz, Andrew Purtell)
-// * </a> <br/><br/>
-// *
-// * @see AggregateProtocol
-// *
-//// * @since : 10/31/14,2014
-// */
-//@SuppressWarnings("unused")
-//public class AggregateProtocolEndPoint extends AbstractAggregateEndPoint {
-//	private final static Logger LOG = LoggerFactory.getLogger(AggregateProtocolEndPoint.class);
-//	/**
-//	 *
-//	 * @param entityDefinition
-//	 * @param scan
-//	 * @param groupbyFields
-//	 * @param aggregateFuncTypes
-//	 * @param aggregatedFields
-//	 * @return
-//	 * @throws Exception
-//	 */
-//	@Override
-//	public AggregateResult aggregate(EntityDefinition entityDefinition, Scan scan, List<String> groupbyFields, List<byte[]> aggregateFuncTypes, List<String> aggregatedFields) throws IOException {
-////		LOG.info("Using coprocessor instance: "+this);
-//		checkNotNull(entityDefinition, "entityDefinition");
-//		String serviceName = entityDefinition.getService();
-//		LOG.info(this.getLogHeader() +" raw group aggregate on service: " + serviceName + " by: " + groupbyFields + " func: " + AggregateFunctionType.fromBytesList(aggregateFuncTypes) + " fields: " + aggregatedFields);
-//		if(LOG.isDebugEnabled()) LOG.debug("SCAN: "+scan.toJSON());
-//		long _start = System.currentTimeMillis();
-//		final RawAggregator aggregator = new RawAggregator(groupbyFields,AggregateFunctionType.fromBytesList(aggregateFuncTypes),aggregatedFields,entityDefinition);
-//		InternalReadReport report = this.asyncStreamRead(entityDefinition, scan, aggregator);
-//
-//		List<GroupbyKeyValue> keyValues = aggregator.getGroupbyKeyValues();
-//		AggregateResult result = new AggregateResult();
-//		result.setKeyValues(keyValues);
-//		result.setStartTimestamp(report.getStartTimestamp());
-//		result.setStopTimestamp(report.getStopTimestamp());
-//
-//		long _stop = System.currentTimeMillis();
-//		LOG.info(String.format("%s: scan = %d rows, group = %d keys, startTime = %d, endTime = %d, spend = %d ms", this.getLogHeader(),report.getCounter(),keyValues.size(),report.getStartTimestamp(),report.getStopTimestamp(),(_stop - _start)));
-//
-//		return result;
-//	}
-//
-//	/**
-//	 * TODO: refactor time series aggregator to remove dependency of business logic entity class
-//	 *
-//	 * @param entityDefinition
-//	 * @param scan
-//	 * @param groupbyFields
-//	 * @param aggregateFuncTypes
-//	 * @param aggregatedFields
-//	 * @param intervalMin
-//	 * @return
-//	 * @throws Exception
-//	 */
-//	@Override
-//	public AggregateResult aggregate(EntityDefinition entityDefinition, Scan scan, List<String> groupbyFields, List<byte[]> aggregateFuncTypes, List<String> aggregatedFields, long startTime,long endTime,long intervalMin) throws IOException {
-////		LOG.info("Using coprocessor instance: "+this);
-//		checkNotNull(entityDefinition, "entityDefinition");
-//		String serviceName = entityDefinition.getService();
-//		LOG.info(this.getLogHeader() + " time series group aggregate on service: " + serviceName + " by: " + groupbyFields + " func: " + AggregateFunctionType.fromBytesList(aggregateFuncTypes) + " fields: " + aggregatedFields + " intervalMin: " + intervalMin +
-//				" from: " + DateTimeUtil.millisecondsToHumanDateWithMilliseconds(startTime) + " to: " + DateTimeUtil.millisecondsToHumanDateWithMilliseconds(endTime));
-//		if(LOG.isDebugEnabled()) LOG.debug("SCAN: "+scan.toJSON());
-//		long _start = System.currentTimeMillis();
-//		final TimeSeriesAggregator aggregator = new TimeSeriesAggregator(groupbyFields,AggregateFunctionType.fromBytesList(aggregateFuncTypes),aggregatedFields,startTime,endTime,intervalMin);
-//		InternalReadReport report = this.asyncStreamRead(entityDefinition, scan,aggregator);
-//		List<GroupbyKeyValue> keyValues = aggregator.getGroupbyKeyValues();
-//
-//		AggregateResult result = new AggregateResult();
-//		result.setKeyValues(keyValues);
-//		result.setStartTimestamp(report.getStartTimestamp());
-//		result.setStopTimestamp(report.getStopTimestamp());
-//
-//		long _stop = System.currentTimeMillis();
-//		LOG.info(String.format("%s: scan = %d rows, group = %d keys, startTime = %d, endTime = %d, spend = %d ms", this.getLogHeader(),report.getCounter(),keyValues.size(),report.getStartTimestamp(),report.getStopTimestamp(),(_stop - _start)));
-//
-//		return result;
-//	}
-//}
\ No newline at end of file


[25/52] [abbrv] incubator-eagle git commit: EAGLE-506 : AlertEngine : Make policy definition handler extensible

Posted by yo...@apache.org.
EAGLE-506 : AlertEngine : Make policy definition handler extensible

Author : ralphsu


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

Branch: refs/heads/master
Commit: da7f5520613c4860fb2dfb2cb5fad64f9d5d727b
Parents: 3cc1830
Author: Ralph, Su <su...@gmail.com>
Authored: Mon Aug 29 23:33:27 2016 +0800
Committer: Ralph, Su <su...@gmail.com>
Committed: Mon Aug 29 23:33:27 2016 +0800

----------------------------------------------------------------------
 .../engine/coordinator/PolicyDefinition.java    |   9 ++
 .../evaluator/CompositePolicyHandler.java       |   2 +-
 .../engine/evaluator/PolicyStreamHandlers.java  |  25 ++++-
 .../eagle/alert/engine/runner/AlertBolt.java    |   1 +
 .../alert/engine/router/CustomizedHandler.java  |  51 +++++++++
 .../alert/engine/router/TestAlertBolt.java      | 109 +++++++++++++++----
 6 files changed, 168 insertions(+), 29 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/da7f5520/eagle-core/eagle-alert-parent/eagle-alert/alert-common/src/main/java/org/apache/eagle/alert/engine/coordinator/PolicyDefinition.java
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-alert-parent/eagle-alert/alert-common/src/main/java/org/apache/eagle/alert/engine/coordinator/PolicyDefinition.java b/eagle-core/eagle-alert-parent/eagle-alert/alert-common/src/main/java/org/apache/eagle/alert/engine/coordinator/PolicyDefinition.java
index 363264e..827172f 100644
--- a/eagle-core/eagle-alert-parent/eagle-alert/alert-common/src/main/java/org/apache/eagle/alert/engine/coordinator/PolicyDefinition.java
+++ b/eagle-core/eagle-alert-parent/eagle-alert/alert-common/src/main/java/org/apache/eagle/alert/engine/coordinator/PolicyDefinition.java
@@ -162,6 +162,7 @@ public class PolicyDefinition implements Serializable{
 
         public String type;
         public String value;
+        public String handlerClass;
 
         private List<String> inputStreams = new ArrayList<String>();
         private List<String> outputStreams = new ArrayList<String>();
@@ -228,6 +229,14 @@ public class PolicyDefinition implements Serializable{
             return outputStreams;
         }
 
+        public String getHandlerClass() {
+            return handlerClass;
+        }
+
+        public void setHandlerClass(String handlerClass) {
+            this.handlerClass = handlerClass;
+        }
+
         @Override
         public String toString() {
             return String.format("{type=\"%s\",value=\"%s\", inputStreams=\"%s\", outputStreams=\"%s\" }",type,value, inputStreams, outputStreams);

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/da7f5520/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/main/java/org/apache/eagle/alert/engine/evaluator/CompositePolicyHandler.java
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/main/java/org/apache/eagle/alert/engine/evaluator/CompositePolicyHandler.java b/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/main/java/org/apache/eagle/alert/engine/evaluator/CompositePolicyHandler.java
index 047ee6f..4d69bca 100644
--- a/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/main/java/org/apache/eagle/alert/engine/evaluator/CompositePolicyHandler.java
+++ b/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/main/java/org/apache/eagle/alert/engine/evaluator/CompositePolicyHandler.java
@@ -49,7 +49,7 @@ public class CompositePolicyHandler implements PolicyStreamHandler {
     public void prepare(Collector<AlertStreamEvent> collector, PolicyHandlerContext context) throws Exception {
         this.collector = collector;
         // TODO: create two handlers
-        policyHandler = PolicyStreamHandlers.createHandler(context.getPolicyDefinition().getDefinition().type, sds);
+        policyHandler = PolicyStreamHandlers.createHandler(context.getPolicyDefinition().getDefinition(), sds);
         policyHandler.prepare(collector, context);
         handlers.add(policyHandler);
 

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/da7f5520/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/main/java/org/apache/eagle/alert/engine/evaluator/PolicyStreamHandlers.java
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/main/java/org/apache/eagle/alert/engine/evaluator/PolicyStreamHandlers.java b/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/main/java/org/apache/eagle/alert/engine/evaluator/PolicyStreamHandlers.java
index 93327b7..ef9caf0 100644
--- a/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/main/java/org/apache/eagle/alert/engine/evaluator/PolicyStreamHandlers.java
+++ b/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/main/java/org/apache/eagle/alert/engine/evaluator/PolicyStreamHandlers.java
@@ -16,11 +16,14 @@
  */
 package org.apache.eagle.alert.engine.evaluator;
 
+import org.apache.eagle.alert.engine.coordinator.PolicyDefinition;
 import org.apache.eagle.alert.engine.coordinator.StreamDefinition;
 import org.apache.eagle.alert.engine.evaluator.absence.AbsencePolicyHandler;
 import org.apache.eagle.alert.engine.evaluator.impl.SiddhiPolicyHandler;
 import org.apache.eagle.alert.engine.evaluator.impl.SiddhiPolicyStateHandler;
 import org.apache.eagle.alert.engine.evaluator.nodata.NoDataPolicyHandler;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import java.util.Map;
 
@@ -28,19 +31,31 @@ import java.util.Map;
  * TODO/FIXME: to support multiple stage definition in single policy. The methods in this class is not good to understand now.(Hard code of 0/1).
  */
 public class PolicyStreamHandlers {
+    private static final Logger LOG = LoggerFactory.getLogger(PolicyStreamHandlers.class);
+
     public static final String SIDDHI_ENGINE = "siddhi";
     public static final String NO_DATA_ALERT_ENGINE = "nodataalert";
     public static final String ABSENCE_ALERT_ENGINE = "absencealert";
+    public static final String CUSTOMIZED_ENGINE = "Custom";
 
-    public static PolicyStreamHandler createHandler(String type, Map<String, StreamDefinition> sds) {
-        if (SIDDHI_ENGINE.equals(type)) {
+    public static PolicyStreamHandler createHandler(PolicyDefinition.Definition definition, Map<String, StreamDefinition> sds) {
+        if (SIDDHI_ENGINE.equals(definition.getType())) {
             return new SiddhiPolicyHandler(sds, 0);// // FIXME: 8/2/16 
-        } else if (NO_DATA_ALERT_ENGINE.equals(type)) {
+        } else if (NO_DATA_ALERT_ENGINE.equals(definition.getType())) {
             return new NoDataPolicyHandler(sds);
-        } else if (ABSENCE_ALERT_ENGINE.equals(type)) {
+        } else if (ABSENCE_ALERT_ENGINE.equals(definition.getType())) {
             return new AbsencePolicyHandler(sds);
+        } else if (CUSTOMIZED_ENGINE.equals(definition.getType())) {
+            try {
+                Class<?> handlerClz = Class.forName(definition.getHandlerClass());
+                PolicyStreamHandler handler = (PolicyStreamHandler) handlerClz.getConstructor(Map.class).newInstance(sds);
+                return handler;
+            } catch (Exception e) {
+                LOG.error("Not able to create policy handler for handler class " + definition.getHandlerClass(), e);
+                throw new IllegalArgumentException("Illegal extended policy handler class!" + definition.getHandlerClass());
+            }
         }
-        throw new IllegalArgumentException("Illegal policy stream handler type " + type);
+        throw new IllegalArgumentException("Illegal policy stream handler type " + definition.getType());
     }
 
     public static PolicyStreamHandler createStateHandler(String type, Map<String, StreamDefinition> sds) {

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/da7f5520/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/main/java/org/apache/eagle/alert/engine/runner/AlertBolt.java
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/main/java/org/apache/eagle/alert/engine/runner/AlertBolt.java b/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/main/java/org/apache/eagle/alert/engine/runner/AlertBolt.java
index ce5125b..fecb2f1 100755
--- a/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/main/java/org/apache/eagle/alert/engine/runner/AlertBolt.java
+++ b/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/main/java/org/apache/eagle/alert/engine/runner/AlertBolt.java
@@ -70,6 +70,7 @@ public class AlertBolt extends AbstractStreamBolt implements AlertBoltSpecListen
     private volatile Map<String, PolicyDefinition> cachedPolicies = new HashMap<>(); // for one streamGroup, there are multiple policies
 
     private AlertBoltSpec spec;
+
     public AlertBolt(String boltId, Config config, IMetadataChangeNotifyService changeNotifyService){
         super(boltId, changeNotifyService, config);
         this.boltId = boltId;

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/da7f5520/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/test/java/org/apache/eagle/alert/engine/router/CustomizedHandler.java
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/test/java/org/apache/eagle/alert/engine/router/CustomizedHandler.java b/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/test/java/org/apache/eagle/alert/engine/router/CustomizedHandler.java
new file mode 100644
index 0000000..be69ffb
--- /dev/null
+++ b/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/test/java/org/apache/eagle/alert/engine/router/CustomizedHandler.java
@@ -0,0 +1,51 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.eagle.alert.engine.router;
+
+import org.apache.eagle.alert.engine.Collector;
+import org.apache.eagle.alert.engine.coordinator.StreamDefinition;
+import org.apache.eagle.alert.engine.evaluator.PolicyHandlerContext;
+import org.apache.eagle.alert.engine.evaluator.PolicyStreamHandler;
+import org.apache.eagle.alert.engine.model.AlertStreamEvent;
+import org.apache.eagle.alert.engine.model.StreamEvent;
+
+import java.util.Map;
+
+/**
+ * Created on 8/29/16.
+ */
+public class CustomizedHandler implements PolicyStreamHandler {
+    private Collector<AlertStreamEvent> collector;
+
+    public CustomizedHandler(Map<String, StreamDefinition> sds) {
+    }
+
+    @Override
+    public void prepare(Collector<AlertStreamEvent> collector, PolicyHandlerContext context) throws Exception {
+        this.collector = collector;
+    }
+
+    @Override
+    public void send(StreamEvent event) throws Exception {
+        this.collector.emit(new AlertStreamEvent());
+    }
+
+    @Override
+    public void close() throws Exception {
+
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/da7f5520/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/test/java/org/apache/eagle/alert/engine/router/TestAlertBolt.java
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/test/java/org/apache/eagle/alert/engine/router/TestAlertBolt.java b/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/test/java/org/apache/eagle/alert/engine/router/TestAlertBolt.java
index 9d2bb38..f3548d8 100755
--- a/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/test/java/org/apache/eagle/alert/engine/router/TestAlertBolt.java
+++ b/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/test/java/org/apache/eagle/alert/engine/router/TestAlertBolt.java
@@ -41,13 +41,14 @@ import org.apache.eagle.alert.engine.runner.AlertBolt;
 import org.apache.eagle.alert.engine.runner.TestStreamRouterBolt;
 import org.apache.eagle.alert.engine.serialization.impl.PartitionedEventSerializerImpl;
 import org.apache.eagle.alert.utils.DateTimeUtil;
-import org.jetbrains.annotations.NotNull;
 import org.junit.Assert;
 import org.junit.Test;
 
+import java.io.IOException;
 import java.util.*;
 import java.util.concurrent.Semaphore;
 import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicInteger;
 
 import static org.mockito.Matchers.any;
@@ -59,6 +60,9 @@ import static org.mockito.Mockito.when;
  */
 @SuppressWarnings({"rawtypes", "unused"})
 public class TestAlertBolt {
+
+    public static final String TEST_STREAM = "test-stream";
+
     /**
      * Following knowledge is guaranteed in
      *
@@ -164,7 +168,6 @@ public class TestAlertBolt {
         bolt.cleanup();
     }
 
-    @NotNull
     public static AlertBolt createAlertBolt(OutputCollector collector) {
         Config config = ConfigFactory.load();
         PolicyGroupEvaluator policyGroupEvaluator = new PolicyGroupEvaluatorImpl("testPolicyGroupEvaluatorImpl");
@@ -260,6 +263,7 @@ public class TestAlertBolt {
         }
     }
 
+    //TODO: no data alert failed, need to check when no data alert merged.
     @Test
     public void testMetaversionConflict() throws Exception {
         AtomicInteger failedCount = new AtomicInteger();
@@ -283,25 +287,6 @@ public class TestAlertBolt {
         });
         AlertBolt bolt = createAlertBolt(collector);
 
-        GeneralTopologyContext context = mock(GeneralTopologyContext.class);
-        int taskId = 1;
-        when(context.getComponentId(taskId)).thenReturn("comp1");
-        when(context.getComponentOutputFields("comp1", "default")).thenReturn(new Fields("f0"));
-        // case 1: bolt prepared but metadata not initialized (no bolt.onAlertBoltSpecChange)
-        PartitionedEvent pe = new PartitionedEvent();
-        pe.setPartitionKey(1);
-        pe.setPartition(createPartition());
-        StreamEvent streamEvent = new StreamEvent();
-        streamEvent.setStreamId("test-stream");
-        streamEvent.setTimestamp(System.currentTimeMillis());
-        streamEvent.setMetaVersion("spec_version_"+System.currentTimeMillis());
-        pe.setEvent(streamEvent);
-
-        PartitionedEventSerializerImpl peSer = new PartitionedEventSerializerImpl(bolt);
-        byte[] serializedEvent = peSer.serialize(pe);
-        Tuple input = new TupleImpl(context, Collections.singletonList(serializedEvent), taskId, "default");
-
-
         Map<String, StreamDefinition> sds = new HashMap();
         StreamDefinition sdTest = new StreamDefinition();
         String streamId = "test-stream";
@@ -318,13 +303,16 @@ public class TestAlertBolt {
 
         PolicyDefinition.Definition definition = new PolicyDefinition.Definition();
         definition.setType(PolicyStreamHandlers.NO_DATA_ALERT_ENGINE);
-        definition.setValue("PT0M,plain,1,host,host1");
+        definition.setValue("PT0M,provided,1,host,host1");
         def.setDefinition(definition);
+        def.setPartitionSpec(Arrays.asList(createPartition()));
+        def.setOutputStreams(Arrays.asList("out"));
 
         boltSpecs.getBoltPoliciesMap().put(bolt.getBoltId(), Arrays.asList(def));
         bolt = createAlertBolt(collector);
         bolt.onAlertBoltSpecChange(boltSpecs, sds);
 
+        Tuple input = createTuple(bolt, boltSpecs.getVersion());
         bolt.execute(input);
 
         // Sleep 10s to wait thread in bolt.execute() to finish works
@@ -335,12 +323,87 @@ public class TestAlertBolt {
 
     }
 
-    @NotNull
+    private Tuple createTuple(AlertBolt bolt, String version) throws IOException {
+        GeneralTopologyContext context = mock(GeneralTopologyContext.class);
+        int taskId = 1;
+        when(context.getComponentId(taskId)).thenReturn("comp1");
+        when(context.getComponentOutputFields("comp1", "default")).thenReturn(new Fields("f0"));
+        // case 1: bolt prepared but metadata not initialized (no bolt.onAlertBoltSpecChange)
+        PartitionedEvent pe = new PartitionedEvent();
+        pe.setPartitionKey(1);
+        pe.setPartition(createPartition());
+        StreamEvent streamEvent = new StreamEvent();
+        streamEvent.setStreamId(TEST_STREAM);
+        streamEvent.setTimestamp(System.currentTimeMillis());
+        streamEvent.setMetaVersion(version);
+        pe.setEvent(streamEvent);
+
+        PartitionedEventSerializerImpl peSer = new PartitionedEventSerializerImpl(bolt);
+        byte[] serializedEvent = peSer.serialize(pe);
+        return new TupleImpl(context, Collections.singletonList(serializedEvent), taskId, "default");
+    }
+
     private StreamPartition createPartition() {
         StreamPartition sp = new StreamPartition();
+        sp.setStreamId(TEST_STREAM);
         sp.setType(StreamPartition.Type.GROUPBY);
         return sp;
     }
 
+    @Test
+    public void testExtendDefinition() throws IOException {
+        PolicyDefinition def = new PolicyDefinition();
+        def.setName("policy-definition");
+        def.setInputStreams(Arrays.asList(TEST_STREAM));
+
+        PolicyDefinition.Definition definition = new PolicyDefinition.Definition();
+        definition.setType(PolicyStreamHandlers.CUSTOMIZED_ENGINE);
+        definition.setHandlerClass("org.apache.eagle.alert.engine.router.CustomizedHandler");
+        definition.setValue("PT0M,plain,1,host,host1");
+        def.setDefinition(definition);
+        def.setPartitionSpec(Arrays.asList(createPartition()));
+
+        AlertBoltSpec boltSpecs = new AlertBoltSpec();
+
+        AtomicBoolean recieved = new AtomicBoolean(false);
+        OutputCollector collector = new OutputCollector(new IOutputCollector() {
+            @Override
+            public List<Integer> emit(String streamId, Collection<Tuple> anchors, List<Object> tuple) {
+                recieved.set(true);
+                return Collections.emptyList();
+            }
+
+            @Override
+            public void emitDirect(int taskId, String streamId, Collection<Tuple> anchors, List<Object> tuple) {}
+
+            @Override
+            public void ack(Tuple input) {}
+
+            @Override
+            public void fail(Tuple input) {}
+
+            @Override
+            public void reportError(Throwable error) {}
+        });
+        AlertBolt bolt = createAlertBolt(collector);
+
+        boltSpecs.getBoltPoliciesMap().put(bolt.getBoltId(), Arrays.asList(def));
+        boltSpecs.setVersion("spec_"+System.currentTimeMillis());
+        // stream def map
+        Map<String, StreamDefinition> sds = new HashMap();
+        StreamDefinition sdTest = new StreamDefinition();
+        sdTest.setStreamId(TEST_STREAM);
+        sds.put(sdTest.getStreamId(), sdTest);
+
+        bolt.onAlertBoltSpecChange(boltSpecs, sds);
+
+        // how to assert
+        Tuple t = createTuple(bolt, boltSpecs.getVersion());
+
+        bolt.execute(t);
+
+        Assert.assertTrue(recieved.get());
+    }
 
 }
+


[27/52] [abbrv] incubator-eagle git commit: [EAGLE-514] Add two job count apis

Posted by yo...@apache.org.
[EAGLE-514] Add two job count apis

https://issues.apache.org/jira/browse/EAGLE-514

1. adding two job counting apis

2. add tracking url in running/history job execution entity

3. unify the status presentation for job/task execution entity

4. unify the name of the common fields between running job entity & history job entity

Author: Qingwen Zhao <qi...@gmail.com>

Closes #408 from qingwen220/jobStats.


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

Branch: refs/heads/master
Commit: a66f64cf9f4212f4923f0e6ea6c7270449aa2ce4
Parents: 4aa5b45
Author: Qingwen Zhao <qi...@gmail.com>
Authored: Wed Aug 31 14:21:22 2016 +0800
Committer: Qingwen Zhao <qi...@gmail.com>
Committed: Wed Aug 31 14:21:22 2016 +0800

----------------------------------------------------------------------
 .../mr/historyentity/JobExecutionAPIEntity.java |  11 +
 .../mr/runningentity/JobExecutionAPIEntity.java |  36 ++-
 .../jpm/mr/history/MRHistoryJobConfig.java      |   2 +
 .../history/crawler/JHFCrawlerDriverImpl.java   |   2 +-
 .../jpm/mr/history/parser/EagleJobStatus.java   |   2 +-
 .../mr/history/parser/JHFEventReaderBase.java   |  21 +-
 .../mr/history/parser/JHFMRVer1EventReader.java |   5 +-
 .../mr/history/parser/JHFMRVer2EventReader.java |  15 +-
 .../jpm/mr/history/parser/JHFParserFactory.java |   4 +-
 .../JobEntityCreationEagleServiceListener.java  |   6 +-
 .../src/main/resources/application.conf         |   1 +
 .../jpm/mr/running/parser/MRJobParser.java      |   5 +-
 .../eagle/service/jpm/MRJobCountHelper.java     | 121 ++++++++
 .../service/jpm/MRJobExecutionResource.java     | 286 +++++++++++++------
 .../service/jpm/MRJobTaskCountResponse.java     |  65 +++++
 .../service/jpm/MRJobTaskGroupResponse.java     |  41 ---
 .../service/jpm/TaskCountByDurationHelper.java  | 106 +++++++
 .../jpm/TestJobCountPerBucketHelper.java        |  87 ++++++
 .../service/jpm/TestMRJobExecutionResource.java |  99 -------
 .../service/jpm/TestTaskCountPerJobHelper.java  |  96 +++++++
 .../org/apache/eagle/jpm/util/Constants.java    |   2 +
 21 files changed, 752 insertions(+), 261 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/a66f64cf/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/mr/historyentity/JobExecutionAPIEntity.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/mr/historyentity/JobExecutionAPIEntity.java b/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/mr/historyentity/JobExecutionAPIEntity.java
index 97e77b2..cdc5810 100644
--- a/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/mr/historyentity/JobExecutionAPIEntity.java
+++ b/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/mr/historyentity/JobExecutionAPIEntity.java
@@ -89,6 +89,17 @@ public class JobExecutionAPIEntity extends JobBaseAPIEntity {
     private int totalReduceAttempts;
     @Column("ac")
     private int failedReduceAttempts;
+    @Column("ad")
+    private String trackingUrl;
+
+    public String getTrackingUrl() {
+        return trackingUrl;
+    }
+
+    public void setTrackingUrl(String trackingUrl) {
+        this.trackingUrl = trackingUrl;
+        valueChanged("trackingUrl");
+    }
 
     public long getDurationTime() {
         return durationTime;

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/a66f64cf/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/mr/runningentity/JobExecutionAPIEntity.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/mr/runningentity/JobExecutionAPIEntity.java b/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/mr/runningentity/JobExecutionAPIEntity.java
index dd81eb4..245fc0f 100644
--- a/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/mr/runningentity/JobExecutionAPIEntity.java
+++ b/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/mr/runningentity/JobExecutionAPIEntity.java
@@ -49,11 +49,11 @@ public class JobExecutionAPIEntity extends TaggedLogAPIEntity {
     @Column("e")
     private int numTotalMaps;
     @Column("f")
-    private int mapsCompleted;
+    private int numFinishedMaps;
     @Column("g")
     private int numTotalReduces;
     @Column("h")
-    private int reducesCompleted;
+    private int numFinishedReduces;
     @Column("i")
     private double mapProgress;
     @Column("j")
@@ -112,6 +112,18 @@ public class JobExecutionAPIEntity extends TaggedLogAPIEntity {
     private long submissionTime;
     @Column("ak")
     private String internalState;
+    @Column("al")
+    private String trackingUrl;
+
+    public String getTrackingUrl() {
+        return trackingUrl;
+    }
+
+    public void setTrackingUrl(String trackingUrl) {
+        this.trackingUrl = trackingUrl;
+        valueChanged("trackingUrl");
+    }
+
 
     public JobConfig getJobConfig() {
         return jobConfig;
@@ -176,13 +188,13 @@ public class JobExecutionAPIEntity extends TaggedLogAPIEntity {
         valueChanged("numTotalMaps");
     }
 
-    public int getMapsCompleted() {
-        return mapsCompleted;
+    public int getNumFinishedMaps() {
+        return numFinishedMaps;
     }
 
-    public void setMapsCompleted(int mapsCompleted) {
-        this.mapsCompleted = mapsCompleted;
-        valueChanged("mapsCompleted");
+    public void setNumFinishedMaps(int numFinishedMaps) {
+        this.numFinishedMaps = numFinishedMaps;
+        valueChanged("numFinishedMaps");
     }
 
     public int getNumTotalReduces() {
@@ -194,13 +206,13 @@ public class JobExecutionAPIEntity extends TaggedLogAPIEntity {
         valueChanged("numTotalReduces");
     }
 
-    public int getReducesCompleted() {
-        return reducesCompleted;
+    public int getNumFinishedReduces() {
+        return numFinishedReduces;
     }
 
-    public void setReducesCompleted(int reducesCompleted) {
-        this.reducesCompleted = reducesCompleted;
-        valueChanged("reducesCompleted");
+    public void setNumFinishedReduces(int numFinishedReduces) {
+        this.numFinishedReduces = numFinishedReduces;
+        valueChanged("numFinishedReduces");
     }
 
     public double getMapProgress() {

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/a66f64cf/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/MRHistoryJobConfig.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/MRHistoryJobConfig.java b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/MRHistoryJobConfig.java
index ae86904..c0943de 100644
--- a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/MRHistoryJobConfig.java
+++ b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/MRHistoryJobConfig.java
@@ -86,6 +86,7 @@ public class MRHistoryJobConfig implements Serializable {
 
     public static class JobHistoryEndpointConfig implements Serializable {
         public String nnEndpoint;
+        public String mrHistoryServerUrl;
         public String basePath;
         public boolean pathContainsJobTrackerName;
         public String jobTrackerName;
@@ -173,6 +174,7 @@ public class MRHistoryJobConfig implements Serializable {
         this.jobHistoryEndpointConfig.basePath = config.getString("dataSourceConfig.basePath");
         this.jobHistoryEndpointConfig.jobTrackerName = config.getString("dataSourceConfig.jobTrackerName");
         this.jobHistoryEndpointConfig.nnEndpoint = config.getString("dataSourceConfig.nnEndpoint");
+        this.jobHistoryEndpointConfig.mrHistoryServerUrl = config.getString("dataSourceConfig.mrHistoryServerUrl");
         this.jobHistoryEndpointConfig.pathContainsJobTrackerName = config.getBoolean("dataSourceConfig.pathContainsJobTrackerName");
         this.jobHistoryEndpointConfig.principal = config.getString("dataSourceConfig.principal");
         this.jobHistoryEndpointConfig.keyTab = config.getString("dataSourceConfig.keytab");

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/a66f64cf/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/crawler/JHFCrawlerDriverImpl.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/crawler/JHFCrawlerDriverImpl.java b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/crawler/JHFCrawlerDriverImpl.java
index e16ecce..1a17751 100644
--- a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/crawler/JHFCrawlerDriverImpl.java
+++ b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/crawler/JHFCrawlerDriverImpl.java
@@ -247,7 +247,7 @@ public class JHFCrawlerDriverImpl implements JHFCrawlerDriver {
         JobCountEntity entity = new JobCountEntity();
         entity.setTotal(jobs.size());
         entity.setFail(0);
-        jobs.stream().filter(job -> !job.getRight().equals(EagleJobStatus.SUCCESS.toString())).forEach(
+        jobs.stream().filter(job -> !job.getRight().equals(EagleJobStatus.SUCCEEDED.toString())).forEach(
             job -> entity.setFail(1 + entity.getFail())
         );
 

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/a66f64cf/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/EagleJobStatus.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/EagleJobStatus.java b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/EagleJobStatus.java
index fb218e3..24fa097 100644
--- a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/EagleJobStatus.java
+++ b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/EagleJobStatus.java
@@ -23,7 +23,7 @@ public enum EagleJobStatus {
     LAUNCHED,
     PREP,
     RUNNING,
-    SUCCESS,
+    SUCCEEDED,
     KILLED,
     FAILED;
 }

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/a66f64cf/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JHFEventReaderBase.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JHFEventReaderBase.java b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JHFEventReaderBase.java
index 6916aad..1570956 100644
--- a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JHFEventReaderBase.java
+++ b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JHFEventReaderBase.java
@@ -18,6 +18,8 @@
 
 package org.apache.eagle.jpm.mr.history.parser;
 
+import org.apache.commons.io.FilenameUtils;
+import org.apache.eagle.jpm.mr.history.MRHistoryJobConfig.JobHistoryEndpointConfig;
 import org.apache.eagle.jpm.mr.history.crawler.JobHistoryContentFilter;
 import org.apache.eagle.jpm.mr.history.parser.JHFMRVer1Parser.Keys;
 import org.apache.eagle.jpm.mr.historyentity.*;
@@ -32,6 +34,8 @@ import org.slf4j.LoggerFactory;
 
 import java.io.Closeable;
 import java.io.IOException;
+import java.net.URI;
+import java.net.URISyntaxException;
 import java.util.*;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
@@ -65,6 +69,7 @@ public abstract class JHFEventReaderBase extends JobEntityCreationPublisher impl
     protected String queueName;
     protected Long jobLaunchTime;
     protected JobHistoryContentFilter filter;
+    private JobHistoryEndpointConfig jobHistoryEndpointConfig;
 
     protected final List<HistoryJobEntityLifecycleListener> jobEntityLifecycleListeners = new ArrayList<>();
 
@@ -96,8 +101,9 @@ public abstract class JHFEventReaderBase extends JobEntityCreationPublisher impl
      *
      * @param baseTags
      */
-    public JHFEventReaderBase(Map<String, String> baseTags, Configuration configuration, JobHistoryContentFilter filter) {
+    public JHFEventReaderBase(Map<String, String> baseTags, Configuration configuration, JobHistoryContentFilter filter, JobHistoryEndpointConfig jobHistoryEndpointConfig) {
         this.filter = filter;
+        this.jobHistoryEndpointConfig = jobHistoryEndpointConfig;
 
         this.baseTags = baseTags;
         jobSubmitEventEntity = new JobEventAPIEntity();
@@ -155,6 +161,18 @@ public abstract class JHFEventReaderBase extends JobEntityCreationPublisher impl
         }
     }
 
+    private String buildJobTrackingUrl(String jobId) {
+        String jobTrackingUrlBase = this.jobHistoryEndpointConfig.mrHistoryServerUrl + "/jobhistory/job/";
+        try {
+            URI oldUri = new URI(jobTrackingUrlBase);
+            URI resolved = oldUri.resolve(jobId);
+            return resolved.toString();
+        } catch (URISyntaxException e) {
+            LOG.warn("Tracking url build failed with baseURL=%s, resolvePart=%s", jobTrackingUrlBase, jobId);
+            return jobTrackingUrlBase;
+        }
+    }
+
     /**
      * ...
      * @param id
@@ -236,6 +254,7 @@ public abstract class JHFEventReaderBase extends JobEntityCreationPublisher impl
             jobExecutionEntity.getTags().put(MRJobTagName.JOB_QUEUE.toString(), queueName);
             jobExecutionEntity.getTags().put(MRJobTagName.JOB_TYPE.toString(), this.jobType);
 
+            jobExecutionEntity.setTrackingUrl(buildJobTrackingUrl(jobId));
             jobExecutionEntity.setCurrentState(values.get(Keys.JOB_STATUS));
             jobExecutionEntity.setStartTime(jobLaunchEventEntity.getTimestamp());
             jobExecutionEntity.setEndTime(jobFinishEventEntity.getTimestamp());

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/a66f64cf/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JHFMRVer1EventReader.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JHFMRVer1EventReader.java b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JHFMRVer1EventReader.java
index e20836f..0e9458a 100644
--- a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JHFMRVer1EventReader.java
+++ b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JHFMRVer1EventReader.java
@@ -18,6 +18,7 @@
 
 package org.apache.eagle.jpm.mr.history.parser;
 
+import org.apache.eagle.jpm.mr.history.MRHistoryJobConfig.JobHistoryEndpointConfig;
 import org.apache.eagle.jpm.mr.history.crawler.JobHistoryContentFilter;
 import org.apache.eagle.jpm.mr.history.parser.JHFMRVer1Parser.Keys;
 import org.apache.eagle.jpm.mr.historyentity.JobExecutionAPIEntity;
@@ -48,8 +49,8 @@ public class JHFMRVer1EventReader extends JHFEventReaderBase implements JHFMRVer
      *
      * @param baseTags
      */
-    public JHFMRVer1EventReader(Map<String, String> baseTags, Configuration configuration, JobHistoryContentFilter filter) {
-        super(baseTags, configuration, filter);
+    public JHFMRVer1EventReader(Map<String, String> baseTags, Configuration configuration, JobHistoryContentFilter filter, JobHistoryEndpointConfig jobHistoryEndpointConfig) {
+        super(baseTags, configuration, filter, jobHistoryEndpointConfig);
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/a66f64cf/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JHFMRVer2EventReader.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JHFMRVer2EventReader.java b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JHFMRVer2EventReader.java
index f21fd41..74f84f6 100644
--- a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JHFMRVer2EventReader.java
+++ b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JHFMRVer2EventReader.java
@@ -18,6 +18,7 @@
 
 package org.apache.eagle.jpm.mr.history.parser;
 
+import org.apache.eagle.jpm.mr.history.MRHistoryJobConfig.JobHistoryEndpointConfig;
 import org.apache.eagle.jpm.mr.history.crawler.JobHistoryContentFilter;
 import org.apache.eagle.jpm.mr.history.parser.JHFMRVer1Parser.Keys;
 import org.apache.eagle.jpm.util.jobcounter.JobCounters;
@@ -43,8 +44,8 @@ public class JHFMRVer2EventReader extends JHFEventReaderBase {
      *
      * @throws IOException
      */
-    public JHFMRVer2EventReader(Map<String, String> baseTags, Configuration configuration, JobHistoryContentFilter filter) {
-        super(baseTags, configuration, filter);
+    public JHFMRVer2EventReader(Map<String, String> baseTags, Configuration configuration, JobHistoryContentFilter filter, JobHistoryEndpointConfig jobHistoryEndpointConfig) {
+        super(baseTags, configuration, filter, jobHistoryEndpointConfig);
     }
 
     @SuppressWarnings("deprecation")
@@ -233,7 +234,7 @@ public class JHFMRVer2EventReader extends JHFEventReaderBase {
         if (js.getFailedReduces() != null) {
             values.put(Keys.FAILED_REDUCES, js.getFailedReduces().toString());
         }
-        values.put(Keys.JOB_STATUS, EagleJobStatus.SUCCESS.name());
+        values.put(Keys.JOB_STATUS, EagleJobStatus.SUCCEEDED.name());
         handleJob(wrapper.getType(), values, js.getTotalCounters());
     }
 
@@ -289,7 +290,7 @@ public class JHFMRVer2EventReader extends JHFEventReaderBase {
             values.put(Keys.FINISH_TIME, js.getFinishTime().toString());
         }
         if (js.getStatus() != null) {
-            values.put(Keys.TASK_STATUS, normalizeTaskStatus(js.getStatus().toString()));
+            values.put(Keys.TASK_STATUS, js.getStatus().toString());
         }
         handleTask(RecordTypes.Task, wrapper.getType(), values, js.getCounters());
     }
@@ -308,7 +309,7 @@ public class JHFMRVer2EventReader extends JHFEventReaderBase {
             values.put(Keys.FINISH_TIME, js.getFinishTime().toString());
         }
         if (js.getStatus() != null) {
-            values.put(Keys.TASK_STATUS, normalizeTaskStatus(js.getStatus().toString()));
+            values.put(Keys.TASK_STATUS, js.getStatus().toString());
         }
         if (js.getError() != null) {
             values.put(Keys.ERROR, js.getError().toString());
@@ -381,7 +382,7 @@ public class JHFMRVer2EventReader extends JHFEventReaderBase {
             values.put(Keys.TASK_TYPE, js.getTaskType().toString());
         }
         if (js.getTaskStatus() != null) {
-            values.put(Keys.TASK_STATUS, normalizeTaskStatus(js.getTaskStatus().toString()));
+            values.put(Keys.TASK_STATUS, js.getTaskStatus().toString());
         }
         if (js.getAttemptId() != null) {
             values.put(Keys.TASK_ATTEMPT_ID, js.getAttemptId().toString());
@@ -419,7 +420,7 @@ public class JHFMRVer2EventReader extends JHFEventReaderBase {
             values.put(Keys.TASK_TYPE, js.getTaskType().toString());
         }
         if (js.getTaskStatus() != null) {
-            values.put(Keys.TASK_STATUS, normalizeTaskStatus(js.getTaskStatus().toString()));
+            values.put(Keys.TASK_STATUS, js.getTaskStatus().toString());
         }
         if (js.getAttemptId() != null) {
             values.put(Keys.TASK_ATTEMPT_ID, js.getAttemptId().toString());

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/a66f64cf/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JHFParserFactory.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JHFParserFactory.java b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JHFParserFactory.java
index 718612d..386d50c 100755
--- a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JHFParserFactory.java
+++ b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JHFParserFactory.java
@@ -46,7 +46,7 @@ public class JHFParserFactory {
 
         switch (f) {
             case MRVer2:
-                JHFMRVer2EventReader reader2 = new JHFMRVer2EventReader(baseTags, configuration, filter);
+                JHFMRVer2EventReader reader2 = new JHFMRVer2EventReader(baseTags, configuration, filter, configManager.getJobHistoryEndpointConfig());
                 reader2.addListener(new JobEntityCreationEagleServiceListener(configManager));
                 reader2.addListener(new TaskFailureListener(configManager));
                 reader2.addListener(new TaskAttemptCounterListener(configManager));
@@ -57,7 +57,7 @@ public class JHFParserFactory {
                 break;
             case MRVer1:
             default:
-                JHFMRVer1EventReader reader1 = new JHFMRVer1EventReader(baseTags, configuration, filter);
+                JHFMRVer1EventReader reader1 = new JHFMRVer1EventReader(baseTags, configuration, filter, configManager.getJobHistoryEndpointConfig());
                 reader1.addListener(new JobEntityCreationEagleServiceListener(configManager));
                 reader1.addListener(new TaskFailureListener(configManager));
                 reader1.addListener(new TaskAttemptCounterListener(configManager));

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/a66f64cf/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JobEntityCreationEagleServiceListener.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JobEntityCreationEagleServiceListener.java b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JobEntityCreationEagleServiceListener.java
index a681aca..520fbbc 100644
--- a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JobEntityCreationEagleServiceListener.java
+++ b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JobEntityCreationEagleServiceListener.java
@@ -43,7 +43,6 @@ public class JobEntityCreationEagleServiceListener implements HistoryJobEntityCr
     List<JobEventAPIEntity> jobEvents = new ArrayList<>();
     List<TaskExecutionAPIEntity> taskExecs = new ArrayList<>();
     List<TaskAttemptExecutionAPIEntity> taskAttemptExecs = new ArrayList<>();
-    private JobHistoryZKStateManager zkState;
     private TimeZone timeZone;
 
     public JobEntityCreationEagleServiceListener(MRHistoryJobConfig configManager) {
@@ -56,7 +55,6 @@ public class JobEntityCreationEagleServiceListener implements HistoryJobEntityCr
             throw new IllegalArgumentException("batchSize must be greater than 0 when it is provided");
         }
         this.batchSize = batchSize;
-        zkState = new JobHistoryZKStateManager(configManager.getZkStateConfig());
         timeZone = TimeZone.getTimeZone(configManager.getControlConfig().timeZone);
     }
 
@@ -92,12 +90,13 @@ public class JobEntityCreationEagleServiceListener implements HistoryJobEntityCr
             eagleServiceConfig.password);
 
         client.getJerseyClient().setReadTimeout(jobExtractorConfig.readTimeoutSeconds * 1000);
+        JobHistoryZKStateManager zkState = new JobHistoryZKStateManager(configManager.getZkStateConfig());
         logger.info("start flushing entities of total number " + list.size());
         for (int i = 0; i < list.size(); i++) {
             JobBaseAPIEntity entity = list.get(i);
             if (entity instanceof JobExecutionAPIEntity) {
                 jobs.add((JobExecutionAPIEntity) entity);
-                this.zkState.updateProcessedJob(timeStamp2Date(entity.getTimestamp()),
+                zkState.updateProcessedJob(timeStamp2Date(entity.getTimestamp()),
                     entity.getTags().get(MRJobTagName.JOB_ID.toString()),
                     ((JobExecutionAPIEntity) entity).getCurrentState());
             } else if (entity instanceof JobEventAPIEntity) {
@@ -108,6 +107,7 @@ public class JobEntityCreationEagleServiceListener implements HistoryJobEntityCr
                 taskAttemptExecs.add((TaskAttemptExecutionAPIEntity) entity);
             }
         }
+        zkState.close();
         GenericServiceAPIResponseEntity result;
         if (jobs.size() > 0) {
             logger.info("flush JobExecutionAPIEntity of number " + jobs.size());

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/a66f64cf/eagle-jpm/eagle-jpm-mr-history/src/main/resources/application.conf
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-mr-history/src/main/resources/application.conf b/eagle-jpm/eagle-jpm-mr-history/src/main/resources/application.conf
index db2c716..de874a6 100644
--- a/eagle-jpm/eagle-jpm-mr-history/src/main/resources/application.conf
+++ b/eagle-jpm/eagle-jpm-mr-history/src/main/resources/application.conf
@@ -40,6 +40,7 @@
     "zkRetryTimes" : 3,
     "zkRetryInterval" : 20000,
     "nnEndpoint" : "hdfs://sandbox.hortonworks.com:8020",
+    "mrHistoryServerUrl" : "http://sandbox.hortonworks.com:19888",
     "principal":"", #if not need, then empty
     "keytab":"",
     "basePath" : "/mr-history/done",

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/a66f64cf/eagle-jpm/eagle-jpm-mr-running/src/main/java/org/apache/eagle/jpm/mr/running/parser/MRJobParser.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-mr-running/src/main/java/org/apache/eagle/jpm/mr/running/parser/MRJobParser.java b/eagle-jpm/eagle-jpm-mr-running/src/main/java/org/apache/eagle/jpm/mr/running/parser/MRJobParser.java
index 9e156fa..5811f72 100644
--- a/eagle-jpm/eagle-jpm-mr-running/src/main/java/org/apache/eagle/jpm/mr/running/parser/MRJobParser.java
+++ b/eagle-jpm/eagle-jpm-mr-running/src/main/java/org/apache/eagle/jpm/mr/running/parser/MRJobParser.java
@@ -207,14 +207,15 @@ public class MRJobParser implements Runnable {
             }
             jobExecutionAPIEntity.setTimestamp(app.getStartedTime());
             jobExecutionAPIEntity.setSubmissionTime(app.getStartedTime());
+            jobExecutionAPIEntity.setTrackingUrl(app.getTrackingUrl());
             jobExecutionAPIEntity.setStartTime(mrJob.getStartTime());
             jobExecutionAPIEntity.setDurationTime(mrJob.getElapsedTime());
             jobExecutionAPIEntity.setCurrentState(mrJob.getState());
             jobExecutionAPIEntity.setInternalState(mrJob.getState());
             jobExecutionAPIEntity.setNumTotalMaps(mrJob.getMapsTotal());
-            jobExecutionAPIEntity.setMapsCompleted(mrJob.getMapsCompleted());
+            jobExecutionAPIEntity.setNumFinishedMaps(mrJob.getMapsCompleted());
             jobExecutionAPIEntity.setNumTotalReduces(mrJob.getReducesTotal());
-            jobExecutionAPIEntity.setReducesCompleted(mrJob.getReducesCompleted());
+            jobExecutionAPIEntity.setNumFinishedReduces(mrJob.getReducesCompleted());
             jobExecutionAPIEntity.setMapProgress(mrJob.getMapProgress());
             jobExecutionAPIEntity.setReduceProgress(mrJob.getReduceProgress());
             jobExecutionAPIEntity.setMapsPending(mrJob.getMapsPending());

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/a66f64cf/eagle-jpm/eagle-jpm-service/src/main/java/org/apache/eagle/service/jpm/MRJobCountHelper.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-service/src/main/java/org/apache/eagle/service/jpm/MRJobCountHelper.java b/eagle-jpm/eagle-jpm-service/src/main/java/org/apache/eagle/service/jpm/MRJobCountHelper.java
new file mode 100644
index 0000000..93c6c00
--- /dev/null
+++ b/eagle-jpm/eagle-jpm-service/src/main/java/org/apache/eagle/service/jpm/MRJobCountHelper.java
@@ -0,0 +1,121 @@
+/*
+ *  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.eagle.service.jpm;
+
+import org.apache.eagle.common.DateTimeUtil;
+import org.apache.eagle.jpm.mr.historyentity.JobExecutionAPIEntity;
+import org.apache.eagle.jpm.util.MRJobTagName;
+import org.apache.eagle.service.jpm.MRJobTaskCountResponse.JobCountResponse;
+import org.apache.eagle.service.jpm.MRJobTaskCountResponse.UnitJobCount;
+
+import java.text.ParseException;
+import java.util.ArrayList;
+import java.util.List;
+
+public class MRJobCountHelper {
+
+    public void initJobCountList(List<UnitJobCount> jobCounts, long startTime, long endTime, long intervalInSecs) {
+        for (long i = startTime / intervalInSecs; i * intervalInSecs <= endTime; i++) {
+            jobCounts.add(new UnitJobCount(i * intervalInSecs));
+        }
+    }
+
+    public String moveTimeforwardOneDay(String startTime) throws ParseException {
+        long timeInSecs = DateTimeUtil.humanDateToSeconds(startTime);
+        timeInSecs -= 24L * 60L * 60L;
+        return DateTimeUtil.secondsToHumanDate(timeInSecs);
+    }
+
+    public JobCountResponse getRunningJobCount(List<JobExecutionAPIEntity> jobDurations,
+                                        long startTimeInSecs,
+                                        long endTimeInSecs,
+                                        long intervalInSecs) {
+        JobCountResponse response = new JobCountResponse();
+        List<UnitJobCount> jobCounts = new ArrayList<>();
+        initJobCountList(jobCounts, startTimeInSecs, endTimeInSecs, intervalInSecs);
+        for (JobExecutionAPIEntity jobDuration: jobDurations) {
+            countJob(jobCounts, jobDuration.getStartTime() / 1000, jobDuration.getEndTime() / 1000, intervalInSecs, jobDuration.getTags().get(MRJobTagName.JOB_TYPE.toString()));
+        }
+        response.jobCounts = jobCounts;
+        return response;
+    }
+
+    public JobCountResponse getHistoryJobCount(List<JobExecutionAPIEntity> jobDurations, String timeList) {
+        JobCountResponse response = new JobCountResponse();
+        List<UnitJobCount> jobCounts = new ArrayList<>();
+        List<Long> times = TaskCountByDurationHelper.parseTimeList(timeList);
+        for (int i = 0; i < times.size(); i++) {
+            jobCounts.add(new UnitJobCount(times.get(i)));
+        }
+        for (JobExecutionAPIEntity job : jobDurations) {
+            int jobIndex = TaskCountByDurationHelper.getPosition(times, job.getDurationTime());
+            UnitJobCount counter = jobCounts.get(jobIndex);
+            countJob(counter, job.getTags().get(MRJobTagName.JOB_TYPE.toString()));
+        }
+        response.jobCounts = jobCounts;
+        return response;
+    }
+
+    public void countJob(UnitJobCount counter, String jobType) {
+        if (null  ==  jobType) {
+            jobType = "null";
+        }
+        counter.jobCount++;
+        if (counter.jobCountByType.containsKey(jobType)) {
+            counter.jobCountByType.put(jobType, counter.jobCountByType.get(jobType) + 1);
+        } else {
+            counter.jobCountByType.put(jobType, 1L);
+        }
+    }
+
+    public void countJob(List<UnitJobCount> jobCounts, long jobStartTimeSecs, long jobEndTimeSecs, long intervalInSecs, String jobType) {
+        long startCountPoint = jobCounts.get(0).timeBucket;
+        if (jobEndTimeSecs < startCountPoint) {
+            return;
+        }
+        int startIndex = 0;
+        if (jobStartTimeSecs > startCountPoint) {
+            long relativeStartTime = jobStartTimeSecs - startCountPoint;
+            startIndex = (int) (relativeStartTime / intervalInSecs) + (relativeStartTime % intervalInSecs == 0 ? 0 : 1);
+        }
+        long relativeEndTime = jobEndTimeSecs - startCountPoint;
+        int endIndex = (int) (relativeEndTime / intervalInSecs);
+
+        for (int i = startIndex; i <= endIndex && i < jobCounts.size(); i++) {
+            countJob(jobCounts.get(i), jobType);
+        }
+    }
+
+    public List<String> getSearchTimeDuration(List<JobExecutionAPIEntity> jobEntities) {
+        List<String> pair = new ArrayList<>();
+        long minStartTime = System.currentTimeMillis();
+        long maxEndTime = 0;
+        for (JobExecutionAPIEntity jobEntity : jobEntities) {
+            if (minStartTime > jobEntity.getStartTime()) {
+                minStartTime = jobEntity.getStartTime();
+            }
+            if (maxEndTime < jobEntity.getEndTime()) {
+                maxEndTime = jobEntity.getEndTime();
+            }
+        }
+        pair.add(DateTimeUtil.millisecondsToHumanDateWithSeconds(minStartTime));
+        pair.add(DateTimeUtil.millisecondsToHumanDateWithSeconds(maxEndTime));
+        return pair;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/a66f64cf/eagle-jpm/eagle-jpm-service/src/main/java/org/apache/eagle/service/jpm/MRJobExecutionResource.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-service/src/main/java/org/apache/eagle/service/jpm/MRJobExecutionResource.java b/eagle-jpm/eagle-jpm-service/src/main/java/org/apache/eagle/service/jpm/MRJobExecutionResource.java
index 3e487ae..5af9811 100644
--- a/eagle-jpm/eagle-jpm-service/src/main/java/org/apache/eagle/service/jpm/MRJobExecutionResource.java
+++ b/eagle-jpm/eagle-jpm-service/src/main/java/org/apache/eagle/service/jpm/MRJobExecutionResource.java
@@ -18,19 +18,25 @@
 
 package org.apache.eagle.service.jpm;
 
-
 import static org.apache.eagle.jpm.util.MRJobTagName.JOB_ID;
+import static org.apache.eagle.jpm.util.MRJobTagName.TASK_TYPE;
 
+import org.apache.eagle.common.DateTimeUtil;
+import org.apache.eagle.jpm.mr.historyentity.JobExecutionAPIEntity;
 import org.apache.eagle.jpm.mr.runningentity.TaskExecutionAPIEntity;
 import org.apache.eagle.jpm.util.Constants;
 import org.apache.eagle.log.base.taggedlog.TaggedLogAPIEntity;
 import org.apache.eagle.log.entity.GenericServiceAPIResponseEntity;
 import org.apache.eagle.service.generic.GenericEntityServiceResource;
+import org.apache.eagle.service.generic.ListQueryResource;
+import org.apache.eagle.service.jpm.MRJobTaskCountResponse.JobCountResponse;
+import org.apache.eagle.service.jpm.MRJobTaskCountResponse.TaskCountPerJobResponse;
 
 import org.apache.commons.lang.time.StopWatch;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import java.text.ParseException;
 import java.util.*;
 import javax.ws.rs.*;
 import javax.ws.rs.core.MediaType;
@@ -59,14 +65,14 @@ public class MRJobExecutionResource {
         List<TaggedLogAPIEntity> jobs = new ArrayList<>();
         List<TaggedLogAPIEntity> finishedJobs = new ArrayList<>();
         Set<String> jobIds = new HashSet<>();
-        final Map<String,Object> meta = new HashMap<>();
+        final Map<String, Object> meta = new HashMap<>();
         StopWatch stopWatch = new StopWatch();
 
         stopWatch.start();
         String jobQuery = String.format(query, Constants.JPA_JOB_EXECUTION_SERVICE_NAME);
         GenericServiceAPIResponseEntity<TaggedLogAPIEntity> res =
-                resource.search(jobQuery, startTime, endTime, pageSize, startRowkey, treeAgg, timeSeries, intervalmin,
-                top,filterIfMissing, parallel, metricName, verbose);
+            resource.search(jobQuery, startTime, endTime, pageSize, startRowkey, treeAgg, timeSeries, intervalmin,
+                top, filterIfMissing, parallel, metricName, verbose);
         if (res.isSuccess() && res.getObj() != null) {
             for (TaggedLogAPIEntity o : res.getObj()) {
                 finishedJobs.add(o);
@@ -74,10 +80,10 @@ public class MRJobExecutionResource {
             }
             jobQuery = String.format(query, Constants.JPA_RUNNING_JOB_EXECUTION_SERVICE_NAME);
             res = resource.search(jobQuery, startTime, endTime, pageSize, startRowkey, treeAgg, timeSeries, intervalmin,
-                    top,filterIfMissing, parallel, metricName, verbose);
+                top, filterIfMissing, parallel, metricName, verbose);
             if (res.isSuccess() && res.getObj() != null) {
                 for (TaggedLogAPIEntity o : res.getObj()) {
-                    if (! isDuplicate(jobIds, o)) {
+                    if (!isDuplicate(jobIds, o)) {
                         jobs.add(o);
                     }
                 }
@@ -92,7 +98,7 @@ public class MRJobExecutionResource {
             response.setException(new Exception(res.getException()));
         }
         meta.put(TOTAL_RESULTS, jobs.size());
-        meta.put(ELAPSEDMS,stopWatch.getTime());
+        meta.put(ELAPSEDMS, stopWatch.getTime());
         response.setObj(jobs);
         response.setMeta(meta);
         return response;
@@ -107,7 +113,7 @@ public class MRJobExecutionResource {
     }
 
     private String buildCondition(String jobId, String jobDefId, String site) {
-        String conditionFormat = "@site=\"%s\"" ;
+        String conditionFormat = "@site=\"%s\"";
         String condition = null;
         if (jobDefId != null) {
             conditionFormat = conditionFormat + " AND @jobDefId=\"%s\"";
@@ -144,12 +150,12 @@ public class MRJobExecutionResource {
         }
         LOG.debug("search condition=" + condition);
 
-        final Map<String,Object> meta = new HashMap<>();
+        final Map<String, Object> meta = new HashMap<>();
         StopWatch stopWatch = new StopWatch();
         stopWatch.start();
         String queryFormat = "%s[%s]{*}";
         String queryString = String.format(queryFormat, Constants.JPA_JOB_EXECUTION_SERVICE_NAME, condition);
-        GenericServiceAPIResponseEntity<TaggedLogAPIEntity> res = resource.search(queryString, null, null, pageSize, null, false, true,  0L, 0, true, 0, null, false);
+        GenericServiceAPIResponseEntity<TaggedLogAPIEntity> res = resource.search(queryString, null, null, pageSize, null, false, true, 0L, 0, true, 0, null, false);
         if (res.isSuccess() && res.getObj() != null) {
             for (TaggedLogAPIEntity o : res.getObj()) {
                 jobs.add(o);
@@ -157,10 +163,10 @@ public class MRJobExecutionResource {
             }
         }
         queryString = String.format(queryFormat, Constants.JPA_RUNNING_JOB_EXECUTION_SERVICE_NAME, condition);
-        res = resource.search(queryString, null, null, pageSize, null, false, true,  0L, 0, true, 0, null, false);
+        res = resource.search(queryString, null, null, pageSize, null, false, true, 0L, 0, true, 0, null, false);
         if (res.isSuccess() && res.getObj() != null) {
             for (TaggedLogAPIEntity o : res.getObj()) {
-                if (! isDuplicate(jobIds, o)) {
+                if (!isDuplicate(jobIds, o)) {
                     jobs.add(o);
                 }
             }
@@ -181,128 +187,228 @@ public class MRJobExecutionResource {
             response.setException(new Exception(res.getException()));
         }
         meta.put(TOTAL_RESULTS, jobs.size());
-        meta.put(ELAPSEDMS,stopWatch.getTime());
+        meta.put(ELAPSEDMS, stopWatch.getTime());
         response.setObj(jobs);
         response.setMeta(meta);
         return response;
     }
 
-    public List<Long> parseTimeList(String timelist) {
-        List<Long> times = new ArrayList<>();
-        String [] strs = timelist.split("[,\\s]");
-        for (String str : strs) {
-            try {
-                times.add(Long.parseLong(str));
-            } catch (Exception ex) {
-                LOG.warn(str + " is not a number");
-            }
-        }
-        return times;
-    }
 
-    public int getPosition(List<Long> times, Long duration) {
-        duration = duration / 1000;
-        for (int i = 1; i < times.size(); i++) {
-            if (duration < times.get(i)) {
-                return i - 1;
-            }
-        }
-        return times.size() - 1;
-    }
 
-    public void getTopTasks(List<MRJobTaskGroupResponse.UnitTaskCount> list, long top) {
-        for (MRJobTaskGroupResponse.UnitTaskCount taskCounter : list) {
-            Iterator<TaskExecutionAPIEntity> iterator = taskCounter.entities.iterator();
-            for (int i = 0; i < top && iterator.hasNext(); i++) {
-                taskCounter.topEntities.add(iterator.next());
-            }
-            taskCounter.entities.clear();
-        }
-    }
-
-    public void initTaskCountList(List<MRJobTaskGroupResponse.UnitTaskCount> runningTaskCount,
-                                  List<MRJobTaskGroupResponse.UnitTaskCount> finishedTaskCount,
-                                  List<Long> times,
-                                  Comparator comparator) {
-        for (int i = 0; i < times.size(); i++) {
-            runningTaskCount.add(new MRJobTaskGroupResponse.UnitTaskCount(times.get(i), comparator));
-            finishedTaskCount.add(new MRJobTaskGroupResponse.UnitTaskCount(times.get(i), comparator));
-        }
-    }
 
     @GET
-    @Path("{jobId}/taskCounts")
+    @Path("{jobId}/taskCountsByDuration")
     @Produces(MediaType.APPLICATION_JSON)
-    public MRJobTaskGroupResponse getTaskCounts(@PathParam("jobId") String jobId,
-                                                @QueryParam("site") String site,
-                                                @QueryParam("timelineInSecs") String timeList,
-                                                @QueryParam("top") long top) {
-        MRJobTaskGroupResponse response = new MRJobTaskGroupResponse();
+    public TaskCountPerJobResponse getTaskCountsPerJob(@PathParam("jobId") String jobId,
+                                                       @QueryParam("site") String site,
+                                                       @QueryParam("timelineInSecs") String timeList,
+                                                       @QueryParam("top") long top) {
+        TaskCountPerJobResponse response = new TaskCountPerJobResponse();
         if (jobId == null || site == null || timeList == null || timeList.isEmpty()) {
             response.errMessage = "IllegalArgumentException: jobId == null || site == null || timelineInSecs == null or isEmpty";
             return response;
         }
-        List<MRJobTaskGroupResponse.UnitTaskCount> runningTaskCount = new ArrayList<>();
-        List<MRJobTaskGroupResponse.UnitTaskCount> finishedTaskCount = new ArrayList<>();
+        TaskCountByDurationHelper helper = new TaskCountByDurationHelper();
+        List<MRJobTaskCountResponse.UnitTaskCount> runningTaskCount = new ArrayList<>();
+        List<MRJobTaskCountResponse.UnitTaskCount> finishedTaskCount = new ArrayList<>();
 
-        List<Long> times = parseTimeList(timeList);
+        List<Long> times = helper.parseTimeList(timeList);
         String query = String.format("%s[@site=\"%s\" AND @jobId=\"%s\"]{*}", Constants.JPA_TASK_EXECUTION_SERVICE_NAME, site, jobId);
         GenericServiceAPIResponseEntity<org.apache.eagle.jpm.mr.historyentity.TaskExecutionAPIEntity> historyRes =
-                resource.search(query,  null, null, Integer.MAX_VALUE, null, false, true,  0L, 0, true, 0, null, false);
+            resource.search(query, null, null, Integer.MAX_VALUE, null, false, true, 0L, 0, true, 0, null, false);
         if (historyRes.isSuccess() && historyRes.getObj() != null && historyRes.getObj().size() > 0) {
-            initTaskCountList(runningTaskCount, finishedTaskCount, times, new HistoryTaskComparator());
+            helper.initTaskCountList(runningTaskCount, finishedTaskCount, times, new TaskCountByDurationHelper.HistoryTaskComparator());
             for (org.apache.eagle.jpm.mr.historyentity.TaskExecutionAPIEntity o : historyRes.getObj()) {
-                int index = getPosition(times, o.getDuration());
-                MRJobTaskGroupResponse.UnitTaskCount counter = finishedTaskCount.get(index);
-                counter.taskCount++;
+                int index = helper.getPosition(times, o.getDuration());
+                MRJobTaskCountResponse.UnitTaskCount counter = finishedTaskCount.get(index);
+                helper.countTask(counter, o.getTags().get(TASK_TYPE.toString()));
                 counter.entities.add(o);
             }
         } else {
             query = String.format("%s[@site=\"%s\" AND @jobId=\"%s\"]{*}", Constants.JPA_RUNNING_TASK_EXECUTION_SERVICE_NAME, site, jobId);
             GenericServiceAPIResponseEntity<TaskExecutionAPIEntity> runningRes =
-                    resource.search(query,  null, null, Integer.MAX_VALUE, null, false, true,  0L, 0, true, 0, null, false);
+                resource.search(query, null, null, Integer.MAX_VALUE, null, false, true, 0L, 0, true, 0, null, false);
             if (runningRes.isSuccess() && runningRes.getObj() != null) {
-                initTaskCountList(runningTaskCount, finishedTaskCount, times, new RunningTaskComparator());
+                helper.initTaskCountList(runningTaskCount, finishedTaskCount, times, new TaskCountByDurationHelper.RunningTaskComparator());
                 for (TaskExecutionAPIEntity o : runningRes.getObj()) {
-                    int index = getPosition(times, o.getDuration());
+                    int index = helper.getPosition(times, o.getDuration());
                     if (o.getTaskStatus().equalsIgnoreCase(Constants.TaskState.RUNNING.toString())) {
-                        MRJobTaskGroupResponse.UnitTaskCount counter = runningTaskCount.get(index);
-                        counter.taskCount++;
+                        MRJobTaskCountResponse.UnitTaskCount counter = runningTaskCount.get(index);
+                        helper.countTask(counter, o.getTags().get(TASK_TYPE.toString()));
                         counter.entities.add(o);
                     } else if (o.getEndTime() != 0) {
-                        MRJobTaskGroupResponse.UnitTaskCount counter = finishedTaskCount.get(index);
-                        counter.taskCount++;
+                        MRJobTaskCountResponse.UnitTaskCount counter = finishedTaskCount.get(index);
+                        helper.countTask(counter, o.getTags().get(TASK_TYPE.toString()));
                         counter.entities.add(o);
                     }
                 }
             }
         }
-        if (top > 0)  {
-            getTopTasks(runningTaskCount, top);
+        if (top > 0) {
+            helper.getTopTasks(runningTaskCount, top);
             response.runningTaskCount = runningTaskCount;
-            getTopTasks(finishedTaskCount, top);
+            helper.getTopTasks(finishedTaskCount, top);
             response.finishedTaskCount = finishedTaskCount;
         }
+        response.topNumber = top;
         return response;
     }
 
-    static class RunningTaskComparator implements Comparator<TaskExecutionAPIEntity> {
-        @Override
-        public int compare(TaskExecutionAPIEntity o1, TaskExecutionAPIEntity o2) {
-            Long time1 = o1.getDuration();
-            Long time2 = o2.getDuration();
-            return (time1 > time2 ? -1 : (time1 == time2) ? 0 : 1);
+    @GET
+    @Path("runningJobCounts")
+    @Produces(MediaType.APPLICATION_JSON)
+    public JobCountResponse getRunningJobCount(@QueryParam("site") String site,
+                                               @QueryParam("durationBegin") String startTime,
+                                               @QueryParam("durationEnd") String endTime,
+                                               @QueryParam("intervalInSecs") long intervalInSecs) {
+        JobCountResponse response = new JobCountResponse();
+        MRJobCountHelper helper = new MRJobCountHelper();
+        if (site == null || startTime == null || endTime == null) {
+            response.errMessage = "IllegalArgument: site, durationBegin, durationEnd is null";
+            return response;
+        }
+        if (intervalInSecs <= 0) {
+            response.errMessage = String.format("IllegalArgument: intervalInSecs=%s is invalid", intervalInSecs);
+            return response;
+        }
+        long startTimeInMills;
+        String searchStartTime = startTime;
+        String searchEndTime = endTime;
+        try {
+            startTimeInMills = DateTimeUtil.humanDateToSeconds(startTime) * DateTimeUtil.ONESECOND;
+            searchStartTime = helper.moveTimeforwardOneDay(searchStartTime);
+        } catch (Exception e) {
+            response.errMessage = e.getMessage();
+            return response;
+        }
+        String query = String.format("%s[@site=\"%s\" AND @endTime>=%s]{@startTime,@endTime,@jobType}", Constants.JPA_JOB_EXECUTION_SERVICE_NAME, site, startTimeInMills);
+        GenericServiceAPIResponseEntity<JobExecutionAPIEntity> historyRes =
+            resource.search(query, searchStartTime, searchEndTime, Integer.MAX_VALUE, null, false, true, 0L, 0, true, 0, null, false);
+        if (!historyRes.isSuccess() || historyRes.getObj() == null) {
+            response.errMessage = String.format("Catch an exception: %s with query=%s", historyRes.getException(), query);
+            return response;
         }
+
+        try {
+            long startTimeInSecs = DateTimeUtil.humanDateToSeconds(startTime);
+            long endTimeInSecs = DateTimeUtil.humanDateToSeconds(endTime);
+            return helper.getRunningJobCount(historyRes.getObj(), startTimeInSecs, endTimeInSecs, intervalInSecs);
+        } catch (Exception e) {
+            response.errMessage = e.getMessage();
+            return response;
+        }
+    }
+
+    @GET
+    @Path("jobMetrics/entities")
+    @Produces(MediaType.APPLICATION_JSON)
+    public Object getJobMetricsByEntitiesQuery(@QueryParam("site") String site,
+                                               @QueryParam("timePoint") String timePoint,
+                                               @QueryParam("metricName") String metricName,
+                                               @QueryParam("intervalmin") long intervalmin,
+                                               @QueryParam("top") int top) {
+        return getJobMetrics(site, timePoint, metricName, intervalmin, top, queryMetricEntitiesFunc);
     }
 
-    static class HistoryTaskComparator implements Comparator<org.apache.eagle.jpm.mr.historyentity.TaskExecutionAPIEntity> {
-        @Override
-        public int compare(org.apache.eagle.jpm.mr.historyentity.TaskExecutionAPIEntity o1,
-                           org.apache.eagle.jpm.mr.historyentity.TaskExecutionAPIEntity o2) {
-            Long time1 = o1.getDuration();
-            Long time2 = o2.getDuration();
-            return (time1 > time2 ? -1 : (time1 == time2) ? 0 : 1);
+    @GET
+    @Path("jobMetrics/list")
+    @Produces(MediaType.APPLICATION_JSON)
+    public Object getJobMetricsByListQuery(@QueryParam("site") String site,
+                                           @QueryParam("timePoint") String timePoint,
+                                           @QueryParam("metricName") String metricName,
+                                           @QueryParam("intervalmin") long intervalmin,
+                                           @QueryParam("top") int top) {
+        return getJobMetrics(site, timePoint, metricName, intervalmin, top, queryMetricListFunc);
+    }
+
+    public Object getJobMetrics(String site, String timePoint, String metricName, long intervalmin, int top,
+                                Function6<String, String, String, Long, Integer, String, Object> metricQueryFunc) {
+        GenericServiceAPIResponseEntity response = new GenericServiceAPIResponseEntity();
+        MRJobCountHelper helper = new MRJobCountHelper();
+        if (site == null || timePoint == null || metricName == null) {
+            response.setException(new IllegalArgumentException("Error: site, timePoint, metricName may be unset"));
+            response.setSuccess(false);
+            return response;
+        }
+        if (intervalmin <= 0) {
+            LOG.warn("query parameter intervalmin <= 0, use default value 5 instead");
+            intervalmin = 5;
         }
+        if (top <= 0) {
+            LOG.warn("query parameter top <= 0, use default value 10 instead");
+            top = 10;
+        }
+
+        long timePointsInMills;
+        String searchStartTime = timePoint;
+        String searchEndTime = timePoint;
+        try {
+            timePointsInMills = DateTimeUtil.humanDateToSeconds(timePoint) * DateTimeUtil.ONESECOND;
+            searchStartTime = helper.moveTimeforwardOneDay(searchStartTime);
+        } catch (ParseException e) {
+            response.setException(e);
+            response.setSuccess(false);
+            return response;
+        }
+        String query = String.format("%s[@site=\"%s\" AND @startTime<=\"%s\" AND @endTime>=\"%s\"]{@startTime,@endTime}",
+            Constants.JPA_JOB_EXECUTION_SERVICE_NAME, site, timePointsInMills, timePointsInMills);
+        GenericServiceAPIResponseEntity<JobExecutionAPIEntity> historyRes =
+            resource.search(query, searchStartTime, searchEndTime, Integer.MAX_VALUE, null, false, true, 0L, 0, true, 0, null, false);
+        if (!historyRes.isSuccess() || historyRes.getObj() == null) {
+            return historyRes;
+        }
+
+        List<String> timeDuration = helper.getSearchTimeDuration(historyRes.getObj());
+        LOG.info(String.format("new search time range: startTime=%s, endTime=%s", timeDuration.get(0), timeDuration.get(1)));
+        query = String.format("%s[@site=\"%s\"]<@jobId>{sum(value)}.{sum(value) desc}", Constants.GENERIC_METRIC_SERVICE, site);
+        return metricQueryFunc.apply(query, timeDuration.get(0), timeDuration.get(1), intervalmin, top, metricName);
     }
 
+    Function6<String, String, String, Long, Integer, String, Object> queryMetricEntitiesFunc
+        = (query, startTime, endTime, intervalmin, top, metricName) -> {
+            GenericEntityServiceResource resource = new GenericEntityServiceResource();
+            return resource.search(query, startTime, endTime, Integer.MAX_VALUE, null,
+            false, true, intervalmin, top, true, 0, metricName, false);
+        };
+
+    Function6<String, String, String, Long, Integer, String, Object> queryMetricListFunc
+        = (query, startTime, endTime, intervalmin, top, metricName) -> {
+            ListQueryResource resource = new ListQueryResource();
+            return resource.listQuery(query, startTime, endTime, Integer.MAX_VALUE, null,
+            false, true, intervalmin, top, true, 0, metricName, false);
+        };
+
+    @FunctionalInterface
+    interface Function6<A, B, C, D, E, F, R> {
+        R apply(A a, B b, C c, D d, E e, F f);
+    }
+
+    @GET
+    @Path("jobCountsByDuration")
+    @Produces(MediaType.APPLICATION_JSON)
+    public JobCountResponse getJobCountGroupByDuration(@QueryParam("site") String site,
+                                                       @QueryParam("timelineInSecs") String timeList,
+                                                       @QueryParam("jobStartTimeBegin") String startTime,
+                                                       @QueryParam("jobStartTimeEnd") String endTime) {
+        JobCountResponse response = new JobCountResponse();
+        MRJobCountHelper helper = new MRJobCountHelper();
+        if (site == null || startTime == null || endTime == null || timeList == null) {
+            response.errMessage = "IllegalArgument: site, jobStartTimeBegin, jobStartTimeEnd, or timelineInSecs is null";
+            return response;
+        }
+        String query = String.format("%s[@site=\"%s\"]{@durationTime,@jobType}", Constants.JPA_JOB_EXECUTION_SERVICE_NAME, site);
+        GenericServiceAPIResponseEntity<JobExecutionAPIEntity> historyRes =
+            resource.search(query, startTime, endTime, Integer.MAX_VALUE, null, false, true, 0L, 0, true, 0, null, false);
+        if (!historyRes.isSuccess() || historyRes.getObj() == null) {
+            response.errMessage = String.format("Catch an exception: %s with query=%s", historyRes.getException(), query);
+            return response;
+        }
+        try {
+            return helper.getHistoryJobCount(historyRes.getObj(), timeList);
+        } catch (Exception e) {
+            response.errMessage = e.getMessage();
+            return response;
+        }
+
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/a66f64cf/eagle-jpm/eagle-jpm-service/src/main/java/org/apache/eagle/service/jpm/MRJobTaskCountResponse.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-service/src/main/java/org/apache/eagle/service/jpm/MRJobTaskCountResponse.java b/eagle-jpm/eagle-jpm-service/src/main/java/org/apache/eagle/service/jpm/MRJobTaskCountResponse.java
new file mode 100644
index 0000000..c546198
--- /dev/null
+++ b/eagle-jpm/eagle-jpm-service/src/main/java/org/apache/eagle/service/jpm/MRJobTaskCountResponse.java
@@ -0,0 +1,65 @@
+/*
+ *  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.eagle.service.jpm;
+
+import java.util.*;
+
+public class MRJobTaskCountResponse {
+    public String errMessage;
+
+    public static class TaskCountPerJobResponse extends MRJobTaskCountResponse {
+        public long topNumber;
+        public List<UnitTaskCount> runningTaskCount;
+        public List<UnitTaskCount> finishedTaskCount;
+    }
+
+    public static class JobCountResponse extends MRJobTaskCountResponse {
+        public List<UnitJobCount> jobCounts;
+    }
+
+    static class UnitTaskCount {
+        public long timeBucket;
+        public int taskCount;
+        public int mapTaskCount;
+        public int reduceTaskCount;
+        public Set entities;
+        public List topEntities;
+
+        UnitTaskCount(long timeBucket, Comparator comparator) {
+            this.timeBucket = timeBucket;
+            this.taskCount = 0;
+            this.mapTaskCount = 0;
+            this.reduceTaskCount = 0;
+            entities = new TreeSet<>(comparator);
+            topEntities = new ArrayList<>();
+        }
+    }
+
+    static class UnitJobCount {
+        public long timeBucket;
+        public long jobCount;
+        public Map<String, Long> jobCountByType;
+
+        UnitJobCount(long timeBucket) {
+            this.timeBucket = timeBucket;
+            this.jobCount = 0;
+            this.jobCountByType = new HashMap<>();
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/a66f64cf/eagle-jpm/eagle-jpm-service/src/main/java/org/apache/eagle/service/jpm/MRJobTaskGroupResponse.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-service/src/main/java/org/apache/eagle/service/jpm/MRJobTaskGroupResponse.java b/eagle-jpm/eagle-jpm-service/src/main/java/org/apache/eagle/service/jpm/MRJobTaskGroupResponse.java
deleted file mode 100644
index 3be9b43..0000000
--- a/eagle-jpm/eagle-jpm-service/src/main/java/org/apache/eagle/service/jpm/MRJobTaskGroupResponse.java
+++ /dev/null
@@ -1,41 +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.eagle.service.jpm;
-
-import java.util.*;
-
-class MRJobTaskGroupResponse {
-    public List<UnitTaskCount> runningTaskCount;
-    public List<UnitTaskCount> finishedTaskCount;
-    public String errMessage;
-
-   static class UnitTaskCount {
-        public long timeBucket;
-        public int taskCount;
-        public Set entities;
-        public List topEntities;
-
-        UnitTaskCount(long timeBucket, Comparator comparator) {
-            this.timeBucket = timeBucket;
-            this.taskCount = 0;
-            entities = new TreeSet<>(comparator);
-            topEntities = new ArrayList<>();
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/a66f64cf/eagle-jpm/eagle-jpm-service/src/main/java/org/apache/eagle/service/jpm/TaskCountByDurationHelper.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-service/src/main/java/org/apache/eagle/service/jpm/TaskCountByDurationHelper.java b/eagle-jpm/eagle-jpm-service/src/main/java/org/apache/eagle/service/jpm/TaskCountByDurationHelper.java
new file mode 100644
index 0000000..0eeb440
--- /dev/null
+++ b/eagle-jpm/eagle-jpm-service/src/main/java/org/apache/eagle/service/jpm/TaskCountByDurationHelper.java
@@ -0,0 +1,106 @@
+/*
+ *  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.eagle.service.jpm;
+
+import org.apache.eagle.jpm.mr.runningentity.TaskExecutionAPIEntity;
+import org.apache.eagle.jpm.util.Constants;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Comparator;
+import java.util.Iterator;
+import java.util.List;
+
+public class TaskCountByDurationHelper {
+
+    private static final Logger LOG = LoggerFactory.getLogger(TaskCountByDurationHelper.class);
+
+    public static List<Long> parseTimeList(String timelist) {
+        List<Long> times = new ArrayList<>();
+        String [] strs = timelist.split("[,\\s]");
+        for (String str : strs) {
+            try {
+                times.add(Long.parseLong(str));
+            } catch (Exception ex) {
+                LOG.warn(str + " is not a number");
+            }
+        }
+        return times;
+    }
+
+    public static int getPosition(List<Long> times, Long duration) {
+        duration = duration / 1000;
+        for (int i = 1; i < times.size(); i++) {
+            if (duration < times.get(i)) {
+                return i - 1;
+            }
+        }
+        return times.size() - 1;
+    }
+
+    public void getTopTasks(List<MRJobTaskCountResponse.UnitTaskCount> list, long top) {
+        for (MRJobTaskCountResponse.UnitTaskCount taskCounter : list) {
+            Iterator<TaskExecutionAPIEntity> iterator = taskCounter.entities.iterator();
+            for (int i = 0; i < top && iterator.hasNext(); i++) {
+                taskCounter.topEntities.add(iterator.next());
+            }
+            taskCounter.entities.clear();
+        }
+    }
+
+    public void countTask(MRJobTaskCountResponse.UnitTaskCount counter, String taskType) {
+        counter.taskCount++;
+        if (taskType.equalsIgnoreCase(Constants.TaskType.MAP.toString())) {
+            counter.mapTaskCount++;
+        } else if (taskType.equalsIgnoreCase(Constants.TaskType.REDUCE.toString())) {
+            counter.reduceTaskCount++;
+        }
+    }
+
+    public void initTaskCountList(List<MRJobTaskCountResponse.UnitTaskCount> runningTaskCount,
+                                  List<MRJobTaskCountResponse.UnitTaskCount> finishedTaskCount,
+                                  List<Long> times,
+                                  Comparator comparator) {
+        for (int i = 0; i < times.size(); i++) {
+            runningTaskCount.add(new MRJobTaskCountResponse.UnitTaskCount(times.get(i), comparator));
+            finishedTaskCount.add(new MRJobTaskCountResponse.UnitTaskCount(times.get(i), comparator));
+        }
+    }
+
+    static class RunningTaskComparator implements Comparator<TaskExecutionAPIEntity> {
+        @Override
+        public int compare(TaskExecutionAPIEntity o1, TaskExecutionAPIEntity o2) {
+            Long time1 = o1.getDuration();
+            Long time2 = o2.getDuration();
+            return (time1 > time2 ? -1 : (time1 == time2) ? 0 : 1);
+        }
+    }
+
+    static class HistoryTaskComparator implements Comparator<org.apache.eagle.jpm.mr.historyentity.TaskExecutionAPIEntity> {
+        @Override
+        public int compare(org.apache.eagle.jpm.mr.historyentity.TaskExecutionAPIEntity o1,
+                           org.apache.eagle.jpm.mr.historyentity.TaskExecutionAPIEntity o2) {
+            Long time1 = o1.getDuration();
+            Long time2 = o2.getDuration();
+            return (time1 > time2 ? -1 : (time1 == time2) ? 0 : 1);
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/a66f64cf/eagle-jpm/eagle-jpm-service/src/test/java/org/apache/eagle/service/jpm/TestJobCountPerBucketHelper.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-service/src/test/java/org/apache/eagle/service/jpm/TestJobCountPerBucketHelper.java b/eagle-jpm/eagle-jpm-service/src/test/java/org/apache/eagle/service/jpm/TestJobCountPerBucketHelper.java
new file mode 100644
index 0000000..718f068
--- /dev/null
+++ b/eagle-jpm/eagle-jpm-service/src/test/java/org/apache/eagle/service/jpm/TestJobCountPerBucketHelper.java
@@ -0,0 +1,87 @@
+/*
+ *  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.eagle.service.jpm;
+
+import org.apache.eagle.common.DateTimeUtil;
+import org.junit.Assert;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.text.ParseException;
+import java.util.ArrayList;
+import java.util.List;
+
+public class TestJobCountPerBucketHelper {
+    MRJobCountHelper helper = new MRJobCountHelper();
+
+    private static final Logger LOG = LoggerFactory.getLogger(TestJobCountPerBucketHelper.class);
+
+    @Test
+    public void test() throws ParseException {
+        String timeString = "2016-08-22 20:13:00";
+        long timestamp = DateTimeUtil.humanDateToSeconds(timeString);
+        String timeString2 = DateTimeUtil.secondsToHumanDate(timestamp);
+        Assert.assertTrue(timeString2.equals(timeString));
+
+        String timeString3 = helper.moveTimeforwardOneDay(timeString);
+        Assert.assertTrue(timeString3.equals("2016-08-21 20:13:00"));
+    }
+
+    @Test
+    public void test2() throws ParseException {
+        String startTime = "2016-08-22 20:13:00";
+        String endTime = "2016-08-22 24:13:00";
+        List<MRJobTaskCountResponse.UnitJobCount> jobCounts = new ArrayList<>();
+        helper.initJobCountList(jobCounts, DateTimeUtil.humanDateToSeconds(startTime), DateTimeUtil.humanDateToSeconds(endTime), 15 * 60);
+        /*for (MRJobTaskCountResponse.UnitJobCount jobCount : jobCounts) {
+            LOG.info(DateTimeUtil.secondsToHumanDate(jobCount.timeBucket));
+        }*/
+        Assert.assertTrue(DateTimeUtil.secondsToHumanDate(jobCounts.get(1).timeBucket).equals("2016-08-22 20:15:00"));
+    }
+
+    @Test
+    public void test3() {
+        List<MRJobTaskCountResponse.UnitJobCount> jobCounts = new ArrayList<>();
+        long intervalSecs = 5;
+        helper.initJobCountList(jobCounts, 3, 31, intervalSecs);
+        helper.countJob(jobCounts, 5, 10, intervalSecs, "hive");
+        helper.countJob(jobCounts, 13, 18, intervalSecs, "hive");
+        helper.countJob(jobCounts, 18, 28, intervalSecs, "hive");
+        helper.countJob(jobCounts, 25, 33, intervalSecs, "hive");
+        Assert.assertTrue(jobCounts.size() == 7);
+        Assert.assertTrue(jobCounts.get(1).jobCount == 1);
+        Assert.assertTrue(jobCounts.get(5).jobCount == 2);
+    }
+
+    @Test
+    public void test4() throws ParseException {
+        List<MRJobTaskCountResponse.UnitJobCount> jobCounts = new ArrayList<>();
+        long intervalSecs = 60 * 15;
+        String startTime = "2016-08-22 20:13:00";
+        String endTime = "2016-08-22 24:13:00";
+        helper.initJobCountList(jobCounts, DateTimeUtil.humanDateToSeconds(startTime), DateTimeUtil.humanDateToSeconds(endTime), intervalSecs);
+        helper.countJob(jobCounts,
+                DateTimeUtil.humanDateToSeconds("2016-08-22 20:23:00"),
+                DateTimeUtil.humanDateToSeconds("2016-08-22 20:30:00"),
+                intervalSecs,
+                "hive");
+        Assert.assertTrue(jobCounts.get(2).jobCount == 1);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/a66f64cf/eagle-jpm/eagle-jpm-service/src/test/java/org/apache/eagle/service/jpm/TestMRJobExecutionResource.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-service/src/test/java/org/apache/eagle/service/jpm/TestMRJobExecutionResource.java b/eagle-jpm/eagle-jpm-service/src/test/java/org/apache/eagle/service/jpm/TestMRJobExecutionResource.java
deleted file mode 100644
index 824556b..0000000
--- a/eagle-jpm/eagle-jpm-service/src/test/java/org/apache/eagle/service/jpm/TestMRJobExecutionResource.java
+++ /dev/null
@@ -1,99 +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.eagle.service.jpm;
-
-import org.apache.eagle.jpm.mr.runningentity.TaskExecutionAPIEntity;
-import org.apache.eagle.jpm.util.Constants;
-import org.junit.Assert;
-import org.junit.Test;
-
-import java.util.ArrayList;
-import java.util.Comparator;
-import java.util.List;
-
-public class TestMRJobExecutionResource {
-
-    @Test
-    public void test() {
-        MRJobExecutionResource resource = new MRJobExecutionResource();
-        String timeList = " 0, 10,20,40 ";
-        List<Long> times = resource.parseTimeList(timeList);
-        Assert.assertTrue(times.size() == 4);
-
-        long val = 25 * 1000;
-        int index = resource.getPosition(times, val);
-        Assert.assertTrue(index == 2);
-    }
-
-    @Test
-    public void test2() {
-        MRJobExecutionResource resource = new MRJobExecutionResource();
-        String timeList = " 0, 10,20,40 ";
-        List<Long> times = resource.parseTimeList(timeList);
-
-        TaskExecutionAPIEntity test1 = new TaskExecutionAPIEntity();
-        test1.setDuration(15 * 1000);
-        test1.setTaskStatus("running");
-        TaskExecutionAPIEntity test4 = new TaskExecutionAPIEntity();
-        test4.setDuration(13 * 1000);
-        test4.setTaskStatus("running");
-        TaskExecutionAPIEntity test2 = new TaskExecutionAPIEntity();
-        test2.setDuration(0 * 1000);
-        test2.setEndTime(100);
-        test2.setTaskStatus("x");
-        TaskExecutionAPIEntity test3 = new TaskExecutionAPIEntity();
-        test3.setDuration(19 * 1000);
-        test3.setTaskStatus("running");
-        TaskExecutionAPIEntity test5 = new TaskExecutionAPIEntity();
-        test5.setDuration(20 * 1000);
-        test5.setEndTime(28);
-        test5.setTaskStatus("x");
-        List<TaskExecutionAPIEntity> tasks = new ArrayList<>();
-        tasks.add(test1);
-        tasks.add(test2);
-        tasks.add(test3);
-        tasks.add(test4);
-        tasks.add(test5);
-
-        List<MRJobTaskGroupResponse.UnitTaskCount> runningTaskCount = new ArrayList<>();
-        List<MRJobTaskGroupResponse.UnitTaskCount> finishedTaskCount = new ArrayList<>();
-
-        Comparator comparator = new MRJobExecutionResource.RunningTaskComparator();
-        resource.initTaskCountList(runningTaskCount, finishedTaskCount, times, comparator);
-
-        for (TaskExecutionAPIEntity o : tasks) {
-            int index = resource.getPosition(times, o.getDuration());
-            if (o.getTaskStatus().equalsIgnoreCase(Constants.TaskState.RUNNING.toString())) {
-                MRJobTaskGroupResponse.UnitTaskCount counter = runningTaskCount.get(index);
-                counter.taskCount++;
-                counter.entities.add(o);
-            } else if (o.getEndTime() != 0) {
-                MRJobTaskGroupResponse.UnitTaskCount counter = finishedTaskCount.get(index);
-                counter.taskCount++;
-                counter.entities.add(o);
-            }
-        }
-        int top = 2;
-        if (top > 0)  {
-            resource.getTopTasks(runningTaskCount, top);
-        }
-        Assert.assertTrue(runningTaskCount.get(1).taskCount == 3);
-        Assert.assertTrue(runningTaskCount.get(1).topEntities.size() == 2);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/a66f64cf/eagle-jpm/eagle-jpm-service/src/test/java/org/apache/eagle/service/jpm/TestTaskCountPerJobHelper.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-service/src/test/java/org/apache/eagle/service/jpm/TestTaskCountPerJobHelper.java b/eagle-jpm/eagle-jpm-service/src/test/java/org/apache/eagle/service/jpm/TestTaskCountPerJobHelper.java
new file mode 100644
index 0000000..2cd0b8e
--- /dev/null
+++ b/eagle-jpm/eagle-jpm-service/src/test/java/org/apache/eagle/service/jpm/TestTaskCountPerJobHelper.java
@@ -0,0 +1,96 @@
+/*
+ *  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.eagle.service.jpm;
+
+import org.apache.eagle.jpm.mr.runningentity.TaskExecutionAPIEntity;
+import org.apache.eagle.jpm.util.Constants;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.List;
+
+public class TestTaskCountPerJobHelper {
+    TaskCountByDurationHelper helper = new TaskCountByDurationHelper();
+
+    @Test
+    public void test() {
+        String timeList = " 0, 10,20,40 ";
+        List<Long> times = helper.parseTimeList(timeList);
+        Assert.assertTrue(times.size() == 4);
+
+        long val = 25 * 1000;
+        int index = helper.getPosition(times, val);
+        Assert.assertTrue(index == 2);
+    }
+
+    @Test
+    public void test2() {
+        TaskExecutionAPIEntity test1 = new TaskExecutionAPIEntity();
+        test1.setDuration(15 * 1000);
+        test1.setTaskStatus("running");
+        TaskExecutionAPIEntity test4 = new TaskExecutionAPIEntity();
+        test4.setDuration(13 * 1000);
+        test4.setTaskStatus("running");
+        TaskExecutionAPIEntity test2 = new TaskExecutionAPIEntity();
+        test2.setDuration(0 * 1000);
+        test2.setEndTime(100);
+        test2.setTaskStatus("x");
+        TaskExecutionAPIEntity test3 = new TaskExecutionAPIEntity();
+        test3.setDuration(19 * 1000);
+        test3.setTaskStatus("running");
+        TaskExecutionAPIEntity test5 = new TaskExecutionAPIEntity();
+        test5.setDuration(20 * 1000);
+        test5.setEndTime(28);
+        test5.setTaskStatus("x");
+        List<TaskExecutionAPIEntity> tasks = new ArrayList<>();
+        tasks.add(test1);
+        tasks.add(test2);
+        tasks.add(test3);
+        tasks.add(test4);
+        tasks.add(test5);
+
+        List<MRJobTaskCountResponse.UnitTaskCount> runningTaskCount = new ArrayList<>();
+        List<MRJobTaskCountResponse.UnitTaskCount> finishedTaskCount = new ArrayList<>();
+
+        String timeList = " 0, 10,20,40 ";
+        List<Long> times = helper.parseTimeList(timeList);
+
+        helper.initTaskCountList(runningTaskCount, finishedTaskCount, times, new TaskCountByDurationHelper.RunningTaskComparator());
+
+        for (TaskExecutionAPIEntity o : tasks) {
+            int index = helper.getPosition(times, o.getDuration());
+            if (o.getTaskStatus().equalsIgnoreCase(Constants.TaskState.RUNNING.toString())) {
+                MRJobTaskCountResponse.UnitTaskCount counter = runningTaskCount.get(index);
+                counter.taskCount++;
+                counter.entities.add(o);
+            } else if (o.getEndTime() != 0) {
+                MRJobTaskCountResponse.UnitTaskCount counter = finishedTaskCount.get(index);
+                counter.taskCount++;
+                counter.entities.add(o);
+            }
+        }
+        int top = 2;
+        if (top > 0)  {
+            helper.getTopTasks(runningTaskCount, top);
+        }
+        Assert.assertTrue(runningTaskCount.get(1).taskCount == 3);
+        Assert.assertTrue(runningTaskCount.get(1).topEntities.size() == 2);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/a66f64cf/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/Constants.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/Constants.java b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/Constants.java
index 7dce0a2..ec56eac 100644
--- a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/Constants.java
+++ b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/Constants.java
@@ -23,6 +23,8 @@ import org.slf4j.LoggerFactory;
 public class Constants {
     private static final Logger LOG = LoggerFactory.getLogger(Constants.class);
 
+    public static final String GENERIC_METRIC_SERVICE = "GenericMetricService";
+
     //SPARK
     public static final String SPARK_APP_SERVICE_ENDPOINT_NAME = "SparkAppService";
     public static final String SPARK_JOB_SERVICE_ENDPOINT_NAME = "SparkJobService";


[21/52] [abbrv] incubator-eagle git commit: wrapper alert engine as one application of eagle server hdfs audit log + alert engine end-2-end

Posted by yo...@apache.org.
wrapper alert engine as one application of eagle server
hdfs audit log + alert engine end-2-end

https://issues.apache.org/jira/browse/EAGLE-481

Author: @yonzhang <yo...@apache.org>

Closes: #392


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

Branch: refs/heads/master
Commit: 000b746041ec07774e4ed5d5562f9c66307f1c20
Parents: 36f9a84
Author: yonzhang <yo...@gmail.com>
Authored: Thu Aug 25 21:39:53 2016 -0700
Committer: yonzhang <yo...@gmail.com>
Committed: Thu Aug 25 21:39:53 2016 -0700

----------------------------------------------------------------------
 .../eagle-alert-parent/eagle-alert-app/pom.xml  |  35 ++++
 .../eagle/alert/app/AlertUnitTopologyApp.java   |  39 ++++
 .../alert/app/AlertUnitTopologyAppProvider.java |  29 +++
 ...e.alert.app.AlertUnitTopologyAppProvider.xml | 135 +++++++++++++
 ...org.apache.eagle.app.spi.ApplicationProvider |  18 ++
 .../src/main/resources/application.conf         |  60 ++++++
 .../alert/AlertPolicyValidateProvider.java      |  36 ----
 .../eagle/alert/config/ZKConfigBuilder.java     |   8 +-
 .../scheme/JsonStringStreamNameSelector.java    |   6 +-
 .../alert/engine/spout/CorrelationSpout.java    |   4 +-
 .../src/main/resources/application.conf         |  19 +-
 .../alert/engine/e2e/CoordinatorClient.java     |   6 +-
 eagle-core/eagle-alert-parent/pom.xml           |   9 +-
 .../eagle/app/service/ApplicationContext.java   |   6 +-
 .../src/main/resources/log4j.properties         |   2 +-
 .../eagle-security-hdfs-auditlog/README.md      | 191 +++++++++++++++++++
 ...ecurity.auditlog.HdfsAuditLogAppProvider.xml |  26 ++-
 .../src/main/resources/scripts.txt              |  22 +++
 eagle-server/pom.xml                            |   7 +
 19 files changed, 581 insertions(+), 77 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/000b7460/eagle-core/eagle-alert-parent/eagle-alert-app/pom.xml
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-alert-parent/eagle-alert-app/pom.xml b/eagle-core/eagle-alert-parent/eagle-alert-app/pom.xml
new file mode 100644
index 0000000..768709f
--- /dev/null
+++ b/eagle-core/eagle-alert-parent/eagle-alert-app/pom.xml
@@ -0,0 +1,35 @@
+<?xml version="1.0"?>
+<!--
+  ~ Licensed to the Apache Software Foundation (ASF) under one or more
+  ~ contributor license agreements.  See the NOTICE file distributed with
+  ~ this work for additional information regarding copyright ownership.
+  ~ The ASF licenses this file to You under the Apache License, Version 2.0
+  ~ (the "License"); you may not use this file except in compliance with
+  ~ the License.  You may obtain a copy of the License at
+  ~
+  ~    http://www.apache.org/licenses/LICENSE-2.0
+  ~
+  ~ Unless required by applicable law or agreed to in writing, software
+  ~ distributed under the License is distributed on an "AS IS" BASIS,
+  ~ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  ~ See the License for the specific language governing permissions and
+  ~ limitations under the License.
+  -->
+<project xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd" xmlns="http://maven.apache.org/POM/4.0.0"
+    xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance">
+  <modelVersion>4.0.0</modelVersion>
+  <parent>
+    <groupId>org.apache.eagle</groupId>
+    <artifactId>eagle-alert-parent</artifactId>
+    <version>0.5.0-incubating-SNAPSHOT</version>
+  </parent>
+  <artifactId>eagle-alert-app</artifactId>
+  <packaging>jar</packaging>
+  <dependencies>
+    <dependency>
+      <groupId>org.apache.eagle</groupId>
+      <artifactId>eagle-app-base</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+  </dependencies>
+</project>

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/000b7460/eagle-core/eagle-alert-parent/eagle-alert-app/src/main/java/org/apache/eagle/alert/app/AlertUnitTopologyApp.java
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-alert-parent/eagle-alert-app/src/main/java/org/apache/eagle/alert/app/AlertUnitTopologyApp.java b/eagle-core/eagle-alert-parent/eagle-alert-app/src/main/java/org/apache/eagle/alert/app/AlertUnitTopologyApp.java
new file mode 100644
index 0000000..a80c3b8
--- /dev/null
+++ b/eagle-core/eagle-alert-parent/eagle-alert-app/src/main/java/org/apache/eagle/alert/app/AlertUnitTopologyApp.java
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.eagle.alert.app;import backtype.storm.generated.StormTopology;
+import com.typesafe.config.Config;
+import com.typesafe.config.ConfigFactory;
+import org.apache.eagle.alert.engine.UnitTopologyMain;
+import org.apache.eagle.app.StormApplication;
+import org.apache.eagle.app.environment.impl.StormEnvironment;
+
+/**
+ * since 8/25/16.
+ */
+public class AlertUnitTopologyApp extends StormApplication {
+    @Override
+    public StormTopology execute(Config config, StormEnvironment environment) {
+        return UnitTopologyMain.createTopology(config);
+    }
+
+    public static void main(String[] args){
+        Config config = ConfigFactory.load();
+        AlertUnitTopologyApp app = new AlertUnitTopologyApp();
+        app.run(config);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/000b7460/eagle-core/eagle-alert-parent/eagle-alert-app/src/main/java/org/apache/eagle/alert/app/AlertUnitTopologyAppProvider.java
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-alert-parent/eagle-alert-app/src/main/java/org/apache/eagle/alert/app/AlertUnitTopologyAppProvider.java b/eagle-core/eagle-alert-parent/eagle-alert-app/src/main/java/org/apache/eagle/alert/app/AlertUnitTopologyAppProvider.java
new file mode 100644
index 0000000..39a4583
--- /dev/null
+++ b/eagle-core/eagle-alert-parent/eagle-alert-app/src/main/java/org/apache/eagle/alert/app/AlertUnitTopologyAppProvider.java
@@ -0,0 +1,29 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.eagle.alert.app;
+import org.apache.eagle.app.spi.AbstractApplicationProvider;
+
+/**
+ * since 8/25/16.
+ */
+public class AlertUnitTopologyAppProvider extends AbstractApplicationProvider<AlertUnitTopologyApp> {
+    @Override
+    public AlertUnitTopologyApp getApplication() {
+        return new AlertUnitTopologyApp();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/000b7460/eagle-core/eagle-alert-parent/eagle-alert-app/src/main/resources/META-INF/providers/org.apache.eagle.alert.app.AlertUnitTopologyAppProvider.xml
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-alert-parent/eagle-alert-app/src/main/resources/META-INF/providers/org.apache.eagle.alert.app.AlertUnitTopologyAppProvider.xml b/eagle-core/eagle-alert-parent/eagle-alert-app/src/main/resources/META-INF/providers/org.apache.eagle.alert.app.AlertUnitTopologyAppProvider.xml
new file mode 100644
index 0000000..498cb8d
--- /dev/null
+++ b/eagle-core/eagle-alert-parent/eagle-alert-app/src/main/resources/META-INF/providers/org.apache.eagle.alert.app.AlertUnitTopologyAppProvider.xml
@@ -0,0 +1,135 @@
+<?xml version="1.0" encoding="UTF-8" ?>
+<!--
+  ~ Licensed to the Apache Software Foundation (ASF) under one or more
+  ~ contributor license agreements.  See the NOTICE file distributed with
+  ~ this work for additional information regarding copyright ownership.
+  ~ The ASF licenses this file to You under the Apache License, Version 2.0
+  ~ (the "License"); you may not use this file except in compliance with
+  ~ the License.  You may obtain a copy of the License at
+  ~
+  ~   http://www.apache.org/licenses/LICENSE-2.0
+  ~
+  ~ Unless required by applicable law or agreed to in writing, software
+  ~ distributed under the License is distributed on an "AS IS" BASIS,
+  ~ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  ~ See the License for the specific language governing permissions and
+  ~ limitations under the License.
+  -->
+
+<application>
+    <type>AlertUnitTopologyApp</type>
+    <name>Alert Unit Topology Application</name>
+    <version>0.5.0-incubating</version>
+    <appClass>org.apache.eagle.alert.app.AlertUnitTopologyApp</appClass>
+    <viewPath>/apps/alert</viewPath>
+    <configuration>
+       <!-- alert topology sizing parameters -->
+        <property>
+            <name>topology.numOfTotalWorkers</name>
+            <displayName>topology.numOfTotalWorkers</displayName>
+            <value>2</value>
+            <description>number of total storm workers</description>
+        </property>
+        <property>
+            <name>topology.numOfSpoutTasks</name>
+            <displayName>topology.numOfSpoutTasks</displayName>
+            <value>1</value>
+            <description>number of spout tasks</description>
+        </property>
+        <property>
+            <name>topology.numOfRouterBolts</name>
+            <displayName>topology.numOfRouterBolts</displayName>
+            <value>4</value>
+            <description>number of router tasks</description>
+        </property>
+        <property>
+            <name>topology.numOfAlertBolts</name>
+            <displayName>topology.numOfAlertBolts</displayName>
+            <value>10</value>
+            <description>number of alert tasks</description>
+        </property>
+        <property>
+            <name>topology.numOfPublishTasks</name>
+            <displayName>topology.numOfPublishTasks</displayName>
+            <value>1</value>
+            <description>number of publish tasks</description>
+        </property>
+        <property>
+            <name>topology.messageTimeoutSecs</name>
+            <displayName>topology.messageTimeoutSecs</displayName>
+            <value>3600</value>
+            <description>number of tuple timeout in seconds</description>
+        </property>
+
+        <!-- alert spout configuration -->
+        <property>
+            <name>spout.kafkaBrokerZkQuorum</name>
+            <displayName>spout.kafkaBrokerZkQuorum</displayName>
+            <value>server.eagle.apache.org:2181</value>
+            <description>zookeeper quorum for spout to consume data</description>
+        </property>
+        <property>
+            <name>spout.kafkaBrokerZkBasePath</name>
+            <displayName>spout.kafkaBrokerZkBasePath</displayName>
+            <value>/brokers</value>
+            <description>zk znode path for kafka brokers</description>
+        </property>
+        <property>
+            <name>spout.stormKafkaUseSameZkQuorumWithKafkaBroker</name>
+            <displayName>spout.stormKafkaUseSameZkQuorumWithKafkaBroker</displayName>
+            <value>true</value>
+            <description>same zookeeper for kafka server and kafka consumer(Storm-Kafka)</description>
+        </property>
+        <property>
+            <name>spout.stormKafkaTransactionZkPath</name>
+            <displayName>spout.stormKafkaTransactionZkPath</displayName>
+            <value>/consumers</value>
+            <description>zk path for storm kafka transaction</description>
+        </property>
+        <property>
+            <name>spout.stormKafkaEagleConsumer</name>
+            <displayName>spout.stormKafkaEagleConsumer</displayName>
+            <value>eagle_consumer</value>
+            <description>zookeeper quorum for spout to consume data</description>
+        </property>
+
+        <!-- zk config for alert engine -->
+        <property>
+            <name>zkConfig.zkQuorum</name>
+            <displayName>zkConfig.zkQuorum</displayName>
+            <value>server.eagle.apache.org:2181</value>
+            <description>zk quorum for alert engine</description>
+        </property>
+        <property>
+            <name>zkConfig.zkRoot</name>
+            <displayName>zkConfig.zkRoot</displayName>
+            <value>/alert</value>
+            <description>zk znode path for alert engine</description>
+        </property>
+
+        <property>
+            <name>metadataService.context</name>
+            <displayName>metadataService.context</displayName>
+            <value>/rest</value>
+            <description>metadata service context path</description>
+        </property>
+        <property>
+            <name>metadataService.host</name>
+            <displayName>metadataService.host</displayName>
+            <value>localhost</value>
+            <description>metadata service host</description>
+        </property>
+        <property>
+            <name>metadataService.port</name>
+            <displayName>metadataService.port</displayName>
+            <value>9090</value>
+            <description>metadata service port</description>
+        </property>
+    </configuration>
+    <docs>
+        <install>
+        </install>
+        <uninstall>
+        </uninstall>
+    </docs>
+</application>

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/000b7460/eagle-core/eagle-alert-parent/eagle-alert-app/src/main/resources/META-INF/services/org.apache.eagle.app.spi.ApplicationProvider
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-alert-parent/eagle-alert-app/src/main/resources/META-INF/services/org.apache.eagle.app.spi.ApplicationProvider b/eagle-core/eagle-alert-parent/eagle-alert-app/src/main/resources/META-INF/services/org.apache.eagle.app.spi.ApplicationProvider
new file mode 100644
index 0000000..ceb079d
--- /dev/null
+++ b/eagle-core/eagle-alert-parent/eagle-alert-app/src/main/resources/META-INF/services/org.apache.eagle.app.spi.ApplicationProvider
@@ -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.
+#
+
+org.apache.eagle.alert.app.AlertUnitTopologyAppProvider

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/000b7460/eagle-core/eagle-alert-parent/eagle-alert-app/src/main/resources/application.conf
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-alert-parent/eagle-alert-app/src/main/resources/application.conf b/eagle-core/eagle-alert-parent/eagle-alert-app/src/main/resources/application.conf
new file mode 100644
index 0000000..1a25cfa
--- /dev/null
+++ b/eagle-core/eagle-alert-parent/eagle-alert-app/src/main/resources/application.conf
@@ -0,0 +1,60 @@
+# 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.
+{
+  "appId" : "HBaseAuditLogApp",
+  "mode" : "LOCAL",
+  "siteId" : "testsite",
+  "topology" : {
+    "name" : "alertUnitTopology_1",
+    "numOfTotalWorkers" : 2,
+    "numOfSpoutTasks" : 1,
+    "numOfRouterBolts" : 4,
+    "numOfAlertBolts" : 10,
+    "numOfPublishTasks" : 1,
+    "messageTimeoutSecs": 3600,
+    "localMode" : "true"
+  },
+  "spout" : {
+    "kafkaBrokerZkQuorum": "server.eagle.apache.org:2181",
+    "kafkaBrokerZkBasePath": "/kafka",
+    "stormKafkaUseSameZkQuorumWithKafkaBroker": true,
+    "stormKafkaTransactionZkQuorum": "",
+    "stormKafkaTransactionZkPath": "/consumers",
+    "stormKafkaEagleConsumer": "eagle_consumer"
+  },
+  "zkConfig" : {
+    "zkQuorum" : "server.eagle.apache.org:2181",
+    "zkRoot" : "/alert"
+  },
+  "metadataService": {
+    "context" : "/rest",
+    "host" : "localhost",
+    "port" : 9090
+  },
+  "metric":{
+    "sink": {
+      // "kafka": {
+      //  "topic": "alert_metric"
+      //  "bootstrap.servers": "localhost:6667"
+      // }
+      //      "stdout": {}
+      //      "elasticsearch": {
+      //        "hosts": ["localhost:9200"]
+      //        "index": "alert_metric"
+      //        "timestampField": "timestamp"
+      //      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/000b7460/eagle-core/eagle-alert-parent/eagle-alert-service/src/main/java/org/apache/eagle/service/alert/AlertPolicyValidateProvider.java
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-alert-parent/eagle-alert-service/src/main/java/org/apache/eagle/service/alert/AlertPolicyValidateProvider.java b/eagle-core/eagle-alert-parent/eagle-alert-service/src/main/java/org/apache/eagle/service/alert/AlertPolicyValidateProvider.java
deleted file mode 100644
index 47ddcb9..0000000
--- a/eagle-core/eagle-alert-parent/eagle-alert-service/src/main/java/org/apache/eagle/service/alert/AlertPolicyValidateProvider.java
+++ /dev/null
@@ -1,36 +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.eagle.service.alert;
-
-import java.util.List;
-
-import org.apache.eagle.log.entity.GenericServiceAPIResponseEntity;
-import com.fasterxml.jackson.annotation.JsonIgnoreProperties;
-import com.fasterxml.jackson.annotation.JsonTypeInfo;
-import com.fasterxml.jackson.databind.Module;
-
-@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type", visible=true)
-@JsonIgnoreProperties(ignoreUnknown = true)
-public abstract class AlertPolicyValidateProvider {
-	public String type;
-
-	public abstract GenericServiceAPIResponseEntity<String> validate();
-	
-	public abstract String PolicyType();
-	
-	public abstract List<Module> BindingModules();
-}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/000b7460/eagle-core/eagle-alert-parent/eagle-alert/alert-common/src/main/java/org/apache/eagle/alert/config/ZKConfigBuilder.java
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-alert-parent/eagle-alert/alert-common/src/main/java/org/apache/eagle/alert/config/ZKConfigBuilder.java b/eagle-core/eagle-alert-parent/eagle-alert/alert-common/src/main/java/org/apache/eagle/alert/config/ZKConfigBuilder.java
index 9d77a58..d7b3232 100644
--- a/eagle-core/eagle-alert-parent/eagle-alert/alert-common/src/main/java/org/apache/eagle/alert/config/ZKConfigBuilder.java
+++ b/eagle-core/eagle-alert-parent/eagle-alert/alert-common/src/main/java/org/apache/eagle/alert/config/ZKConfigBuilder.java
@@ -29,10 +29,10 @@ public class ZKConfigBuilder {
         ZKConfig zkConfig = new ZKConfig();
         zkConfig.zkQuorum = config.getString("zkConfig.zkQuorum");
         zkConfig.zkRoot = config.getString("zkConfig.zkRoot");
-        zkConfig.zkSessionTimeoutMs = config.getInt("zkConfig.zkSessionTimeoutMs");
-        zkConfig.connectionTimeoutMs = config.getInt("zkConfig.connectionTimeoutMs");
-        zkConfig.zkRetryTimes = config.getInt("zkConfig.zkRetryTimes");
-        zkConfig.zkRetryInterval = config.getInt("zkConfig.zkRetryInterval");
+        zkConfig.zkSessionTimeoutMs = config.hasPath("zkConfig.zkSessionTimeoutMs") ? config.getInt("zkConfig.zkSessionTimeoutMs") : 10000;
+        zkConfig.connectionTimeoutMs = config.hasPath("zkConfig.connectionTimeoutMs") ? config.getInt("zkConfig.connectionTimeoutMs") : 10000;
+        zkConfig.zkRetryTimes = config.hasPath("zkConfig.zkRetryTimes") ? config.getInt("zkConfig.zkRetryTimes") : 3;
+        zkConfig.zkRetryInterval = config.hasPath("zkConfig.zkRetryInterval") ? config.getInt("zkConfig.zkRetryInterval") : 3000;
         return zkConfig;
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/000b7460/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/main/java/org/apache/eagle/alert/engine/scheme/JsonStringStreamNameSelector.java
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/main/java/org/apache/eagle/alert/engine/scheme/JsonStringStreamNameSelector.java b/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/main/java/org/apache/eagle/alert/engine/scheme/JsonStringStreamNameSelector.java
index 1182e3f..226dd84 100644
--- a/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/main/java/org/apache/eagle/alert/engine/scheme/JsonStringStreamNameSelector.java
+++ b/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/main/java/org/apache/eagle/alert/engine/scheme/JsonStringStreamNameSelector.java
@@ -33,9 +33,9 @@ import org.slf4j.LoggerFactory;
  */
 public class JsonStringStreamNameSelector implements StreamNameSelector {
     private final static Logger LOG = LoggerFactory.getLogger(JsonStringStreamNameSelector.class);
-    private final static String USER_PROVIDED_STREAM_NAME_PROPERTY = "userProvidedStreamName";
-    private final static String FIELD_NAMES_TO_INFER_STREAM_NAME_PROPERTY = "fieldNamesToInferStreamName";
-    private final static String STREAM_NAME_FORMAT = "streamNameFormat";
+    public final static String USER_PROVIDED_STREAM_NAME_PROPERTY = "userProvidedStreamName";
+    public final static String FIELD_NAMES_TO_INFER_STREAM_NAME_PROPERTY = "fieldNamesToInferStreamName";
+    public final static String STREAM_NAME_FORMAT = "streamNameFormat";
 
     private String userProvidedStreamName;
     private String[] fieldNamesToInferStreamName;

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/000b7460/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/main/java/org/apache/eagle/alert/engine/spout/CorrelationSpout.java
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/main/java/org/apache/eagle/alert/engine/spout/CorrelationSpout.java b/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/main/java/org/apache/eagle/alert/engine/spout/CorrelationSpout.java
index f54d5cd..99c1fed 100644
--- a/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/main/java/org/apache/eagle/alert/engine/spout/CorrelationSpout.java
+++ b/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/main/java/org/apache/eagle/alert/engine/spout/CorrelationSpout.java
@@ -332,9 +332,9 @@ public class CorrelationSpout extends BaseRichSpout implements SpoutSpecListener
             spoutConfig.zkPort = utils.getZkPort();
         }
         // transaction update interval
-        spoutConfig.stateUpdateIntervalMs = config.getLong("spout.stormKafkaStateUpdateIntervalMs");
+        spoutConfig.stateUpdateIntervalMs = config.hasPath("spout.stormKafkaStateUpdateIntervalMs") ? config.getInt("spout.stormKafkaStateUpdateIntervalMs") : 2000;
         // Kafka fetch size
-        spoutConfig.fetchSizeBytes = config.getInt("spout.stormKafkaFetchSizeBytes");
+        spoutConfig.fetchSizeBytes = config.hasPath("spout.stormKafkaFetchSizeBytes") ? config.getInt("spout.stormKafkaFetchSizeBytes") : 1048586;
         // "startOffsetTime" is for test usage, prod should not use this
         if (config.hasPath("spout.stormKafkaStartOffsetTime")) {
             spoutConfig.startOffsetTime = config.getInt("spout.stormKafkaStartOffsetTime");

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/000b7460/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/main/resources/application.conf
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/main/resources/application.conf b/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/main/resources/application.conf
index 7030e45..dac2f07 100644
--- a/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/main/resources/application.conf
+++ b/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/main/resources/application.conf
@@ -29,32 +29,17 @@
     "stormKafkaUseSameZkQuorumWithKafkaBroker": true,
     "stormKafkaTransactionZkQuorum": "",
     "stormKafkaTransactionZkPath": "/consumers",
-    "stormKafkaEagleConsumer": "eagle_consumer",
-    "stormKafkaStateUpdateIntervalMs": 2000,
-    "stormKafkaFetchSizeBytes": 1048586,
+    "stormKafkaEagleConsumer": "eagle_consumer"
   },
   "zkConfig" : {
     "zkQuorum" : "server.eagle.apache.org:2181",
-    "zkRoot" : "/alert",
-    "zkSessionTimeoutMs" : 10000,
-    "connectionTimeoutMs" : 10000,
-    "zkRetryTimes" : 3,
-    "zkRetryInterval" : 3000
-  },
-  "dynamicConfigSource" : {
-    "initDelayMillis": 3000,
-    "delayMillis" : 10000
+    "zkRoot" : "/alert"
   },
   "metadataService": {
     "context" : "/rest",
     "host" : "localhost",
     "port" : 9090
   },
-  "coordinatorService": {
-    "host": "localhost",
-    "port": 9090,
-    "context" : "/rest"
-  }
   "metric":{
     "sink": {
       // "kafka": {

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/000b7460/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/test/java/org/apache/eagle/alert/engine/e2e/CoordinatorClient.java
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/test/java/org/apache/eagle/alert/engine/e2e/CoordinatorClient.java b/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/test/java/org/apache/eagle/alert/engine/e2e/CoordinatorClient.java
index aebf3b5..0cf5115 100644
--- a/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/test/java/org/apache/eagle/alert/engine/e2e/CoordinatorClient.java
+++ b/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/test/java/org/apache/eagle/alert/engine/e2e/CoordinatorClient.java
@@ -41,9 +41,9 @@ public class CoordinatorClient implements Closeable {
     @SuppressWarnings("unused")
     private static final Logger LOG = LoggerFactory.getLogger(CoordinatorClient.class);
 
-    private static final String EAGLE_COORDINATOR_SERVICE_CONTEXT = "coordinatorService.context";
-    private static final String EAGLE_COORDINATOR_SERVICE_PORT = "coordinatorService.port";
-    private static final String EAGLE_COORDINATOR_SERVICE_HOST = "coordinatorService.host";
+    private static final String EAGLE_COORDINATOR_SERVICE_CONTEXT = "metadataService.context";
+    private static final String EAGLE_COORDINATOR_SERVICE_PORT = "metadataService.port";
+    private static final String EAGLE_COORDINATOR_SERVICE_HOST = "metadataService.host";
     private static final String COORDINATOR_SCHEDULE_API = "/coordinator/build";
 
     private String host;

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/000b7460/eagle-core/eagle-alert-parent/pom.xml
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-alert-parent/pom.xml b/eagle-core/eagle-alert-parent/pom.xml
index d5166cb..e2ba4d8 100644
--- a/eagle-core/eagle-alert-parent/pom.xml
+++ b/eagle-core/eagle-alert-parent/pom.xml
@@ -15,9 +15,7 @@
   ~ See the License for the specific language governing permissions and
   ~ limitations under the License.
   -->
-
-<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
-         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
     <modelVersion>4.0.0</modelVersion>
     <parent>
         <groupId>org.apache.eagle</groupId>
@@ -33,5 +31,6 @@
     <modules>
         <module>eagle-alert</module>
         <module>eagle-alert-service</module>
-    </modules>
-</project>
+    <module>eagle-alert-app</module>
+  </modules>
+</project>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/000b7460/eagle-core/eagle-app/eagle-app-base/src/main/java/org/apache/eagle/app/service/ApplicationContext.java
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-app/eagle-app-base/src/main/java/org/apache/eagle/app/service/ApplicationContext.java b/eagle-core/eagle-app/eagle-app-base/src/main/java/org/apache/eagle/app/service/ApplicationContext.java
index 52eb628..1a80b78 100644
--- a/eagle-core/eagle-app/eagle-app-base/src/main/java/org/apache/eagle/app/service/ApplicationContext.java
+++ b/eagle-core/eagle-app/eagle-app-base/src/main/java/org/apache/eagle/app/service/ApplicationContext.java
@@ -102,9 +102,9 @@ public class ApplicationContext implements Serializable, ApplicationLifecycle {
                     datasource.setTopic(kafkaCfg.getTopicId());
                     datasource.setSchemeCls(JsonScheme.class.getCanonicalName());
                     Tuple2StreamMetadata tuple2Stream = new Tuple2StreamMetadata();
-                    Set<String> activeStreamNames = new HashSet<>();
-                    activeStreamNames.add(streamDesc.getSchema().getStreamId());
-                    tuple2Stream.setActiveStreamNames(activeStreamNames);
+                    Properties prop = new Properties();
+                    prop.put(JsonStringStreamNameSelector.USER_PROVIDED_STREAM_NAME_PROPERTY, streamDesc.getStreamId());
+                    tuple2Stream.setStreamNameSelectorProp(prop);
                     tuple2Stream.setTimestampColumn("timestamp");
                     tuple2Stream.setStreamNameSelectorCls(JsonStringStreamNameSelector.class.getCanonicalName());
                     datasource.setCodec(tuple2Stream);

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/000b7460/eagle-core/eagle-app/eagle-app-base/src/main/resources/log4j.properties
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-app/eagle-app-base/src/main/resources/log4j.properties b/eagle-core/eagle-app/eagle-app-base/src/main/resources/log4j.properties
index fb13ad5..d59ded6 100644
--- a/eagle-core/eagle-app/eagle-app-base/src/main/resources/log4j.properties
+++ b/eagle-core/eagle-app/eagle-app-base/src/main/resources/log4j.properties
@@ -13,7 +13,7 @@
 # See the License for the specific language governing permissions and
 # limitations under the License.
 
-log4j.rootLogger=DEBUG, stdout
+log4j.rootLogger=INFO, stdout
 
 # standard output
 log4j.appender.stdout=org.apache.log4j.ConsoleAppender

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/000b7460/eagle-security/eagle-security-hdfs-auditlog/README.md
----------------------------------------------------------------------
diff --git a/eagle-security/eagle-security-hdfs-auditlog/README.md b/eagle-security/eagle-security-hdfs-auditlog/README.md
new file mode 100644
index 0000000..8d2e97a
--- /dev/null
+++ b/eagle-security/eagle-security-hdfs-auditlog/README.md
@@ -0,0 +1,191 @@
+<!--
+{% comment %}
+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.
+{% endcomment %}
+-->
+
+Development in IDE
+
+## 1. Start eagle-server
+In IDE, configure the following main class and program arguments
+
+org.apache.eagle.server.ServerMain server src/main/resources/configuration.yml
+
+## 2. Start alert engine
+
+### 2.1 Create new site
+
+http://localhost:9090/rest/sites POST
+```
+{
+"siteId" : "testsite",
+"siteName" :"testsite",
+"description" : "test description",
+"context" : {}
+}
+```
+
+###n2.2 Create logic alert engine topology
+
+http://localhost:9090/rest/metadata/topologies POST
+```
+{
+   "name": "alertUnitTopology_1",
+   "numOfSpout": 1,
+   "numOfAlertBolt": 10,
+   "numOfGroupBolt": 4,
+   "spoutId": "alertEngineSpout",
+   "groupNodeIds": [
+      "streamRouterBolt0",
+      "streamRouterBolt1",
+      "streamRouterBolt2",
+      "streamRouterBolt3"
+   ],
+   "alertBoltIds": [
+      "alertBolt0",
+      "alertBolt1",
+      "alertBolt2",
+      "alertBolt3",
+      "alertBolt4",
+      "alertBolt5",
+      "alertBolt6",
+      "alertBolt7",
+      "alertBolt8",
+      "alertBolt9"
+   ],
+   "pubBoltId": "alertPublishBolt",
+   "spoutParallelism": 1,
+   "groupParallelism": 1,
+   "alertParallelism": 1
+}
+```
+
+### 2.3 Install alert engine application
+Please reference eagle-core/eagle-alert-parent/eagle-alert-app/src/main/resources/META-INF/providers/org.apache.eagle.alert.app.AlertUnitTopologyAppProvider.xml for
+complete configuration.
+
+http://localhost:9090/rest/apps/install POST
+```
+{
+"siteId" : "testsite",
+"appType" : "AlertUnitTopologyApp",
+"mode" : "LOCAL",
+"configuration" : {
+  }
+}
+```
+
+### 2.4 Run alert engine
+Please use correct uuid
+
+http://localhost:9090/rest/apps/start POST
+```
+{
+"uuid": "dc61c4b8-f60d-4d95-bfd7-f6b07382a3f3",
+"appId": "AlertUnitTopologyApp\uff0dtestsite"
+}
+```
+
+## 3 Start Hdfs audit log monitoring application
+
+### 3.1 Install HdfsAuditLog app
+
+http://localhost:9090/rest/apps/install POST
+```
+{
+"siteId" : "testsite",
+"appType" : "HdfsAuditLogApplication",
+"mode" : "LOCAL",
+"configuration" : {
+  "dataSourceConfig.topic" :"hdfs_audit_log"}
+}
+```
+
+### 3.2 Start HdfsAuditLog app
+Please use correct uuid
+
+http://localhost:9090/rest/apps/start POST
+```
+{
+"uuid": "dc61c4b8-f60d-4d95-bfd7-f6b07382a3f3",
+"appId": "HdfsAuditLogApplication\uff0dtestsite"
+}
+```
+
+## 4 Check
+### 4.1 Check if alert data source is created
+http://localhost:9090/rest/metadata/datasources GET
+
+### 4.2 Check if alert stream is creatd
+http://localhost:9090/rest/metadata/streams GET
+
+## 5 Create alert policy and verify alert
+### 5.1 create one policy
+
+http://localhost:9090/rest/metadata/policies POST
+```
+{
+   "name": "hdfsPolicy",
+   "description": "hdfsPolicy",
+   "inputStreams": [
+      "hdfs_audit_log_enriched_stream"
+   ],
+   "outputStreams": [
+      "hdfs_audit_log_enriched_stream_out"
+   ],
+   "definition": {
+      "type": "siddhi",
+      "value": "from hdfs_audit_log_enriched_stream[user=='hadoop'] select * insert into hdfs_audit_log_enriched_stream_out"
+   },
+   "partitionSpec": [
+      {
+         "streamId": "hdfs_audit_log_enriched_stream",
+         "type": "GROUPBY",
+         "columns" : [
+            "user"
+         ]
+      }
+   ],
+   "parallelismHint": 2
+}
+```
+
+### 5.2 Create alert publishment
+```
+{
+	"name":"hdfs_audit_log_enriched_stream_out",
+	"type":"org.apache.eagle.alert.engine.publisher.impl.AlertEmailPublisher",
+	"policyIds": [
+		"hdfsPolicy"
+	],
+	"properties": {
+	  "subject":"alert when user is hadoop",
+	  "template":"",
+	  "sender": "eagle@apache.org",
+	  "recipients": "eagle@apache.org",
+	  "mail.smtp.host":"",
+	  "connection": "plaintext",
+	  "mail.smtp.port": "25"
+	},
+	"dedupIntervalMin" : "PT1M",
+	"serializer" : "org.apache.eagle.alert.engine.publisher.impl.StringEventSerializer"
+}
+```
+
+### 5.3 Send message and verify alert
+./kafka-console-producer.sh --topic hdfs_audit_log --broker-list sandbox.hortonworks.com:6667
+
+2015-04-24 12:51:31,798 INFO FSNamesystem.audit: allowed=true	ugi=hdfs (auth:SIMPLE)	ip=/10.0.2.15	cmd=getfileinfo	src=/apps/hbase/data	dst=null	perm=null	proto=rpc

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/000b7460/eagle-security/eagle-security-hdfs-auditlog/src/main/resources/META-INF/providers/org.apache.eagle.security.auditlog.HdfsAuditLogAppProvider.xml
----------------------------------------------------------------------
diff --git a/eagle-security/eagle-security-hdfs-auditlog/src/main/resources/META-INF/providers/org.apache.eagle.security.auditlog.HdfsAuditLogAppProvider.xml b/eagle-security/eagle-security-hdfs-auditlog/src/main/resources/META-INF/providers/org.apache.eagle.security.auditlog.HdfsAuditLogAppProvider.xml
index 2a8ff0f..2fb7274 100644
--- a/eagle-security/eagle-security-hdfs-auditlog/src/main/resources/META-INF/providers/org.apache.eagle.security.auditlog.HdfsAuditLogAppProvider.xml
+++ b/eagle-security/eagle-security-hdfs-auditlog/src/main/resources/META-INF/providers/org.apache.eagle.security.auditlog.HdfsAuditLogAppProvider.xml
@@ -155,21 +155,41 @@
             <timeseries>true</timeseries>
             <columns>
                 <column>
-                    <name>action</name>
+                    <name>src</name>
                     <type>string</type>
                 </column>
                 <column>
-                    <name>host</name>
+                    <name>dst</name>
                     <type>string</type>
                 </column>
                 <column>
-                    <name>status</name>
+                    <name>host</name>
                     <type>string</type>
                 </column>
                 <column>
                     <name>timestamp</name>
                     <type>long</type>
                 </column>
+                <column>
+                    <name>allowed</name>
+                    <type>bool</type>
+                </column>
+                <column>
+                    <name>user</name>
+                    <type>string</type>
+                </column>
+                <column>
+                    <name>cmd</name>
+                    <type>string</type>
+                </column>
+                <column>
+                    <name>sensitivityType</name>
+                    <type>string</type>
+                </column>
+                <column>
+                    <name>securityZone</name>
+                    <type>string</type>
+                </column>
             </columns>
         </stream>
     </streams>

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/000b7460/eagle-security/eagle-security-hdfs-auditlog/src/main/resources/scripts.txt
----------------------------------------------------------------------
diff --git a/eagle-security/eagle-security-hdfs-auditlog/src/main/resources/scripts.txt b/eagle-security/eagle-security-hdfs-auditlog/src/main/resources/scripts.txt
new file mode 100644
index 0000000..c5f7c9b
--- /dev/null
+++ b/eagle-security/eagle-security-hdfs-auditlog/src/main/resources/scripts.txt
@@ -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.
+
+./kafka-topics.sh --topic hdfs_audit_log --zookeeper localhost:2181 --create --replication-factor 1 --partitions 2
+
+./kafka-topics.sh --topic hdfs_audit_log --zookeeper localhost:2181 --create --replication-factor 1 --partitions 2
+
+./kafka-console-producer.sh --topic hdfs_audit_log --broker-list sandbox.hortonworks.com:6667
+
+./kafka-console-consumer.sh --topic hdfs_audit_log_parsed --zookeeper sandbox.hortonworks.com:2181 --from-beginning

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/000b7460/eagle-server/pom.xml
----------------------------------------------------------------------
diff --git a/eagle-server/pom.xml b/eagle-server/pom.xml
index bc9a919..5e07a7d 100644
--- a/eagle-server/pom.xml
+++ b/eagle-server/pom.xml
@@ -164,6 +164,13 @@
             <artifactId>eagle-gc</artifactId>
             <version>${project.version}</version>
         </dependency>
+
+        <!-- Alert unit topology as an application -->
+        <dependency>
+            <groupId>org.apache.eagle</groupId>
+            <artifactId>eagle-alert-app</artifactId>
+            <version>${project.version}</version>
+        </dependency>
     </dependencies>
     <build>
         <resources>



[11/52] [abbrv] incubator-eagle git commit: EAGLE-483 Test email alert publisher Test email alert publisher: configuration and connecting authentication.

Posted by yo...@apache.org.
EAGLE-483 Test email alert publisher
Test email alert publisher: configuration and connecting authentication.

https://issues.apache.org/jira/browse/EAGLE-483

Author: @pkuwm <ih...@gmail.com>
Reviewer: @DadanielZ <da...@apache.org>

Closes #365


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

Branch: refs/heads/master
Commit: 24e6622cacf6189fc04dbf7dfeb32eb123fcbc35
Parents: b54a63e
Author: DadanielZ <da...@apache.org>
Authored: Wed Aug 24 14:51:11 2016 -0700
Committer: DadanielZ <da...@apache.org>
Committed: Wed Aug 24 14:51:11 2016 -0700

----------------------------------------------------------------------
 .../eagle/alert/engine/runner/AlertBolt.java    |  4 +-
 .../src/test/resources/publishments1.json       | 41 +++++++++++---------
 .../src/test/resources/publishments2.json       | 39 +++++++++++--------
 .../src/test/resources/testPublishSpec.json     | 20 ++++++----
 .../src/test/resources/testPublishSpec2.json    | 20 ++++++----
 5 files changed, 71 insertions(+), 53 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/24e6622c/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/main/java/org/apache/eagle/alert/engine/runner/AlertBolt.java
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/main/java/org/apache/eagle/alert/engine/runner/AlertBolt.java b/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/main/java/org/apache/eagle/alert/engine/runner/AlertBolt.java
index 86c8b3d..ce5125b 100755
--- a/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/main/java/org/apache/eagle/alert/engine/runner/AlertBolt.java
+++ b/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/main/java/org/apache/eagle/alert/engine/runner/AlertBolt.java
@@ -115,8 +115,8 @@ public class AlertBolt extends AbstractStreamBolt implements AlertBoltSpecListen
                         specVersion.contains("spec_version_") && stream_event_version.contains("spec_version_")){
 //                    Long timestamp_of_specVersion = Long.valueOf(specVersion.split("spec_version_")[1]);
 //                    Long timestamp_of_streamEventVersion = Long.valueOf(stream_event_version.split("spec_version_")[1]);
-                    Long timestamp_of_specVersion = Long.valueOf(specVersion.substring(13));
-                    Long timestamp_of_streamEventVersion = Long.valueOf(stream_event_version.substring(13));
+                    long timestamp_of_specVersion = Long.valueOf(specVersion.substring(13));
+                    long timestamp_of_streamEventVersion = Long.valueOf(stream_event_version.substring(13));
                     specVersionOutofdate = timestamp_of_specVersion < timestamp_of_streamEventVersion;
                     if (!specVersionOutofdate){
                         pe.getEvent().setMetaVersion(specVersion);

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/24e6622c/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/test/resources/publishments1.json
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/test/resources/publishments1.json b/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/test/resources/publishments1.json
index 2e04e78..0b2d727 100644
--- a/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/test/resources/publishments1.json
+++ b/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/test/resources/publishments1.json
@@ -1,20 +1,25 @@
 [
-{
-	"name":"test-stream-output",
-	"type":"org.apache.eagle.alert.engine.publisher.impl.AlertEmailPublisher",
-	"policyIds": [
-		"perfmon_cpu_host_check", "perfmon_cpu_pool_check"
-	],
-	"properties": {
-	  "subject":"UMP Test Alert",
-	  "template":"",
-	  "sender": "sender@corp.com",
-	  "recipients": "services@corp.com",
-	  "smtp.server":"sender.mailhost.com",
-	  "connection": "plaintext",
-	  "smtp.port": "25"
-	},
-	"dedupIntervalMin" : "PT1M",
-	"serializer" : "org.apache.eagle.alert.engine.publisher.impl.StringEventSerializer"
-}
+  {
+    "name": "test-stream-output",
+    "type": "org.apache.eagle.alert.engine.publisher.impl.AlertEmailPublisher",
+    "policyIds": [
+      "perfmon_cpu_host_check",
+      "perfmon_cpu_pool_check"
+    ],
+    "properties": {
+      "subject": "Test Alert",
+      "template": "",
+      "sender": "sender@corp.com",
+      "recipients": "services@corp.com",
+      "mail.smtp.host": "smtp.mailhost.com",
+      "mail.smtp.auth": "true",
+      "mail.username": "username",
+      "mail.password": "password",
+      "mail.debug": "false",
+      "mail.connection": "tls",
+      "mail.smtp.port": "587"
+    },
+    "dedupIntervalMin": "PT1M",
+    "serializer": "org.apache.eagle.alert.engine.publisher.impl.StringEventSerializer"
+  }
 ]
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/24e6622c/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/test/resources/publishments2.json
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/test/resources/publishments2.json b/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/test/resources/publishments2.json
index 277ba75..6a4918a 100644
--- a/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/test/resources/publishments2.json
+++ b/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/test/resources/publishments2.json
@@ -1,19 +1,24 @@
 [
-{
-	"name":"test-stream-output",
-	"type":"org.apache.eagle.alert.engine.publisher.impl.AlertEmailPublisher",
-	"policyIds": [
-		"policy1", "policy2"
-	],
-	"properties": {
-	  "subject":"UMP Test Alert",
-	  "template":"",
-	  "sender": "sender@corp.com",
-	  "recipients": "receiver@corp.com",
-	  "smtp.server":"mailhost.com",
-	  "connection": "plaintext",
-	  "smtp.port": "25"
-	},
-	"dedupIntervalMin" : "PT0M"
-}
+  {
+    "name": "test-stream-output",
+    "type": "org.apache.eagle.alert.engine.publisher.impl.AlertEmailPublisher",
+    "policyIds": [
+      "policy1",
+      "policy2"
+    ],
+    "properties": {
+      "subject": "Test Alert",
+      "template": "",
+      "sender": "sender@corp.com",
+      "recipients": "receiver@corp.com",
+      "mail.smtp.host": "smtp.mailhost.com",
+      "mail.smtp.auth": "true",
+      "mail.username": "username",
+      "mail.password": "password",
+      "mail.debug": "false",
+      "mail.connection": "tls",
+      "mail.smtp.port": "587"
+    },
+    "dedupIntervalMin": "PT0M"
+  }
 ]
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/24e6622c/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/test/resources/testPublishSpec.json
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/test/resources/testPublishSpec.json b/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/test/resources/testPublishSpec.json
index 220c8fe..66b3852 100644
--- a/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/test/resources/testPublishSpec.json
+++ b/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/test/resources/testPublishSpec.json
@@ -8,14 +8,18 @@
       "name":"email-testAlertStream",
       "policyIds": ["policy1", "policy2", "policy3"],
       "dedupIntervalMin": "PT1M",
-      "properties":{
-        "subject":"UMP Test Alert",
-        "template":"",
-	  "sender": "sender@corp.com",
-	  "recipients": "receiver@corp.com",
-	  "smtp.server":"mailhost.com",
-        "connection": "plaintext",
-        "smtp.port": "25"
+      "properties": {
+        "subject": "Test Alert",
+        "template": "",
+        "sender": "sender@corp.com",
+        "recipients": "receiver@corp.com",
+        "mail.smtp.host": "smtp.mailhost.com",
+        "mail.smtp.auth": "true",
+        "mail.username": "username",
+        "mail.password": "password",
+        "mail.debug": "false",
+        "mail.connection": "tls",
+        "mail.smtp.port": "587"
       }
     }
 /*    {

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/24e6622c/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/test/resources/testPublishSpec2.json
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/test/resources/testPublishSpec2.json b/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/test/resources/testPublishSpec2.json
index 0a16540..0108a1e 100644
--- a/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/test/resources/testPublishSpec2.json
+++ b/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/test/resources/testPublishSpec2.json
@@ -8,14 +8,18 @@
       "name":"email-testAlertStream",
       "policyIds": ["policy1"],
       "dedupIntervalMin": "PT2M",
-      "properties":{
-        "subject":"UMP Test Alert",
-        "template":"",
-	  "sender": "sender@corp.com",
-	  "recipients": "receiver@corp.com",
-	  "smtp.server":"mailhost.com",
-        "connection": "plaintext",
-        "smtp.port": "25"
+      "properties": {
+        "subject": "Test Alert",
+        "template": "",
+        "sender": "sender@corp.com",
+        "recipients": "receiver@corp.com",
+        "mail.smtp.host": "smtp.mailhost.com",
+        "mail.smtp.auth": "true",
+        "mail.username": "username",
+        "mail.password": "password",
+        "mail.debug": "false",
+        "mail.connection": "tls",
+        "mail.smtp.port": "587"
       }
     }
 //    {


[20/52] [abbrv] incubator-eagle git commit: [EAGLE-499] Fix flushing entities for spark history job doFlush() was commented.. Adding back.

Posted by yo...@apache.org.
[EAGLE-499] Fix flushing entities for spark history job
doFlush() was commented.. Adding back.

https://issues.apache.org/jira/browse/EAGLE-499

Author: @pkuwm <ih...@gmail.com>
Reviewer: @DadanielZ <da...@apache.org>

Closes #385


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

Branch: refs/heads/master
Commit: 36f9a8481e507d3ac0a4522da28a600f0ec6d451
Parents: c5d05ab
Author: DadanielZ <da...@apache.org>
Authored: Thu Aug 25 14:41:28 2016 -0700
Committer: DadanielZ <da...@apache.org>
Committed: Thu Aug 25 14:41:28 2016 -0700

----------------------------------------------------------------------
 .../java/org/apache/eagle/jpm/spark/crawl/JHFSparkEventReader.java  | 1 +
 1 file changed, 1 insertion(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/36f9a848/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/spark/crawl/JHFSparkEventReader.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/spark/crawl/JHFSparkEventReader.java b/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/spark/crawl/JHFSparkEventReader.java
index a5e630a..edb3854 100644
--- a/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/spark/crawl/JHFSparkEventReader.java
+++ b/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/spark/crawl/JHFSparkEventReader.java
@@ -709,6 +709,7 @@ public class JHFSparkEventReader {
 
     private void doFlush(List entities) throws Exception {
         LOG.info("start flushing entities of total number " + entities.size());
+        client.create(entities);
         LOG.info("finish flushing entities of total number " + entities.size());
     }
 }
\ No newline at end of file


[51/52] [abbrv] incubator-eagle git commit: [EAGLE-520] Fix and decouple co-processor from eagle aggreation query service

Posted by yo...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/21187b55/eagle-core/eagle-query/eagle-entity-base/src/main/java/org/apache/eagle/log/base/taggedlog/TaggedLogAPIEntity.java
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-query/eagle-entity-base/src/main/java/org/apache/eagle/log/base/taggedlog/TaggedLogAPIEntity.java b/eagle-core/eagle-query/eagle-entity-base/src/main/java/org/apache/eagle/log/base/taggedlog/TaggedLogAPIEntity.java
index b001955..b396b06 100755
--- a/eagle-core/eagle-query/eagle-entity-base/src/main/java/org/apache/eagle/log/base/taggedlog/TaggedLogAPIEntity.java
+++ b/eagle-core/eagle-query/eagle-entity-base/src/main/java/org/apache/eagle/log/base/taggedlog/TaggedLogAPIEntity.java
@@ -16,6 +16,21 @@
  */
 package org.apache.eagle.log.base.taggedlog;
 
+import org.apache.eagle.common.DateTimeUtil;
+import org.apache.eagle.log.entity.meta.EntityDefinitionManager;
+
+import com.fasterxml.jackson.annotation.JsonFilter;
+import com.fasterxml.jackson.core.JsonFactory;
+import com.fasterxml.jackson.core.JsonGenerator;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.SerializerProvider;
+import com.fasterxml.jackson.databind.ser.FilterProvider;
+import com.fasterxml.jackson.databind.ser.PropertyWriter;
+import com.fasterxml.jackson.databind.ser.impl.SimpleBeanPropertyFilter;
+import com.fasterxml.jackson.databind.ser.impl.SimpleFilterProvider;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
 import java.beans.PropertyChangeEvent;
 import java.beans.PropertyChangeListener;
 import java.beans.PropertyChangeSupport;
@@ -26,210 +41,208 @@ import java.util.HashSet;
 import java.util.Map;
 import java.util.Set;
 
-import org.apache.eagle.log.entity.meta.EntityDefinitionManager;
-import org.codehaus.jackson.JsonFactory;
-import org.codehaus.jackson.JsonGenerator;
-import org.codehaus.jackson.map.ObjectMapper;
-import org.codehaus.jackson.map.SerializerProvider;
-import org.codehaus.jackson.map.annotate.JsonFilter;
-import org.codehaus.jackson.map.ser.BeanPropertyWriter;
-import org.codehaus.jackson.map.ser.FilterProvider;
-import org.codehaus.jackson.map.ser.impl.SimpleBeanPropertyFilter;
-import org.codehaus.jackson.map.ser.impl.SimpleFilterProvider;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import org.apache.eagle.common.DateTimeUtil;
-
 /**
  * rowkey: prefix + timestamp + tagNameValues
  * as of now, all tags will be persisted as a column in hbase table
  * tag name is column qualifier name
- * tag value is column value
+ * tag value is column value.
  */
 @JsonFilter(TaggedLogAPIEntity.PropertyBeanFilterName)
-public class TaggedLogAPIEntity implements PropertyChangeListener, Serializable{
-	private static final Logger LOG = LoggerFactory.getLogger(TaggedLogAPIEntity.class);
-	private String prefix;
-	private long timestamp;
-	private Map<String, String> tags;
-
-	public void setExp(Map<String, Object> exp) {
-		this.exp = exp;
-	}
-
-	public Map<String,Object> getExp() {
-		return this.exp;
-	}
-
-	/**
-	 * Extra dynamic attributes. 
-	 * TODO: can we move exp, _serializeAlias, _serializeVerbose to a wrapper class? 
-	 */
-	private Map<String,Object> exp;
-
-	private String encodedRowkey;
-	// track what qualifiers are changed
-	private Set<String> _modifiedProperties = new HashSet<String>();
-	protected PropertyChangeSupport _pcs = new PropertyChangeSupport(this);
-
-
-	public Map<String, String> getSerializeAlias() {
-		return _serializeAlias;
-	}
-
-	public void setSerializeAlias(Map<String, String> _serializeAlias) {
-		this._serializeAlias = _serializeAlias;
-	}
-
-	private Map<String,String> _serializeAlias = null;
-
-	public boolean isSerializeVerbose() {
-		return _serializeVerbose;
-	}
-
-	public void setSerializeVerbose(boolean _serializeVerbose) {
-		this._serializeVerbose = _serializeVerbose;
-	}
-
-	private boolean _serializeVerbose = true;
-
-	public TaggedLogAPIEntity(){
-		_pcs.addPropertyChangeListener(this);
-	}
-	
-	public String getPrefix() {
-		return prefix;
-	}
-	public void setPrefix(String prefix) {
-		this.prefix = prefix;
-	}
-	public long getTimestamp() {
-		return timestamp;
-	}
-	public void setTimestamp(long timestamp) {
-		this.timestamp = timestamp;
-	}
-	public Map<String, String> getTags() {
-		return tags;
-	}
-	public void setTags(Map<String, String> tags) {
-		this.tags = tags;
-	}
-	public String getEncodedRowkey() {
-		return encodedRowkey;
-	}
-	public void setEncodedRowkey(String encodedRowkey) {
-		this.encodedRowkey = encodedRowkey;
-	}
-	
-	protected void valueChanged(String fieldModified){
-		_pcs.firePropertyChange(fieldModified, null, null);
-	}
-	
-	public void propertyChange(PropertyChangeEvent evt) {
-		_modifiedProperties.add(evt.getPropertyName());
-	}
-	
-	public Set<String> modifiedQualifiers(){
-		return this._modifiedProperties;
-	}
-	
-	public String toString(){
-		StringBuffer sb = new StringBuffer();
-		sb.append("prefix:");
-		sb.append(prefix);
-		sb.append(", timestamp:");
-		sb.append(timestamp);
-		sb.append(", humanReadableDate:");
-		sb.append(DateTimeUtil.millisecondsToHumanDateWithMilliseconds(timestamp));
-		sb.append(", tags: ");
-		if(tags != null){
-			for(Map.Entry<String, String> entry : tags.entrySet()){
-				sb.append(entry.toString());
-				sb.append(",");
-			}
-		}
-		sb.append(", encodedRowkey:");
-		sb.append(encodedRowkey);
-		return sb.toString();
-	}
-
-	private static Set<String> getPropertyNames(){
-		if(_propertyNames == null) {
-			Field[] fields = TaggedLogAPIEntity.class.getDeclaredFields();
-			Set<String> fieldName = new HashSet<String>();
-			for (Field f : fields) fieldName.add(f.getName());
-			_propertyNames = fieldName;
-		}
-		return _propertyNames;
-	}
-
-	private static class BeanPropertyFilter extends SimpleBeanPropertyFilter {
-		private final static String prefix = "prefix";
-		private final static String encodedRowkey = "encodedRowkey";
-		private final static String exp = "exp";
-		private final static String timestamp = "timestamp";
-		@SuppressWarnings("serial")
-		private final static Set<String> verboseFields = new HashSet<String>(){{
-			add(prefix);
-			add(encodedRowkey);
-		}};
-
-		@Override
-		public void serializeAsField(Object bean, JsonGenerator jgen, SerializerProvider provider, BeanPropertyWriter writer) throws Exception {
-			if(bean instanceof TaggedLogAPIEntity){
-				TaggedLogAPIEntity entity = (TaggedLogAPIEntity) bean;
-				Set<String> modified = entity.modifiedQualifiers();
-				Set<String> basePropertyNames = getPropertyNames();
-				String writerName = writer.getName();
-				if(modified.contains(writerName) || basePropertyNames.contains(writerName)){
-					if((!entity.isSerializeVerbose() && verboseFields.contains(writerName))||							// skip verbose fields
-							(timestamp.equals(writerName) && !EntityDefinitionManager.isTimeSeries(entity.getClass()))	// skip timestamp for non-timeseries entity
-					){
-						// log skip
-						if(LOG.isDebugEnabled()) LOG.debug("skip field");
-					}else{
-						// if serializeAlias is not null and exp is not null
-						if (exp.equals(writerName) && entity.getSerializeAlias()!=null && entity.getExp()!=null) {
-							Map<String, Object> _exp = new HashMap<String, Object>();
-							for (Map.Entry<String, Object> entry : entity.getExp().entrySet()) {
-								String alias = entity.getSerializeAlias().get(entry.getKey());
-								if (alias != null) {
-									_exp.put(alias, entry.getValue());
-								} else {
-									_exp.put(entry.getKey(), entry.getValue());
-								}
-							}
-							entity.setExp(_exp);
-						}
-						// write included field into serialized json output
-						writer.serializeAsField(bean, jgen, provider);
-					}
-				}
-			}else{
-				writer.serializeAsField(bean, jgen, provider);
-			}
-		}
-	}
-
-	public static FilterProvider getFilterProvider(){
-		if(_filterProvider == null){
-			SimpleFilterProvider _provider = new SimpleFilterProvider();
-			_provider.addFilter(PropertyBeanFilterName,new BeanPropertyFilter());
-			_filterProvider = _provider;
-		}
-		return _filterProvider;
-	}
-
-	//////////////////////////////////////
-	// Static fields
-	//////////////////////////////////////
-	private static Set<String> _propertyNames = null;
-	private static FilterProvider _filterProvider = null;
-	final static String PropertyBeanFilterName = "TaggedLogPropertyBeanFilter";
-
-    public static ObjectMapper buildObjectMapper(){
+public class TaggedLogAPIEntity implements PropertyChangeListener, Serializable {
+    private static final Logger LOG = LoggerFactory.getLogger(TaggedLogAPIEntity.class);
+    private String prefix;
+    private long timestamp;
+    private Map<String, String> tags;
+
+    public void setExp(Map<String, Object> exp) {
+        this.exp = exp;
+    }
+
+    public Map<String, Object> getExp() {
+        return this.exp;
+    }
+
+    /**
+     * Extra dynamic attributes.
+     * TODO: can we move exp, serializeAlias, serializeVerbose to a wrapper class?
+     */
+    private Map<String, Object> exp;
+
+    private String encodedRowkey;
+    // track what qualifiers are changed
+    private Set<String> modifiedProperties = new HashSet<String>();
+    protected PropertyChangeSupport pcs
+            = new PropertyChangeSupport(this);
+
+
+    public Map<String, String> getSerializeAlias() {
+        return serializeAlias;
+    }
+
+    public void setSerializeAlias(Map<String, String> serializeAlias) {
+        this.serializeAlias = serializeAlias;
+    }
+
+    private Map<String, String> serializeAlias = null;
+
+    public boolean isSerializeVerbose() {
+        return serializeVerbose;
+    }
+
+    public void setSerializeVerbose(boolean serializeVerbose) {
+        this.serializeVerbose = serializeVerbose;
+    }
+
+    private boolean serializeVerbose = true;
+
+    public TaggedLogAPIEntity() {
+        pcs.addPropertyChangeListener(this);
+    }
+
+    public String getPrefix() {
+        return prefix;
+    }
+
+    public void setPrefix(String prefix) {
+        this.prefix = prefix;
+    }
+
+    public long getTimestamp() {
+        return timestamp;
+    }
+
+    public void setTimestamp(long timestamp) {
+        this.timestamp = timestamp;
+    }
+
+    public Map<String, String> getTags() {
+        return tags;
+    }
+
+    public void setTags(Map<String, String> tags) {
+        this.tags = tags;
+    }
+
+    public String getEncodedRowkey() {
+        return encodedRowkey;
+    }
+
+    public void setEncodedRowkey(String encodedRowkey) {
+        this.encodedRowkey = encodedRowkey;
+    }
+
+    protected void valueChanged(String fieldModified) {
+        pcs.firePropertyChange(fieldModified, null, null);
+    }
+
+    public void propertyChange(PropertyChangeEvent evt) {
+        modifiedProperties.add(evt.getPropertyName());
+    }
+
+    public Set<String> modifiedQualifiers() {
+        return this.modifiedProperties;
+    }
+
+    public String toString() {
+        StringBuffer sb = new StringBuffer();
+        sb.append("prefix:");
+        sb.append(prefix);
+        sb.append(", timestamp:");
+        sb.append(timestamp);
+        sb.append(", humanReadableDate:");
+        sb.append(DateTimeUtil.millisecondsToHumanDateWithMilliseconds(timestamp));
+        sb.append(", tags: ");
+        if (tags != null) {
+            for (Map.Entry<String, String> entry : tags.entrySet()) {
+                sb.append(entry.toString());
+                sb.append(",");
+            }
+        }
+        sb.append(", encodedRowkey:");
+        sb.append(encodedRowkey);
+        return sb.toString();
+    }
+
+    private static Set<String> getPropertyNames() {
+        if (_propertyNames == null) {
+            Field[] fields = TaggedLogAPIEntity.class.getDeclaredFields();
+            Set<String> fieldName = new HashSet<String>();
+            for (Field f : fields) {
+                fieldName.add(f.getName());
+            }
+            _propertyNames = fieldName;
+        }
+        return _propertyNames;
+    }
+
+    private static class BeanPropertyFilter extends SimpleBeanPropertyFilter {
+        private static final String prefix = "prefix";
+        private static final String encodedRowkey = "encodedRowkey";
+        private static final String exp = "exp";
+        private static final String timestamp = "timestamp";
+        @SuppressWarnings("serial")
+        private static final Set<String> verboseFields = new HashSet<String>() {
+            {
+                add(prefix);
+                add(encodedRowkey);
+            }
+        };
+
+        @Override
+        public void serializeAsField(Object pojo, JsonGenerator jgen, SerializerProvider provider, PropertyWriter writer) throws Exception {
+            if (pojo instanceof TaggedLogAPIEntity) {
+                TaggedLogAPIEntity entity = (TaggedLogAPIEntity) pojo;
+                Set<String> modified = entity.modifiedQualifiers();
+                Set<String> basePropertyNames = getPropertyNames();
+                String writerName = writer.getName();
+                if (modified.contains(writerName) || basePropertyNames.contains(writerName)) {
+                    if ((!entity.isSerializeVerbose() && verboseFields.contains(writerName))
+                            || (timestamp.equals(writerName) && !EntityDefinitionManager.isTimeSeries(entity.getClass()))) {
+                        // log skip
+                        if (LOG.isDebugEnabled()) {
+                            LOG.debug("skip field");
+                        }
+                    } else {
+                        // if serializeAlias is not null and exp is not null
+                        if (exp.equals(writerName) && entity.getSerializeAlias() != null && entity.getExp() != null) {
+                            Map<String, Object> _exp = new HashMap<String, Object>();
+                            for (Map.Entry<String, Object> entry : entity.getExp().entrySet()) {
+                                String alias = entity.getSerializeAlias().get(entry.getKey());
+                                if (alias != null) {
+                                    _exp.put(alias, entry.getValue());
+                                } else {
+                                    _exp.put(entry.getKey(), entry.getValue());
+                                }
+                            }
+                            entity.setExp(_exp);
+                        }
+                        // write included field into serialized json output
+                        writer.serializeAsField(pojo, jgen, provider);
+                    }
+                }
+            } else {
+                writer.serializeAsField(pojo, jgen, provider);
+            }
+        }
+    }
+
+    public static FilterProvider getFilterProvider() {
+        if (_filterProvider == null) {
+            SimpleFilterProvider _provider = new SimpleFilterProvider();
+            _provider.addFilter(PropertyBeanFilterName, new BeanPropertyFilter());
+            _filterProvider = _provider;
+        }
+        return _filterProvider;
+    }
+
+    //////////////////////////////////////
+    // Static fields
+    //////////////////////////////////////
+    private static Set<String> _propertyNames = null;
+    private static FilterProvider _filterProvider = null;
+    static final String PropertyBeanFilterName = "TaggedLogPropertyBeanFilter";
+
+    public static ObjectMapper buildObjectMapper() {
         final JsonFactory factory = new JsonFactory();
         final ObjectMapper mapper = new ObjectMapper(factory);
         mapper.setFilters(TaggedLogAPIEntity.getFilterProvider());

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/21187b55/eagle-core/eagle-query/eagle-entity-base/src/main/java/org/apache/eagle/log/entity/GenericMetricEntity.java
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-query/eagle-entity-base/src/main/java/org/apache/eagle/log/entity/GenericMetricEntity.java b/eagle-core/eagle-query/eagle-entity-base/src/main/java/org/apache/eagle/log/entity/GenericMetricEntity.java
index 90db780..b7260f8 100755
--- a/eagle-core/eagle-query/eagle-entity-base/src/main/java/org/apache/eagle/log/entity/GenericMetricEntity.java
+++ b/eagle-core/eagle-query/eagle-entity-base/src/main/java/org/apache/eagle/log/entity/GenericMetricEntity.java
@@ -48,6 +48,6 @@ public class GenericMetricEntity extends TaggedLogAPIEntity {
 
 	public void setValue(double[] value) {
 		this.value = value;
-		_pcs.firePropertyChange("value", null, null);
+		pcs.firePropertyChange("value", null, null);
 	}
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/21187b55/eagle-core/eagle-query/eagle-entity-base/src/main/java/org/apache/eagle/log/entity/GenericServiceAPIResponseEntity.java
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-query/eagle-entity-base/src/main/java/org/apache/eagle/log/entity/GenericServiceAPIResponseEntity.java b/eagle-core/eagle-query/eagle-entity-base/src/main/java/org/apache/eagle/log/entity/GenericServiceAPIResponseEntity.java
index 2b37d25..6869c7c 100644
--- a/eagle-core/eagle-query/eagle-entity-base/src/main/java/org/apache/eagle/log/entity/GenericServiceAPIResponseEntity.java
+++ b/eagle-core/eagle-query/eagle-entity-base/src/main/java/org/apache/eagle/log/entity/GenericServiceAPIResponseEntity.java
@@ -16,10 +16,10 @@
  */
 package org.apache.eagle.log.entity;
 
+import com.fasterxml.jackson.annotation.JsonIgnoreProperties;
+import com.fasterxml.jackson.databind.annotation.JsonDeserialize;
+import com.fasterxml.jackson.databind.annotation.JsonSerialize;
 import org.apache.eagle.common.EagleExceptionWrapper;
-import org.codehaus.jackson.annotate.JsonIgnoreProperties;
-import org.codehaus.jackson.map.annotate.JsonDeserialize;
-import org.codehaus.jackson.map.annotate.JsonSerialize;
 
 import javax.xml.bind.annotation.XmlAccessType;
 import javax.xml.bind.annotation.XmlAccessorType;
@@ -111,11 +111,8 @@ public class GenericServiceAPIResponseEntity<T>{
 	public String getException() {
 		return exception;
 	}
-//	public void setException(String exception) {
-//		this.exception = exception;
-//	}
 
     public void setException(Exception exceptionObj){
-        if(exception!=null) this.exception = EagleExceptionWrapper.wrap(exceptionObj);
+        this.exception = EagleExceptionWrapper.wrap(exceptionObj);
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/21187b55/eagle-core/eagle-query/eagle-entity-base/src/main/java/org/apache/eagle/log/entity/GenericServiceAPIResponseEntityDeserializer.java
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-query/eagle-entity-base/src/main/java/org/apache/eagle/log/entity/GenericServiceAPIResponseEntityDeserializer.java b/eagle-core/eagle-query/eagle-entity-base/src/main/java/org/apache/eagle/log/entity/GenericServiceAPIResponseEntityDeserializer.java
index 940ee8a..836295b 100644
--- a/eagle-core/eagle-query/eagle-entity-base/src/main/java/org/apache/eagle/log/entity/GenericServiceAPIResponseEntityDeserializer.java
+++ b/eagle-core/eagle-query/eagle-entity-base/src/main/java/org/apache/eagle/log/entity/GenericServiceAPIResponseEntityDeserializer.java
@@ -16,14 +16,15 @@
  */
 package org.apache.eagle.log.entity;
 
-import org.codehaus.jackson.JsonNode;
-import org.codehaus.jackson.JsonParser;
-import org.codehaus.jackson.JsonProcessingException;
-import org.codehaus.jackson.ObjectCodec;
-import org.codehaus.jackson.map.DeserializationContext;
-import org.codehaus.jackson.map.JsonDeserializer;
-import org.codehaus.jackson.map.type.TypeFactory;
-import org.codehaus.jackson.type.JavaType;
+import com.fasterxml.jackson.core.JsonParser;
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.core.ObjectCodec;
+import com.fasterxml.jackson.databind.DeserializationContext;
+import com.fasterxml.jackson.databind.JavaType;
+import com.fasterxml.jackson.databind.JsonDeserializer;
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.type.TypeFactory;
+import com.google.common.base.Preconditions;
 
 import java.io.IOException;
 import java.util.*;
@@ -45,20 +46,20 @@ public class GenericServiceAPIResponseEntityDeserializer extends JsonDeserialize
 
         JsonNode rootNode = jp.getCodec().readTree(jp);
         if(rootNode.isObject()){
-            Iterator<Map.Entry<String,JsonNode>> fields = rootNode.getFields();
+            Iterator<Map.Entry<String,JsonNode>> fields = rootNode.fields();
             JsonNode objNode = null;
             while(fields.hasNext()){
                 Map.Entry<String,JsonNode> field = fields.next();
                 if (META_FIELD.equals(field.getKey()) && field.getValue() != null)
                     entity.setMeta(objectCodec.readValue(field.getValue().traverse(), Map.class));
                 else if(SUCCESS_FIELD.equals(field.getKey()) && field.getValue() != null){
-                    entity.setSuccess(field.getValue().getValueAsBoolean(false));
+                    entity.setSuccess(field.getValue().booleanValue());
                 }else if(EXCEPTION_FIELD.equals(field.getKey()) && field.getValue() != null){
-//                    entity.setException(field.getValue().getTextValue());
-                    entity.setException(new Exception(field.getValue().getTextValue()));
+                    entity.setException(new Exception(field.getValue().textValue()));
                 }else if(TYPE_FIELD.endsWith(field.getKey())  && field.getValue() != null){
+                    Preconditions.checkNotNull(field.getValue().textValue(),"Response type class is null");
                     try {
-                        entity.setType(Class.forName(field.getValue().getTextValue()));
+                        entity.setType(Class.forName(field.getValue().textValue()));
                     } catch (ClassNotFoundException e) {
                         throw new IOException(e);
                     }
@@ -82,4 +83,4 @@ public class GenericServiceAPIResponseEntityDeserializer extends JsonDeserialize
         }
         return entity;
     }
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/21187b55/eagle-core/eagle-query/eagle-entity-base/src/main/java/org/apache/eagle/log/entity/MetricMetadataEntity.java
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-query/eagle-entity-base/src/main/java/org/apache/eagle/log/entity/MetricMetadataEntity.java b/eagle-core/eagle-query/eagle-entity-base/src/main/java/org/apache/eagle/log/entity/MetricMetadataEntity.java
index f655ba0..9a58d35 100755
--- a/eagle-core/eagle-query/eagle-entity-base/src/main/java/org/apache/eagle/log/entity/MetricMetadataEntity.java
+++ b/eagle-core/eagle-query/eagle-entity-base/src/main/java/org/apache/eagle/log/entity/MetricMetadataEntity.java
@@ -62,56 +62,56 @@ public class MetricMetadataEntity extends TaggedLogAPIEntity {
 	}
 	public void setStoreType(String storeType) {
 		this.storeType = storeType;
-		_pcs.firePropertyChange("storeType", null, null);
+		pcs.firePropertyChange("storeType", null, null);
 	}
 	public String getDisplayName() {
 		return displayName;
 	}
 	public void setDisplayName(String displayName) {
 		this.displayName = displayName;
-		_pcs.firePropertyChange("displayName", null, null);
+		pcs.firePropertyChange("displayName", null, null);
 	}
 	public String getDefaultDownSamplingFunction() {
 		return defaultDownSamplingFunction;
 	}
 	public void setDefaultDownSamplingFunction(String defaultDownSamplingFunction) {
 		this.defaultDownSamplingFunction = defaultDownSamplingFunction;
-		_pcs.firePropertyChange("defaultDownSamplingFunction", null, null);
+		pcs.firePropertyChange("defaultDownSamplingFunction", null, null);
 	}
 	public String getDefaultAggregateFunction() {
 		return defaultAggregateFunction;
 	}
 	public void setDefaultAggregateFunction(String defaultAggregateFunction) {
 		this.defaultAggregateFunction = defaultAggregateFunction;
-		_pcs.firePropertyChange("defaultAggregateFunction", null, null);
+		pcs.firePropertyChange("defaultAggregateFunction", null, null);
 	}
 	public String getAggFunctions() {
 		return aggFunctions;
 	}
 	public void setAggFunctions(String aggFunctions) {
 		this.aggFunctions = aggFunctions;
-		_pcs.firePropertyChange("aggFunctions", null, null);
+		pcs.firePropertyChange("aggFunctions", null, null);
 	}
 	public String getDownSamplingFunctions() {
 		return downSamplingFunctions;
 	}
 	public void setDownSamplingFunctions(String downSamplingFunctions) {
 		this.downSamplingFunctions = downSamplingFunctions;
-		_pcs.firePropertyChange("downSamplingFunctions", null, null);
+		pcs.firePropertyChange("downSamplingFunctions", null, null);
 	}
 	public String getResolutions() {
 		return resolutions;
 	}
 	public void setResolutions(String resolutions) {
 		this.resolutions = resolutions;
-		_pcs.firePropertyChange("resolutions", null, null);
+		pcs.firePropertyChange("resolutions", null, null);
 	}
 	public String getDrillDownPaths() {
 		return drillDownPaths;
 	}
 	public void setDrillDownPaths(String drillDownPaths) {
 		this.drillDownPaths = drillDownPaths;
-		_pcs.firePropertyChange("drillDownPaths", null, null);
+		pcs.firePropertyChange("drillDownPaths", null, null);
 	}
 	
 }

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/21187b55/eagle-core/eagle-query/eagle-entity-base/src/main/java/org/apache/eagle/log/entity/test/TestEntity.java
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-query/eagle-entity-base/src/main/java/org/apache/eagle/log/entity/test/TestEntity.java b/eagle-core/eagle-query/eagle-entity-base/src/main/java/org/apache/eagle/log/entity/test/TestEntity.java
index 402e170..af635ec 100755
--- a/eagle-core/eagle-query/eagle-entity-base/src/main/java/org/apache/eagle/log/entity/test/TestEntity.java
+++ b/eagle-core/eagle-query/eagle-entity-base/src/main/java/org/apache/eagle/log/entity/test/TestEntity.java
@@ -56,7 +56,7 @@ public class TestEntity extends TaggedLogAPIEntity {
 
 	public void setNumClusters(Long numClusters) {
 		this.numClusters = numClusters;
-		_pcs.firePropertyChange("numClusters", null, null);
+		pcs.firePropertyChange("numClusters", null, null);
 	}
 
 	public int getNumHosts() {
@@ -65,7 +65,7 @@ public class TestEntity extends TaggedLogAPIEntity {
 
 	public void setNumHosts(int numHosts) {
 		this.numHosts = numHosts;
-		_pcs.firePropertyChange("numHosts", null, null);
+		pcs.firePropertyChange("numHosts", null, null);
 	}
 
 	public long getCount() {
@@ -74,7 +74,7 @@ public class TestEntity extends TaggedLogAPIEntity {
 
 	public void setCount(long count) {
 		this.count = count;
-		_pcs.firePropertyChange("count", null, null);
+		pcs.firePropertyChange("count", null, null);
 	}
 
 	public String getRemediationID() {
@@ -83,7 +83,7 @@ public class TestEntity extends TaggedLogAPIEntity {
 
 	public void setRemediationID(String remediationID) {
 		this.remediationID = remediationID;
-		_pcs.firePropertyChange("remediationID", null, null);
+		pcs.firePropertyChange("remediationID", null, null);
 	}
 
 	public String getRemediationStatus() {
@@ -92,7 +92,7 @@ public class TestEntity extends TaggedLogAPIEntity {
 
 	public void setRemediationStatus(String remediationStatus) {
 		this.remediationStatus = remediationStatus;
-		_pcs.firePropertyChange("remediationStatus", null, null);
+		pcs.firePropertyChange("remediationStatus", null, null);
 	}
 	
 	public String toString(){

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/21187b55/eagle-core/eagle-query/eagle-entity-base/src/main/java/org/apache/eagle/log/entity/test/TestLogAPIEntity.java
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-query/eagle-entity-base/src/main/java/org/apache/eagle/log/entity/test/TestLogAPIEntity.java b/eagle-core/eagle-query/eagle-entity-base/src/main/java/org/apache/eagle/log/entity/test/TestLogAPIEntity.java
index ea73f81..d126421 100755
--- a/eagle-core/eagle-query/eagle-entity-base/src/main/java/org/apache/eagle/log/entity/test/TestLogAPIEntity.java
+++ b/eagle-core/eagle-query/eagle-entity-base/src/main/java/org/apache/eagle/log/entity/test/TestLogAPIEntity.java
@@ -52,48 +52,48 @@ public class TestLogAPIEntity extends TaggedLogAPIEntity {
 	}
 	public void setField1(int field1) {
 		this.field1 = field1;
-		_pcs.firePropertyChange("field1", null, null);
+		pcs.firePropertyChange("field1", null, null);
 	}
 	public Integer getField2() {
 		return field2;
 	}
 	public void setField2(Integer field2) {
 		this.field2 = field2;
-		_pcs.firePropertyChange("field2", null, null);
+		pcs.firePropertyChange("field2", null, null);
 	}
 	public long getField3() {
 		return field3;
 	}
 	public void setField3(long field3) {
 		this.field3 = field3;
-		_pcs.firePropertyChange("field3", null, null);
+		pcs.firePropertyChange("field3", null, null);
 	}
 	public Long getField4() {
 		return field4;
 	}
 	public void setField4(Long field4) {
 		this.field4 = field4;
-		_pcs.firePropertyChange("field4", null, null);
+		pcs.firePropertyChange("field4", null, null);
 	}
 	public double getField5() {
 		return field5;
 	}
 	public void setField5(double field5) {
 		this.field5 = field5;
-		_pcs.firePropertyChange("field5", null, null);
+		pcs.firePropertyChange("field5", null, null);
 	}
 	public Double getField6() {
 		return field6;
 	}
 	public void setField6(Double field6) {
 		this.field6 = field6;
-		_pcs.firePropertyChange("field6", null, null);
+		pcs.firePropertyChange("field6", null, null);
 	}
 	public String getField7() {
 		return field7;
 	}
 	public void setField7(String field7) {
 		this.field7 = field7;
-		_pcs.firePropertyChange("field7", null, null);
+		pcs.firePropertyChange("field7", null, null);
 	}
 }

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/21187b55/eagle-core/eagle-query/eagle-entity-base/src/main/java/org/apache/eagle/log/entity/test/TestTimeSeriesAPIEntity.java
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-query/eagle-entity-base/src/main/java/org/apache/eagle/log/entity/test/TestTimeSeriesAPIEntity.java b/eagle-core/eagle-query/eagle-entity-base/src/main/java/org/apache/eagle/log/entity/test/TestTimeSeriesAPIEntity.java
index 6d5f560..eaa9fd2 100755
--- a/eagle-core/eagle-query/eagle-entity-base/src/main/java/org/apache/eagle/log/entity/test/TestTimeSeriesAPIEntity.java
+++ b/eagle-core/eagle-query/eagle-entity-base/src/main/java/org/apache/eagle/log/entity/test/TestTimeSeriesAPIEntity.java
@@ -50,48 +50,48 @@ public class TestTimeSeriesAPIEntity extends TaggedLogAPIEntity {
 	}
 	public void setField1(int field1) {
 		this.field1 = field1;
-		_pcs.firePropertyChange("field1", null, null);
+		pcs.firePropertyChange("field1", null, null);
 	}
 	public Integer getField2() {
 		return field2;
 	}
 	public void setField2(Integer field2) {
 		this.field2 = field2;
-		_pcs.firePropertyChange("field2", null, null);
+		pcs.firePropertyChange("field2", null, null);
 	}
 	public long getField3() {
 		return field3;
 	}
 	public void setField3(long field3) {
 		this.field3 = field3;
-		_pcs.firePropertyChange("field3", null, null);
+		pcs.firePropertyChange("field3", null, null);
 	}
 	public Long getField4() {
 		return field4;
 	}
 	public void setField4(Long field4) {
 		this.field4 = field4;
-		_pcs.firePropertyChange("field4", null, null);
+		pcs.firePropertyChange("field4", null, null);
 	}
 	public double getField5() {
 		return field5;
 	}
 	public void setField5(double field5) {
 		this.field5 = field5;
-		_pcs.firePropertyChange("field5", null, null);
+		pcs.firePropertyChange("field5", null, null);
 	}
 	public Double getField6() {
 		return field6;
 	}
 	public void setField6(Double field6) {
 		this.field6 = field6;
-		_pcs.firePropertyChange("field6", null, null);
+		pcs.firePropertyChange("field6", null, null);
 	}
 	public String getField7() {
 		return field7;
 	}
 	public void setField7(String field7) {
 		this.field7 = field7;
-		_pcs.firePropertyChange("field7", null, null);
+		pcs.firePropertyChange("field7", null, null);
 	}
 }

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/21187b55/eagle-core/eagle-query/eagle-entity-base/src/test/java/org/apache/eagle/log/TestGenericServiceAPIResponseEntity.java
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-query/eagle-entity-base/src/test/java/org/apache/eagle/log/TestGenericServiceAPIResponseEntity.java b/eagle-core/eagle-query/eagle-entity-base/src/test/java/org/apache/eagle/log/TestGenericServiceAPIResponseEntity.java
index 6ef9365..092ef75 100644
--- a/eagle-core/eagle-query/eagle-entity-base/src/test/java/org/apache/eagle/log/TestGenericServiceAPIResponseEntity.java
+++ b/eagle-core/eagle-query/eagle-entity-base/src/test/java/org/apache/eagle/log/TestGenericServiceAPIResponseEntity.java
@@ -16,10 +16,10 @@
  */
 package org.apache.eagle.log;
 
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.annotation.JsonSerialize;
 import org.apache.eagle.log.entity.GenericServiceAPIResponseEntity;
 import org.junit.Assert;
-import org.codehaus.jackson.map.ObjectMapper;
-import org.codehaus.jackson.map.annotate.JsonSerialize;
 import org.junit.Before;
 import org.junit.Test;
 import org.slf4j.Logger;

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/21187b55/eagle-core/eagle-query/eagle-entity-base/src/test/java/org/apache/eagle/log/entity/base/taggedlog/TestTaggedLogAPIEntity.java
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-query/eagle-entity-base/src/test/java/org/apache/eagle/log/entity/base/taggedlog/TestTaggedLogAPIEntity.java b/eagle-core/eagle-query/eagle-entity-base/src/test/java/org/apache/eagle/log/entity/base/taggedlog/TestTaggedLogAPIEntity.java
index 3b14bff..d6db6a2 100755
--- a/eagle-core/eagle-query/eagle-entity-base/src/test/java/org/apache/eagle/log/entity/base/taggedlog/TestTaggedLogAPIEntity.java
+++ b/eagle-core/eagle-query/eagle-entity-base/src/test/java/org/apache/eagle/log/entity/base/taggedlog/TestTaggedLogAPIEntity.java
@@ -17,18 +17,32 @@
 package org.apache.eagle.log.entity.base.taggedlog;
 
 import java.io.IOException;
+import java.util.Arrays;
 import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
 
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.SerializationFeature;
+import org.apache.eagle.log.base.taggedlog.EntityJsonModule;
 import org.apache.eagle.log.base.taggedlog.TaggedLogAPIEntity;
 import org.apache.eagle.log.entity.meta.Column;
 import org.junit.Assert;
 
-import org.codehaus.jackson.map.ObjectMapper;
 import org.codehaus.jackson.map.annotate.JsonSerialize;
+import org.junit.BeforeClass;
 import org.junit.Test;
 
 public class TestTaggedLogAPIEntity {
-    ObjectMapper objectMapper = new ObjectMapper();
+    private static ObjectMapper objectMapper;
+
+    @BeforeClass
+    public static void setUp(){
+        objectMapper = new ObjectMapper();
+        objectMapper.setFilters(TaggedLogAPIEntity.getFilterProvider());
+        objectMapper.registerModule(new EntityJsonModule());
+    }
 
     @JsonSerialize(include=JsonSerialize.Inclusion.NON_NULL)
     private class MockSubTaggedLogAPIEntity extends TaggedLogAPIEntity {
@@ -38,7 +52,7 @@ public class TestTaggedLogAPIEntity {
 
         public void setField1(double value) {
             this.field1 = value;
-            _pcs.firePropertyChange("field1", null, null);
+            pcs.firePropertyChange("field1", null, null);
         }
 
         @Column("a")
@@ -50,14 +64,14 @@ public class TestTaggedLogAPIEntity {
 
         public void setField2(String field2) {
             this.field2 = field2;
-            _pcs.firePropertyChange("field2", null, null);
+            pcs.firePropertyChange("field2", null, null);
         }
 
         @Column("b")
         private String field2;
     }
 
-    @SuppressWarnings("serial")
+    @SuppressWarnings("unchecked")
 	@Test
     public void testJsonSerializeFilter() throws IOException {
         MockSubTaggedLogAPIEntity mock = new MockSubTaggedLogAPIEntity();
@@ -68,7 +82,7 @@ public class TestTaggedLogAPIEntity {
         mock.setEncodedRowkey("test_encoded_row_key");
         mock.setPrefix("mock");
         mock.setField2("ok");
-        String json = objectMapper.filteredWriter(TaggedLogAPIEntity.getFilterProvider()).writeValueAsString(mock);
+        String json = objectMapper.writeValueAsString(mock);
         System.out.println(json);
         Assert.assertTrue(json.contains("field2"));
         Assert.assertTrue(!json.contains("field1"));
@@ -83,9 +97,21 @@ public class TestTaggedLogAPIEntity {
         mock.setExp(new HashMap<String, Object>() {{
             put("extra_field", 3.14);
         }});
-        json = objectMapper.filteredWriter(TaggedLogAPIEntity.getFilterProvider()).writeValueAsString(mock);
+        json = objectMapper.writeValueAsString(mock);
         System.out.println(json);
         Assert.assertTrue(json.contains("field2"));
         Assert.assertTrue(json.contains("field1"));
     }
+
+    @Test
+    public void testJsonSerializeMap() throws JsonProcessingException {
+        Map<List<String>,List<Object>> entries = new HashMap<List<String>,List<Object>>(){
+            {
+                put(Arrays.asList("a","b"),Arrays.asList(1,2,3));
+            }
+        };
+        String json = objectMapper.writeValueAsString(entries.entrySet());
+        Assert.assertNotNull(json);
+        System.out.print(json);
+    }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/21187b55/eagle-core/eagle-query/eagle-query-base/src/main/java/org/apache/eagle/query/aggregate/raw/GroupbyKey.java
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-query/eagle-query-base/src/main/java/org/apache/eagle/query/aggregate/raw/GroupbyKey.java b/eagle-core/eagle-query/eagle-query-base/src/main/java/org/apache/eagle/query/aggregate/raw/GroupbyKey.java
index 3a54d70..c8ed260 100755
--- a/eagle-core/eagle-query/eagle-query-base/src/main/java/org/apache/eagle/query/aggregate/raw/GroupbyKey.java
+++ b/eagle-core/eagle-query/eagle-query-base/src/main/java/org/apache/eagle/query/aggregate/raw/GroupbyKey.java
@@ -16,12 +16,15 @@
  */
 package org.apache.eagle.query.aggregate.raw;
 
+import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.io.ByteWritable;
 import org.apache.hadoop.io.BytesWritable;
 import org.apache.hadoop.io.Writable;
 
 import java.io.DataInput;
 import java.io.DataOutput;
 import java.io.IOException;
+import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.List;
 import java.util.ListIterator;
@@ -90,8 +93,6 @@ public class GroupbyKey implements Writable {
 			return false;
 		}
 		GroupbyKey that = (GroupbyKey)obj;
-//		ListIterator<byte[]> e1 = this.value.listIterator();
-//		ListIterator<byte[]> e2 = that.value.listIterator();
 		ListIterator<BytesWritable> e1 = this.value.listIterator();
 		ListIterator<BytesWritable> e2 = that.value.listIterator();
 		while(e1.hasNext() && e2.hasNext()){
@@ -102,6 +103,16 @@ public class GroupbyKey implements Writable {
 	}
 
 	@Override
+	public String toString() {
+		List<String> items = new ArrayList<>(this.value.size());
+		ListIterator<BytesWritable> iterator = this.value.listIterator();
+		while(iterator.hasNext()){
+			items.add(iterator.next().toString());
+		}
+		return String.format("%s(%s)",this.getClass().getSimpleName(),StringUtils.join(items,","));
+	}
+
+	@Override
 	public int hashCode(){
 		ListIterator<BytesWritable> e1 = this.value.listIterator();
 		int hash = 0xFFFFFFFF;

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/21187b55/eagle-core/eagle-query/eagle-query-base/src/test/java/org/apache/eagle/query/aggregate/test/TestBucketQuery.java
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-query/eagle-query-base/src/test/java/org/apache/eagle/query/aggregate/test/TestBucketQuery.java b/eagle-core/eagle-query/eagle-query-base/src/test/java/org/apache/eagle/query/aggregate/test/TestBucketQuery.java
index e2e83ec..e44d73b 100755
--- a/eagle-core/eagle-query/eagle-query-base/src/test/java/org/apache/eagle/query/aggregate/test/TestBucketQuery.java
+++ b/eagle-core/eagle-query/eagle-query-base/src/test/java/org/apache/eagle/query/aggregate/test/TestBucketQuery.java
@@ -16,11 +16,11 @@
  */
 package org.apache.eagle.query.aggregate.test;
 
+import com.fasterxml.jackson.core.JsonFactory;
+import com.fasterxml.jackson.databind.ObjectMapper;
 import org.apache.eagle.log.base.taggedlog.TaggedLogAPIEntity;
 import org.apache.eagle.query.aggregate.BucketQuery;
 import org.junit.Assert;
-import org.codehaus.jackson.JsonFactory;
-import org.codehaus.jackson.map.ObjectMapper;
 import org.junit.Test;
 
 import java.util.ArrayList;

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/21187b55/eagle-core/eagle-query/eagle-service-base/pom.xml
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-query/eagle-service-base/pom.xml b/eagle-core/eagle-query/eagle-service-base/pom.xml
index 92a705a..a3fc2e7 100755
--- a/eagle-core/eagle-query/eagle-service-base/pom.xml
+++ b/eagle-core/eagle-query/eagle-service-base/pom.xml
@@ -69,9 +69,9 @@
             <classifier>tests</classifier>
             <scope>test</scope>
         </dependency>
-        <dependency>
-            <groupId>org.codehaus.jackson</groupId>
-            <artifactId>jackson-mapper-asl</artifactId>
-        </dependency>
+        <!--<dependency>-->
+            <!--<groupId>org.codehaus.jackson</groupId>-->
+            <!--<artifactId>jackson-mapper-asl</artifactId>-->
+        <!--</dependency>-->
     </dependencies>
 </project>

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/21187b55/eagle-core/eagle-query/eagle-service-base/src/main/java/org/apache/eagle/service/generic/GenericEntityServiceResource.java
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-query/eagle-service-base/src/main/java/org/apache/eagle/service/generic/GenericEntityServiceResource.java b/eagle-core/eagle-query/eagle-service-base/src/main/java/org/apache/eagle/service/generic/GenericEntityServiceResource.java
index b7f27f5..a0d8143 100644
--- a/eagle-core/eagle-query/eagle-service-base/src/main/java/org/apache/eagle/service/generic/GenericEntityServiceResource.java
+++ b/eagle-core/eagle-query/eagle-service-base/src/main/java/org/apache/eagle/service/generic/GenericEntityServiceResource.java
@@ -38,6 +38,7 @@ import org.slf4j.LoggerFactory;
 import javax.ws.rs.*;
 import javax.ws.rs.core.GenericEntity;
 import javax.ws.rs.core.MediaType;
+import javax.ws.rs.core.Response;
 import java.io.IOException;
 import java.io.InputStream;
 import java.util.HashMap;
@@ -456,6 +457,10 @@ public class GenericEntityServiceResource {
         } catch (Exception e) {
             response.setException(e);
             LOG.error(e.getMessage(),e);
+            throw new WebApplicationException(e,
+                    Response.status(Response.Status.BAD_REQUEST)
+                    .type(MediaType.APPLICATION_JSON_TYPE)
+                    .entity(response).build());
         }finally {
             stopWatch.stop();
         }

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/21187b55/eagle-core/eagle-query/eagle-service-base/src/main/java/org/apache/eagle/service/generic/GenericObjectMapperProvider.java
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-query/eagle-service-base/src/main/java/org/apache/eagle/service/generic/GenericObjectMapperProvider.java b/eagle-core/eagle-query/eagle-service-base/src/main/java/org/apache/eagle/service/generic/GenericObjectMapperProvider.java
index 35dd4b7..35b04d1 100755
--- a/eagle-core/eagle-query/eagle-service-base/src/main/java/org/apache/eagle/service/generic/GenericObjectMapperProvider.java
+++ b/eagle-core/eagle-query/eagle-service-base/src/main/java/org/apache/eagle/service/generic/GenericObjectMapperProvider.java
@@ -16,10 +16,10 @@
  */
 package org.apache.eagle.service.generic;
 
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.ser.FilterProvider;
 import com.google.inject.Singleton;
 import org.apache.eagle.log.base.taggedlog.TaggedLogAPIEntity;
-import org.codehaus.jackson.map.ObjectMapper;
-import org.codehaus.jackson.map.ser.FilterProvider;
 
 import javax.ws.rs.Produces;
 import javax.ws.rs.core.MediaType;

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/21187b55/eagle-core/eagle-query/eagle-service-base/src/main/java/org/apache/eagle/service/selfcheck/ServiceResource.java
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-query/eagle-service-base/src/main/java/org/apache/eagle/service/selfcheck/ServiceResource.java b/eagle-core/eagle-query/eagle-service-base/src/main/java/org/apache/eagle/service/selfcheck/ServiceResource.java
index f108bd5..2db13f5 100644
--- a/eagle-core/eagle-query/eagle-service-base/src/main/java/org/apache/eagle/service/selfcheck/ServiceResource.java
+++ b/eagle-core/eagle-query/eagle-service-base/src/main/java/org/apache/eagle/service/selfcheck/ServiceResource.java
@@ -33,32 +33,33 @@ import com.sun.jersey.api.json.JSONWithPadding;
 
 @Path("services")
 public class ServiceResource {
-	private static final Logger LOG = LoggerFactory.getLogger(ServiceResource.class);
-	
-	@GET
-	@Produces({"application/json", "application/xml"})
-	public Map<String, EntityDefinition> getServiceDefinitions(){
-		Map<String, EntityDefinition> result = null;
-		try{
-			 result = EntityDefinitionManager.entities();
-		}catch(Exception ex){
-			LOG.error("Error in getting entity definitions", ex);
-		}
-		
-		return result;
-	}
-	
-	@GET
-	@Path("/jsonp")
-	@Produces({"application/x-javascript", "application/json", "application/xml"})
-	public JSONWithPadding getServiceDefinitionsWithJsonp(@QueryParam("callback") String callback){
-		Map<String, EntityDefinition> result = null;
-		try{
-			 result = EntityDefinitionManager.entities();
-		}catch(Exception ex){
-			LOG.error("Error in getting entity definitions", ex);
-		}
-		
-		return new JSONWithPadding(new GenericEntity<Map<String, EntityDefinition>>(result){}, callback);
-	}
+    private static final Logger LOG = LoggerFactory.getLogger(ServiceResource.class);
+
+    @GET
+    @Produces({"application/json", "application/xml"})
+    public Map<String, EntityDefinition> getServiceDefinitions() {
+        Map<String, EntityDefinition> result = null;
+        try {
+            result = EntityDefinitionManager.entities();
+        } catch (Exception ex) {
+            LOG.error("Error in getting entity definitions", ex);
+        }
+
+        return result;
+    }
+
+    @GET
+    @Path("/jsonp")
+    @Produces({"application/x-javascript", "application/json", "application/xml"})
+    public JSONWithPadding getServiceDefinitionsWithJsonp(@QueryParam("callback") String callback) {
+        Map<String, EntityDefinition> result = null;
+        try {
+            result = EntityDefinitionManager.entities();
+        } catch (Exception ex) {
+            LOG.error("Error in getting entity definitions", ex);
+        }
+
+        return new JSONWithPadding(new GenericEntity<Map<String, EntityDefinition>>(result) {
+        }, callback);
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/21187b55/eagle-core/eagle-query/eagle-storage-base/pom.xml
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-query/eagle-storage-base/pom.xml b/eagle-core/eagle-query/eagle-storage-base/pom.xml
index 536e497..de5d59d 100644
--- a/eagle-core/eagle-query/eagle-storage-base/pom.xml
+++ b/eagle-core/eagle-query/eagle-storage-base/pom.xml
@@ -33,4 +33,4 @@
             <version>${project.version}</version>
         </dependency>
     </dependencies>
-</project>
+</project>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/21187b55/eagle-core/eagle-query/eagle-storage-base/src/test/resources/application.conf
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-query/eagle-storage-base/src/test/resources/application.conf b/eagle-core/eagle-query/eagle-storage-base/src/test/resources/application.conf
index dbaac1d..77093bc 100644
--- a/eagle-core/eagle-query/eagle-storage-base/src/test/resources/application.conf
+++ b/eagle-core/eagle-query/eagle-storage-base/src/test/resources/application.conf
@@ -13,8 +13,7 @@
 # See the License for the specific language governing permissions and
 # limitations under the License.
 
-eagle{
-	service{
+storage {
 		env="dev"
 		host="localhost"
 		port=8080
@@ -24,5 +23,4 @@ eagle{
 		hbase-zookeeper-quorum="localhost"
 		hbase-zookeeper-property-clientPort=2181
 		zookeeper-znode-parent="/hbase-unsecure"
-	}
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/21187b55/eagle-core/eagle-query/eagle-storage-hbase/pom.xml
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-query/eagle-storage-hbase/pom.xml b/eagle-core/eagle-query/eagle-storage-hbase/pom.xml
index 774684c..406494e 100644
--- a/eagle-core/eagle-query/eagle-storage-hbase/pom.xml
+++ b/eagle-core/eagle-query/eagle-storage-hbase/pom.xml
@@ -22,7 +22,7 @@
         <artifactId>eagle-query-parent</artifactId>
         <groupId>org.apache.eagle</groupId>
         <version>0.5.0-incubating-SNAPSHOT</version>
-        <relativePath>../pom.xml</relativePath>
+        <relativePath>..</relativePath>
     </parent>
     <modelVersion>4.0.0</modelVersion>
 

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/21187b55/eagle-core/eagle-query/eagle-storage-hbase/src/assembly/hbase-coprocessor.xml
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-query/eagle-storage-hbase/src/assembly/hbase-coprocessor.xml b/eagle-core/eagle-query/eagle-storage-hbase/src/assembly/hbase-coprocessor.xml
index fa80c64..e27fcd4 100644
--- a/eagle-core/eagle-query/eagle-storage-hbase/src/assembly/hbase-coprocessor.xml
+++ b/eagle-core/eagle-query/eagle-storage-hbase/src/assembly/hbase-coprocessor.xml
@@ -44,6 +44,7 @@
                 <exclude>org.slf4j:slf4j-api</exclude>
                 <exclude>org.slf4j:log4j-over-slf4j</exclude>
                 <exclude>org.slf4j:slf4j-log4j12</exclude>
+                <exclude>com.google.protobuf:protobuf-java</exclude>
             </excludes>
         </dependencySet>
     </dependencySets>

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/21187b55/eagle-core/eagle-query/eagle-storage-hbase/src/main/java/org/apache/eagle/storage/hbase/HBaseEntitySchemaManager.java
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-query/eagle-storage-hbase/src/main/java/org/apache/eagle/storage/hbase/HBaseEntitySchemaManager.java b/eagle-core/eagle-query/eagle-storage-hbase/src/main/java/org/apache/eagle/storage/hbase/HBaseEntitySchemaManager.java
index 03c0ec2..d7483f5 100644
--- a/eagle-core/eagle-query/eagle-storage-hbase/src/main/java/org/apache/eagle/storage/hbase/HBaseEntitySchemaManager.java
+++ b/eagle-core/eagle-query/eagle-storage-hbase/src/main/java/org/apache/eagle/storage/hbase/HBaseEntitySchemaManager.java
@@ -40,9 +40,10 @@ public class HBaseEntitySchemaManager {
     private static HBaseEntitySchemaManager instance;
     private volatile HBaseAdmin admin;
 
-    private final int DEFAULT_MAX_VERSIONS = 1;
+    private static final int DEFAULT_MAX_VERSIONS = 1;
 
-    private HBaseEntitySchemaManager() {}
+    private HBaseEntitySchemaManager() {
+    }
 
     public static HBaseEntitySchemaManager getInstance() {
         if (instance == null) {

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/21187b55/eagle-core/eagle-query/eagle-storage-hbase/src/main/java/org/apache/eagle/storage/hbase/HBaseStorage.java
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-query/eagle-storage-hbase/src/main/java/org/apache/eagle/storage/hbase/HBaseStorage.java b/eagle-core/eagle-query/eagle-storage-hbase/src/main/java/org/apache/eagle/storage/hbase/HBaseStorage.java
index 1810401..74c90bc 100644
--- a/eagle-core/eagle-query/eagle-storage-hbase/src/main/java/org/apache/eagle/storage/hbase/HBaseStorage.java
+++ b/eagle-core/eagle-query/eagle-storage-hbase/src/main/java/org/apache/eagle/storage/hbase/HBaseStorage.java
@@ -16,6 +16,11 @@
  */
 package org.apache.eagle.storage.hbase;
 
+import static org.apache.eagle.audit.common.AuditConstants.AUDIT_EVENT_CREATE;
+import static org.apache.eagle.audit.common.AuditConstants.AUDIT_EVENT_DELETE;
+import static org.apache.eagle.audit.common.AuditConstants.AUDIT_EVENT_UPDATE;
+
+import org.apache.eagle.common.EagleBase64Wrapper;
 import org.apache.eagle.log.base.taggedlog.TaggedLogAPIEntity;
 import org.apache.eagle.log.entity.GenericEntityWriter;
 import org.apache.eagle.log.entity.HBaseInternalLogHelper;
@@ -29,7 +34,7 @@ import org.apache.eagle.storage.hbase.query.GenericQueryBuilder;
 import org.apache.eagle.storage.operation.CompiledQuery;
 import org.apache.eagle.storage.result.ModifyResult;
 import org.apache.eagle.storage.result.QueryResult;
-import org.apache.eagle.common.EagleBase64Wrapper;
+
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -37,19 +42,13 @@ import java.io.IOException;
 import java.util.ArrayList;
 import java.util.LinkedList;
 import java.util.List;
+import java.util.Map;
 
-import static org.apache.eagle.audit.common.AuditConstants.AUDIT_EVENT_CREATE;
-import static org.apache.eagle.audit.common.AuditConstants.AUDIT_EVENT_UPDATE;
-import static org.apache.eagle.audit.common.AuditConstants.AUDIT_EVENT_DELETE;
-
-/**
- * @since 3/18/15
- */
 public class HBaseStorage extends DataStorageBase {
 
-    private final static Logger LOG = LoggerFactory.getLogger(HBaseStorage.class);
+    private static final Logger LOG = LoggerFactory.getLogger(HBaseStorage.class);
     private HBaseStorageAudit audit = new HBaseStorageAudit();
-    
+
     @Override
     public void init() throws IOException {
         HBaseEntitySchemaManager.getInstance().init();
@@ -58,9 +57,9 @@ public class HBaseStorage extends DataStorageBase {
 
     @Override
     public <E extends TaggedLogAPIEntity> ModifyResult<String> update(List<E> entities, EntityDefinition entityDefinition) throws IOException {
-    	ModifyResult<String> result = create(entities, entityDefinition);
-    	audit.auditOperation(AUDIT_EVENT_UPDATE, entities, null, entityDefinition); // added for jira: EAGLE-47
-    	return result;
+        ModifyResult<String> result = create(entities, entityDefinition);
+        audit.auditOperation(AUDIT_EVENT_UPDATE, entities, null, entityDefinition); // added for jira: EAGLE-47
+        return result;
     }
 
     @Override
@@ -71,118 +70,85 @@ public class HBaseStorage extends DataStorageBase {
             result.setIdentifiers(entityWriter.write(entities));
             result.setSuccess(true);
         } catch (Exception e) {
-            LOG.error(e.getMessage(),e);
+            LOG.error(e.getMessage(), e);
             throw new IOException(e);
         }
 
         audit.auditOperation(AUDIT_EVENT_CREATE, entities, null, entityDefinition); // added for jira: EAGLE-47
         return result;
     }
-    
-    /**
-     * @param entities
-     * @param entityDefinition
-     * @param <E>
-     *
-     * @return ModifyResult
-     *
-     * @throws IOException
-     */
+
+
     @Override
-    public <E extends TaggedLogAPIEntity> ModifyResult<String> delete(List<E> entities, EntityDefinition entityDefinition) throws IOException {
+    public ModifyResult<String> deleteByID(List<String> ids, EntityDefinition entityDefinition) throws IOException {
         ModifyResult<String> result = new ModifyResult<String>();
-        try{
+        try {
             GenericDeleter deleter = new GenericDeleter(entityDefinition.getTable(), entityDefinition.getColumnFamily());
-            result.setIdentifiers(deleter.delete(entities));
-        }catch(Exception ex){
-            LOG.error(ex.getMessage(),ex);
+            deleter.deleteByEncodedRowkeys(ids);
+            result.setIdentifiers(ids);
+        } catch (Exception ex) {
+            LOG.error(ex.getMessage(), ex);
             result.setSuccess(false);
             throw new IOException(ex);
         }
-        
-        audit.auditOperation(AUDIT_EVENT_DELETE, entities, null, entityDefinition); // added for jira: EAGLE-47
+
+        audit.auditOperation(AUDIT_EVENT_DELETE, null, ids, entityDefinition); // added for jira: EAGLE-47
         result.setSuccess(true);
         return result;
     }
 
-    /**
-     * TODO:
-     *
-     * @param ids
-     * @param entityDefinition
-     * @return
-     * @throws IOException
-     */
     @Override
-    public ModifyResult<String> deleteByID(List<String> ids, EntityDefinition entityDefinition) throws IOException {
+    public <E extends TaggedLogAPIEntity> ModifyResult<String> delete(List<E> entities, EntityDefinition entityDefinition) throws IOException {
         ModifyResult<String> result = new ModifyResult<String>();
-        try{
+        try {
             GenericDeleter deleter = new GenericDeleter(entityDefinition.getTable(), entityDefinition.getColumnFamily());
-            deleter.deleteByEncodedRowkeys(ids);
-            result.setIdentifiers(ids);
-        }catch(Exception ex){
-            LOG.error(ex.getMessage(),ex);
+            result.setIdentifiers(deleter.delete(entities));
+        } catch (Exception ex) {
+            LOG.error(ex.getMessage(), ex);
             result.setSuccess(false);
             throw new IOException(ex);
         }
-        
-        audit.auditOperation(AUDIT_EVENT_DELETE, null, ids, entityDefinition); // added for jira: EAGLE-47
+
+        audit.auditOperation(AUDIT_EVENT_DELETE, entities, null, entityDefinition); // added for jira: EAGLE-47
         result.setSuccess(true);
         return result;
     }
 
-    /**
-     * TODO:
-     *
-     * @param query
-     * @param entityDefinition
-     * @return
-     * @throws IOException
-     */
     @Override
     public ModifyResult<String> delete(CompiledQuery query, EntityDefinition entityDefinition) throws IOException {
-        if(query.isHasAgg()){
+        if (query.isHasAgg()) {
             throw new IOException("delete by aggregation query is not supported");
         }
         ModifyResult<String> result;
 
         try {
-            LOG.info("Querying for deleting: "+query);
+            LOG.info("Querying for deleting: " + query);
             GenericQuery reader = GenericQueryBuilder
                     .select(query.getSearchCondition().getOutputFields())
-                    .from(query.getServiceName(),query.getRawQuery().getMetricName()).where(query.getSearchCondition())
+                    .from(query.getServiceName(), query.getRawQuery().getMetricName()).where(query.getSearchCondition())
                     .groupBy(query.isHasAgg(), query.getGroupByFields(), query.getAggregateFunctionTypes(), query.getAggregateFields())
-                    .timeSeries(query.getRawQuery().isTimeSeries(),query.getRawQuery().getIntervalmin())
+                    .timeSeries(query.getRawQuery().isTimeSeries(), query.getRawQuery().getIntervalmin())
                     .treeAgg(query.getRawQuery().isTreeAgg())
-                    .orderBy(query.getSortOptions(),query.getSortFunctions(),query.getSortFields())
+                    .orderBy(query.getSortOptions(), query.getSortFunctions(), query.getSortFields())
                     .top(query.getRawQuery().getTop())
                     .parallel(query.getRawQuery().getParallel())
                     .build();
             List<? extends TaggedLogAPIEntity> entities = reader.result();
-            if(entities != null){
-                LOG.info("Deleting "+entities.size()+" entities");
-                result =  delete(entities,entityDefinition);
-            }else{
+            if (entities != null) {
+                LOG.info("Deleting " + entities.size() + " entities");
+                result = delete(entities, entityDefinition);
+            } else {
                 LOG.info("Deleting 0 entities");
                 result = new ModifyResult<String>();
                 result.setSuccess(true);
             }
         } catch (Exception e) {
-            LOG.error(e.getMessage(),e);
+            LOG.error(e.getMessage(), e);
             throw new IOException(e);
         }
         return result;
     }
 
-    /**
-     * TODO:
-     *
-     * @param query
-     * @param entityDefinition
-     * @param <E>
-     * @return
-     * @throws IOException
-     */
     @Override
     @SuppressWarnings("unchecked")
     public <E extends Object> QueryResult<E> query(CompiledQuery query, EntityDefinition entityDefinition) throws IOException {
@@ -190,52 +156,53 @@ public class HBaseStorage extends DataStorageBase {
         try {
             GenericQuery reader = GenericQueryBuilder
                     .select(query.getSearchCondition().getOutputFields())
-                    .from(query.getServiceName(),query.getRawQuery().getMetricName()).where(query.getSearchCondition())
+                    .from(query.getServiceName(), query.getRawQuery().getMetricName()).where(query.getSearchCondition())
                     .groupBy(query.isHasAgg(), query.getGroupByFields(), query.getAggregateFunctionTypes(), query.getAggregateFields())
-                    .timeSeries(query.getRawQuery().isTimeSeries(),query.getRawQuery().getIntervalmin())
+                    .timeSeries(query.getRawQuery().isTimeSeries(), query.getRawQuery().getIntervalmin())
                     .treeAgg(query.getRawQuery().isTreeAgg())
-                    .orderBy(query.getSortOptions(),query.getSortFunctions(),query.getSortFields())
+                    .orderBy(query.getSortOptions(), query.getSortFunctions(), query.getSortFields())
                     .top(query.getRawQuery().getTop())
                     .parallel(query.getRawQuery().getParallel())
                     .build();
-            List<E> entities =  reader.result();
+            List<E> entities = reader.result();
             result.setData(entities);
             result.setFirstTimestamp(reader.getFirstTimeStamp());
             result.setLastTimestamp(reader.getLastTimestamp());
             result.setSize(entities.size());
-            if(!query.isHasAgg()) result.setEntityType((Class<E>) entityDefinition.getEntityClass());
+            if (!query.isHasAgg()) {
+                result.setEntityType((Class<E>) entityDefinition.getEntityClass());
+            } else {
+                result.setEntityType((Class<E>) Map.class);
+            }
             result.setSuccess(true);
         } catch (Exception e) {
-            LOG.error(e.getMessage(),e);
+            LOG.error(e.getMessage(), e);
             throw new IOException(e);
         }
         return result;
     }
 
     /**
-     * Query by HBase rowkey
-     *
-     * @param ids hbase rowkey list
-     * @param entityDefinition entity definition
-     * @param <E> entity type
-     * @return QueryResult with entity type <E>
-     *
-     * @throws IOException
+     * Query by HBase rowkey.
      */
     @Override
     public <E> QueryResult<E> queryById(List<String> ids, EntityDefinition entityDefinition) throws IOException {
         List<byte[]> rowkeys = new ArrayList<>(ids.size());
         QueryResult<E> result = new QueryResult<E>();
-        for(String id:ids) rowkeys.add(EagleBase64Wrapper.decode(id));
+        for (String id : ids) {
+            rowkeys.add(EagleBase64Wrapper.decode(id));
+        }
         RowKeyLogReader reader = null;
         try {
-            reader = new RowKeyLogReader(entityDefinition, rowkeys,null);
+            reader = new RowKeyLogReader(entityDefinition, rowkeys, null);
             reader.open();
             List<TaggedLogAPIEntity> entities = new LinkedList<>();
 
-            while(true) {
+            while (true) {
                 InternalLog log = reader.read();
-                if(log == null) break;
+                if (log == null) {
+                    break;
+                }
                 TaggedLogAPIEntity entity = HBaseInternalLogHelper.buildEntity(log, entityDefinition);
                 entities.add(entity);
             }
@@ -247,8 +214,10 @@ public class HBaseStorage extends DataStorageBase {
         } catch (Exception e) {
             LOG.error(e.getMessage(), e);
             throw new IOException(e);
-        } finally{
-            if(reader != null) reader.close();
+        } finally {
+            if (reader != null) {
+                reader.close();
+            }
         }
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/21187b55/eagle-core/eagle-query/eagle-storage-hbase/src/main/java/org/apache/eagle/storage/hbase/HBaseStorageAudit.java
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-query/eagle-storage-hbase/src/main/java/org/apache/eagle/storage/hbase/HBaseStorageAudit.java b/eagle-core/eagle-query/eagle-storage-hbase/src/main/java/org/apache/eagle/storage/hbase/HBaseStorageAudit.java
index f6a6180..74a9dc7 100644
--- a/eagle-core/eagle-query/eagle-storage-hbase/src/main/java/org/apache/eagle/storage/hbase/HBaseStorageAudit.java
+++ b/eagle-core/eagle-query/eagle-storage-hbase/src/main/java/org/apache/eagle/storage/hbase/HBaseStorageAudit.java
@@ -17,11 +17,7 @@
 
 package org.apache.eagle.storage.hbase;
 
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
+import static org.apache.eagle.audit.common.AuditConstants.*;
 
 import org.apache.eagle.audit.common.AuditEvent;
 import org.apache.eagle.audit.entity.GenericAuditEntity;
@@ -33,132 +29,138 @@ import org.apache.eagle.log.entity.meta.EntityDefinition;
 import org.apache.eagle.storage.DataStorageManager;
 import org.apache.eagle.storage.exception.IllegalDataStorageTypeException;
 import org.apache.eagle.storage.operation.CreateStatement;
+
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.springframework.security.core.Authentication;
 import org.springframework.security.core.context.SecurityContextHolder;
 
-import static org.apache.eagle.audit.common.AuditConstants.AUDIT_SERVICE_ENDPOINT;
-import static org.apache.eagle.audit.common.AuditConstants.AUDIT_COLUMN_SERVICE_NAME;
-import static org.apache.eagle.audit.common.AuditConstants.AUDIT_COLUMN_USER_ID;
-import static org.apache.eagle.audit.common.AuditConstants.AUDIT_COLUMN_OPERATION;
-import static org.apache.eagle.audit.common.AuditConstants.AUDIT_COLUMN_TIMESTAMP;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
 
 /**
- * Implementation of AuditListener class. 
+ * Implementation of AuditListener class.
  * Used in HBaseStorage class for auditing HBase operations performed.
  */
 public class HBaseStorageAudit implements AuditListener {
-	
-	private final static Logger LOG = LoggerFactory.getLogger(HBaseStorageAudit.class);
-	private AuditSupport auditSupport = new AuditSupport(this);
-	
-	public HBaseStorageAudit() {
-		auditSupport.addAuditListener(this);
-	}
-	
-	@Override
-	public void auditEvent(AuditEvent event) {
-			LOG.info("firing audit event: " + event.toString());
-			persistAuditEntity(event.getAuditEntities());
-	}
-	
-	/**
-	 * Method to be invoked for firing audit event.
-	 * @param operation: HBase operation. Values like CREATE/UPDATE/DELETE.
-	 * @param entities: List of entities used in HBase operation.
-	 * @param encodedRowKeys: List of encodededRowKeys returned from successful HBase operation. To be passed only from deletebyID method. 
-	 * @param entityDefinition: EntityDefinition object used in the HBaseOperation.
-	 */
+
+    private static final Logger LOG = LoggerFactory.getLogger(HBaseStorageAudit.class);
+    private AuditSupport auditSupport = new AuditSupport(this);
+
+    public HBaseStorageAudit() {
+        auditSupport.addAuditListener(this);
+    }
+
+    @Override
+    public void auditEvent(AuditEvent event) {
+        LOG.info("firing audit event: " + event.toString());
+        persistAuditEntity(event.getAuditEntities());
+    }
+
+    /**
+     * Method to be invoked for firing audit event.
+     *
+     * @param operation        HBase operation. Values like CREATE/UPDATE/DELETE.
+     * @param entities         List of entities used in HBase operation.
+     * @param encodedRowKeys   List of encodededRowKeys returned from successful HBase operation. To be passed only from deletebyID method.
+     * @param entityDefinition EntityDefinition object used in the HBaseOperation.
+     */
     public void auditOperation(String operation, List<? extends TaggedLogAPIEntity> entities, List<String> encodedRowKeys, EntityDefinition entityDefinition) {
-    	if (isAuditingRequired(entityDefinition.getService())) {
-    		List<GenericAuditEntity> auditEntities = buildAuditEntities(operation, entities, encodedRowKeys, entityDefinition);
-    		if (null != auditEntities && 0 != auditEntities.size())
-    			auditSupport.fireAudit(entityDefinition.getService(), auditEntities);
-    	}
+        if (isAuditingRequired(entityDefinition.getService())) {
+            List<GenericAuditEntity> auditEntities = buildAuditEntities(operation, entities, encodedRowKeys, entityDefinition);
+            if (null != auditEntities && 0 != auditEntities.size()) {
+                auditSupport.fireAudit(entityDefinition.getService(), auditEntities);
+            }
+        }
     }
-    
+
     /**
      * Check if audit is required based on the service names and audit configuration.
+     *
      * @param serviceName: Name of the service call.
      * @return
      */
-    private boolean isAuditingRequired (String serviceName) {
-    	if (EagleConfigFactory.load().isServiceAuditingEnabled()
-    			// As per jira EAGLE-47, HBase operation level audit is done only for Policy, Site and DataSource definitions. 
-    			&& ("AlertDefinitionService".equals(serviceName) || "AlertDataSourceService".equals(serviceName))) {
-    		return true;
-    	}
-
-    	return false;
+    private boolean isAuditingRequired(String serviceName) {
+        if (EagleConfigFactory.load().isServiceAuditingEnabled()
+            // As per jira EAGLE-47, HBase operation level audit is done only for Policy, Site and DataSource definitions.
+            && ("AlertDefinitionService".equals(serviceName) || "AlertDataSourceService".equals(serviceName))) {
+            return true;
+        }
+
+        return false;
     }
-    
+
     /**
      * Build Audit entities based on the available infomration.
-     * @param operation: HBase operation performed.
-     * @param entities: List of entities used in HBase operation.
-     * @param encodedRowKeys: List of encodededRowKeys returned from successful HBase operation. To be passed only from deletebyID method.
+     *
+     * @param operation:        HBase operation performed.
+     * @param entities:         List of entities used in HBase operation.
+     * @param encodedRowKeys:   List of encodededRowKeys returned from successful HBase operation. To be passed only from deletebyID method.
      * @param entityDefinition: EntityDefinition object used in the HBaseOperation.
      * @return
      */
     private List<GenericAuditEntity> buildAuditEntities(String operation, List<? extends TaggedLogAPIEntity> entities, List<String> encodedRowKeys, EntityDefinition entityDefinition) {
-    	String serviceName = entityDefinition.getService();
-    	long timestamp = System.currentTimeMillis()/1000L;
-    	
-    	Authentication authentication = SecurityContextHolder.getContext().getAuthentication();
-    	String userID = null != authentication.getName() ? authentication.getName() : "data not available"; // empty user
-
-    	List<GenericAuditEntity> auditEntities = new ArrayList<GenericAuditEntity>();
-    	GenericAuditEntity auditEntity = new GenericAuditEntity();
-    	
-		if (null != entities && 0 != entities.size())  {
-			Map<String, String> auditTags;
-			for (TaggedLogAPIEntity entity : entities) {
-	    		auditTags = entity.getTags();
-	    		auditTags.put(AUDIT_COLUMN_SERVICE_NAME, serviceName);
-	    		auditTags.put(AUDIT_COLUMN_USER_ID, userID);
-	    		auditTags.put(AUDIT_COLUMN_OPERATION, operation);
-	    		auditTags.put(AUDIT_COLUMN_TIMESTAMP, timestamp + "");
-	    		
-	    		auditEntity = new GenericAuditEntity();
-	    		auditEntity.setTags(auditTags);
-	    		auditEntities.add(auditEntity);
-	    	}
-			
-			return auditEntities;
-		} else if (null != encodedRowKeys && 0 != encodedRowKeys.size()) { // conditions yields true only in case of deleteByID 
-			Map<String, String> auditTags;
-			for (String encodedRowKey : encodedRowKeys) {
-				auditTags = new HashMap<String, String>();
-				auditTags.put("encodedRowKey", encodedRowKey);
-				auditTags.put(AUDIT_COLUMN_SERVICE_NAME, serviceName);
-	    		auditTags.put(AUDIT_COLUMN_USER_ID, userID);
-	    		auditTags.put(AUDIT_COLUMN_OPERATION, operation);
-	    		auditTags.put(AUDIT_COLUMN_TIMESTAMP, timestamp + "");
-	    		
-	    		auditEntity = new GenericAuditEntity();
-	    		auditEntity.setTags(auditTags);
-	    		auditEntities.add(auditEntity);
-			}
-
-			return auditEntities;
-		} else {
-			return null;
-		}
+        String serviceName = entityDefinition.getService();
+        long timestamp = System.currentTimeMillis() / 1000L;
+
+        Authentication authentication = SecurityContextHolder.getContext().getAuthentication();
+        String userID = null != authentication.getName() ? authentication.getName() : "data not available"; // empty user
+
+        List<GenericAuditEntity> auditEntities = new ArrayList<GenericAuditEntity>();
+        GenericAuditEntity auditEntity = new GenericAuditEntity();
+
+        if (null != entities && 0 != entities.size()) {
+            Map<String, String> auditTags;
+            for (TaggedLogAPIEntity entity : entities) {
+                auditTags = entity.getTags();
+                auditTags.put(AUDIT_COLUMN_SERVICE_NAME, serviceName);
+                auditTags.put(AUDIT_COLUMN_USER_ID, userID);
+                auditTags.put(AUDIT_COLUMN_OPERATION, operation);
+                auditTags.put(AUDIT_COLUMN_TIMESTAMP, timestamp + "");
+
+                auditEntity = new GenericAuditEntity();
+                auditEntity.setTags(auditTags);
+                auditEntities.add(auditEntity);
+            }
+
+            return auditEntities;
+        } else if (null != encodedRowKeys && 0 != encodedRowKeys.size()) { // conditions yields true only in case of deleteByID
+            Map<String, String> auditTags;
+            for (String encodedRowKey : encodedRowKeys) {
+                auditTags = new HashMap<String, String>();
+                auditTags.put("encodedRowKey", encodedRowKey);
+                auditTags.put(AUDIT_COLUMN_SERVICE_NAME, serviceName);
+                auditTags.put(AUDIT_COLUMN_USER_ID, userID);
+                auditTags.put(AUDIT_COLUMN_OPERATION, operation);
+                auditTags.put(AUDIT_COLUMN_TIMESTAMP, timestamp + "");
+
+                auditEntity = new GenericAuditEntity();
+                auditEntity.setTags(auditTags);
+                auditEntities.add(auditEntity);
+            }
+
+            return auditEntities;
+        } else {
+            return null;
+        }
     }
 
     /**
      * Persists audit entries into HBase.
+     *
      * @param entityList
      */
     private void persistAuditEntity(List<? extends TaggedLogAPIEntity> entityList) {
-    	try {
-	    	if (null != entityList && 0 != entityList.size()) {
-		    	CreateStatement createStatement = new CreateStatement(entityList, AUDIT_SERVICE_ENDPOINT);
-		        createStatement.execute(DataStorageManager.newDataStorage("hbase"));
-	    	}
-    	} catch (IOException | IllegalDataStorageTypeException exception) {
-			LOG.error("exception in auditing storage event", exception.getMessage());
-		}
+        try {
+            if (null != entityList && 0 != entityList.size()) {
+                CreateStatement createStatement = new CreateStatement(entityList, AUDIT_SERVICE_ENDPOINT);
+                createStatement.execute(DataStorageManager.newDataStorage("hbase"));
+            }
+        } catch (IOException | IllegalDataStorageTypeException exception) {
+            LOG.error("exception in auditing storage event", exception.getMessage());
+        }
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/21187b55/eagle-core/eagle-query/eagle-storage-hbase/src/main/java/org/apache/eagle/storage/hbase/query/GenericQueryBuilder.java
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-query/eagle-storage-hbase/src/main/java/org/apache/eagle/storage/hbase/query/GenericQueryBuilder.java b/eagle-core/eagle-query/eagle-storage-hbase/src/main/java/org/apache/eagle/storage/hbase/query/GenericQueryBuilder.java
index 3dd83cf..961a762 100755
--- a/eagle-core/eagle-query/eagle-storage-hbase/src/main/java/org/apache/eagle/storage/hbase/query/GenericQueryBuilder.java
+++ b/eagle-core/eagle-query/eagle-storage-hbase/src/main/java/org/apache/eagle/storage/hbase/query/GenericQueryBuilder.java
@@ -16,143 +16,149 @@
  */
 package org.apache.eagle.storage.hbase.query;
 
-import java.util.List;
 
+import org.apache.eagle.common.config.EagleConfigFactory;
+import org.apache.eagle.log.entity.SearchCondition;
 import org.apache.eagle.query.GenericEntityQuery;
 import org.apache.eagle.query.GenericQuery;
-import org.apache.eagle.storage.hbase.query.aggregate.GenericAggregateQuery;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import org.apache.eagle.log.entity.SearchCondition;
 import org.apache.eagle.query.aggregate.AggregateCondition;
 import org.apache.eagle.query.aggregate.AggregateFunctionType;
 import org.apache.eagle.query.aggregate.timeseries.SortOption;
+import org.apache.eagle.storage.hbase.query.aggregate.GenericCoprocessorAggregateQuery;
+import org.apache.eagle.storage.hbase.query.aggregate.GenericInMemoryAggregateQuery;
 
-/**
- * TODO: decouple into eagle-query-base module
- *
- * @since : 10/30/14,2014
- */
-public class GenericQueryBuilder{
-	private Logger LOG = LoggerFactory.getLogger(GenericQueryBuilder.class);
-
-	private List<String> outputFields;
-	private String serviceName;
-	private String metricName;
-	private SearchCondition searchCondition;
-	@Deprecated
-	private int parallel;
-	private boolean hasAgg;
-	private List<String> groupByFields;
-	private List<AggregateFunctionType> aggregateFuncTypes;
-	private List<String> aggregateFields;
-	@Deprecated
-	private boolean treeAgg=false;
-	private boolean timeSeries=false;
-	private long intervalmin;
-	private List<SortOption> sortOptions;
-	private int top;
-	private List<AggregateFunctionType> sortFunctionTypes;
-	private List<String> sortFields;
-
-	public static GenericQueryBuilder select(List<String> outputFields) {
-		GenericQueryBuilder builder = new GenericQueryBuilder();
-		builder.output(outputFields);
-		return builder;
-	}
-
-	public GenericQueryBuilder output(List<String> outputFields) {
-		this.outputFields = outputFields;
-		return this;
-	}
-	public GenericQueryBuilder from(String serviceName, String metricName) {
-		this.serviceName = serviceName;
-		this.metricName = metricName;
-		return this;
-	}
-
-	public GenericQueryBuilder where(SearchCondition condition) {
-		this.searchCondition = condition;
-		return this;
-	}
-
-	/**
-	 * TODO: Parameter "parallel" no longer supported, ignore
-	 *
-	 * @param parallel
-	 * @return
-	 */
-	@Deprecated
-	public GenericQueryBuilder parallel(int parallel) {
-//		throw new IllegalArgumentException("Parameter \"parallel\" no longer supported");
-		if(parallel > 0) LOG.warn("Parameter \"parallel\" is deprecated, ignore");
-		return this;
-	}
-
-	/**
-	 * @param hasAgg
-	 * @param groupByFields
-	 * @param aggregateFunctionTypes
-	 * @param aggregateFields
-	 * @return
-	 */
-	public GenericQueryBuilder groupBy(boolean hasAgg, List<String> groupByFields, List<AggregateFunctionType> aggregateFunctionTypes, List<String> aggregateFields) {
-		this.hasAgg = hasAgg;
-		this.groupByFields = groupByFields;
-		this.aggregateFuncTypes = aggregateFunctionTypes;
-		this.aggregateFields = aggregateFields;
-		return this;
-	}
-
-	public GenericQueryBuilder timeSeries(boolean timeSeries, long intervalMin) {
-		this.timeSeries = timeSeries;
-		this.intervalmin = intervalMin;
-		return this;
-	}
-
-	public GenericQueryBuilder orderBy(List<SortOption> sortOptions, List<AggregateFunctionType> sortFunctionTypes, List<String> sortFields) {
-		this.sortOptions = sortOptions;
-		this.sortFunctionTypes = sortFunctionTypes;
-		this.sortFields = sortFields;
-		return this;
-	}
-
-	public GenericQueryBuilder top(int top) {
-		this.top = top;
-		return this;
-	}
-
-	/**
-	 * TODO: Parameter "treeAgg" no longer supported, ignore
-	 *
-	 * @param treeAgg
-	 * @return
-	 */
-	@Deprecated
-	public GenericQueryBuilder treeAgg(boolean treeAgg){
-		if(treeAgg) LOG.warn("Parameter \"treeAgg\" is deprecated, ignore");
-		return this;
-	}
-
-	public GenericQuery build() throws Exception {
-		if(hasAgg){
-			if(LOG.isDebugEnabled()) LOG.debug("Build GroupAggregateQuery");
-			AggregateCondition aggregateCondition = new AggregateCondition();
-			aggregateCondition.setGroupbyFields(this.groupByFields);
-			aggregateCondition.setAggregateFunctionTypes(this.aggregateFuncTypes);
-			aggregateCondition.setAggregateFields(this.aggregateFields);
-			aggregateCondition.setTimeSeries(this.timeSeries);
-			aggregateCondition.setIntervalMS(this.intervalmin * 60 * 1000);
-			return new GenericAggregateQuery(this.serviceName,
-					this.searchCondition,
-					aggregateCondition,
-					this.metricName,
-					this.sortOptions,this.sortFunctionTypes,this.sortFields,
-					this.top);
-		} else {
-			if(LOG.isDebugEnabled()) LOG.debug("Build GenericBatchQuery");
-			return new GenericEntityQuery(this.serviceName,this.searchCondition,this.metricName);
-		}
-	}
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.List;
+
+public class GenericQueryBuilder {
+    private static final Logger LOG = LoggerFactory.getLogger(GenericQueryBuilder.class);
+
+    private List<String> outputFields;
+    private String serviceName;
+    private String metricName;
+    private SearchCondition searchCondition;
+    @Deprecated
+    private int parallel;
+    private boolean hasAgg;
+    private List<String> groupByFields;
+    private List<AggregateFunctionType> aggregateFuncTypes;
+    private List<String> aggregateFields;
+    @Deprecated
+    private boolean treeAgg = false;
+    private boolean timeSeries = false;
+    private long intervalmin;
+    private List<SortOption> sortOptions;
+    private int top;
+    private List<AggregateFunctionType> sortFunctionTypes;
+    private List<String> sortFields;
+
+    public static GenericQueryBuilder select(List<String> outputFields) {
+        GenericQueryBuilder builder = new GenericQueryBuilder();
+        builder.output(outputFields);
+        return builder;
+    }
+
+    public GenericQueryBuilder output(List<String> outputFields) {
+        this.outputFields = outputFields;
+        return this;
+    }
+
+    public GenericQueryBuilder from(String serviceName, String metricName) {
+        this.serviceName = serviceName;
+        this.metricName = metricName;
+        return this;
+    }
+
+    public GenericQueryBuilder where(SearchCondition condition) {
+        this.searchCondition = condition;
+        return this;
+    }
+
+    /**
+     * @deprecated Parameter "parallel" no longer supported, ignore.
+     */
+    @Deprecated
+    public GenericQueryBuilder parallel(int parallel) {
+        if (parallel > 0) {
+            LOG.warn("Parameter \"parallel\" is deprecated, ignore");
+        }
+        return this;
+    }
+
+    public GenericQueryBuilder groupBy(boolean hasAgg, List<String> groupByFields, List<AggregateFunctionType> aggregateFunctionTypes, List<String> aggregateFields) {
+        this.hasAgg = hasAgg;
+        this.groupByFields = groupByFields;
+        this.aggregateFuncTypes = aggregateFunctionTypes;
+        this.aggregateFields = aggregateFields;
+        return this;
+    }
+
+    public GenericQueryBuilder timeSeries(boolean timeSeries, long intervalMin) {
+        this.timeSeries = timeSeries;
+        this.intervalmin = intervalMin;
+        return this;
+    }
+
+    public GenericQueryBuilder orderBy(List<SortOption> sortOptions, List<AggregateFunctionType> sortFunctionTypes, List<String> sortFields) {
+        this.sortOptions = sortOptions;
+        this.sortFunctionTypes = sortFunctionTypes;
+        this.sortFields = sortFields;
+        return this;
+    }
+
+    public GenericQueryBuilder top(int top) {
+        this.top = top;
+        return this;
+    }
+
+    /**
+     * @deprecated Parameter "treeAgg" no longer supported, ignore.
+     */
+    @Deprecated
+    public GenericQueryBuilder treeAgg(boolean treeAgg) {
+        if (treeAgg) {
+            LOG.warn("Parameter \"treeAgg\" is deprecated, ignore");
+        }
+        return this;
+    }
+
+    public GenericQuery build() throws Exception {
+        if (hasAgg) {
+            if (LOG.isDebugEnabled()) {
+                LOG.debug("Build GroupAggregateQuery");
+            }
+            AggregateCondition aggregateCondition = new AggregateCondition();
+            aggregateCondition.setGroupbyFields(this.groupByFields);
+            aggregateCondition.setAggregateFunctionTypes(this.aggregateFuncTypes);
+            aggregateCondition.setAggregateFields(this.aggregateFields);
+            aggregateCondition.setTimeSeries(this.timeSeries);
+            aggregateCondition.setIntervalMS(this.intervalmin * 60 * 1000);
+            if (EagleConfigFactory.load().isCoprocessorEnabled()) {
+                return new GenericCoprocessorAggregateQuery(this.serviceName,
+                        this.searchCondition,
+                        aggregateCondition,
+                        this.metricName,
+                        this.sortOptions,
+                        this.sortFunctionTypes,
+                        this.sortFields,
+                        this.top);
+            } else {
+                return new GenericInMemoryAggregateQuery(this.serviceName,
+                        this.searchCondition,
+                        aggregateCondition,
+                        this.metricName,
+                        this.sortOptions,
+                        this.sortFunctionTypes,
+                        this.sortFields,
+                        this.top);
+            }
+        } else {
+            if (LOG.isDebugEnabled()) {
+                LOG.debug("Build GenericBatchQuery");
+            }
+            return new GenericEntityQuery(this.serviceName, this.searchCondition, this.metricName);
+        }
+    }
 }



[09/52] [abbrv] incubator-eagle git commit: [EAGLE-496] fix code style of jpm

Posted by yo...@apache.org.
[EAGLE-496] fix code style of jpm

Author: wujinhu <wu...@126.com>

Closes #383 from wujinhu/EAGLE-496.


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

Branch: refs/heads/master
Commit: 0b852cbcd4243e71f51ae7a42c68e1ce7571545e
Parents: 6a55b59
Author: wujinhu <wu...@126.com>
Authored: Wed Aug 24 20:22:58 2016 +0800
Committer: Hao Chen <ha...@apache.org>
Committed: Wed Aug 24 20:22:58 2016 +0800

----------------------------------------------------------------------
 .../queue/common/HadoopYarnResourceUtils.java   |   2 +-
 .../queue/common/YarnURLSelectorImpl.java       |   2 +-
 .../storm/HadoopQueueRunningExtractor.java      |   2 +-
 .../eagle/jpm/mr/historyentity/JobConfig.java   |   2 +-
 .../jpm/mr/historyentity/JobConfigSerDeser.java |  15 +-
 .../JobConfigurationAPIEntity.java              |  13 +-
 .../jpm/mr/historyentity/JobEventAPIEntity.java |   4 +-
 .../mr/historyentity/JobExecutionAPIEntity.java |  23 +-
 .../JobProcessTimeStampEntity.java              |   3 +-
 .../TaskAttemptCounterAPIEntity.java            |   7 +-
 .../TaskAttemptExecutionAPIEntity.java          |  17 +-
 .../historyentity/TaskExecutionAPIEntity.java   |  17 +-
 .../TaskFailureCountAPIEntity.java              |   3 +-
 .../mr/runningentity/JobExecutionAPIEntity.java |  10 +-
 .../TaskAttemptExecutionAPIEntity.java          |   6 +-
 .../runningentity/TaskExecutionAPIEntity.java   |   6 +-
 .../jpm/spark/crawl/JHFInputStreamReader.java   |   3 +-
 .../eagle/jpm/spark/crawl/JHFParserBase.java    |   4 +-
 .../jpm/spark/crawl/JHFSparkEventReader.java    |  82 +++--
 .../eagle/jpm/spark/crawl/JHFSparkParser.java   |  12 +-
 .../SparkFilesystemInputStreamReaderImpl.java   |   4 +-
 .../eagle/jpm/spark/entity/JobConfig.java       |   3 +-
 .../apache/eagle/jpm/spark/entity/SparkApp.java |  11 +-
 .../eagle/jpm/spark/entity/SparkExecutor.java   |   4 +-
 .../apache/eagle/jpm/spark/entity/SparkJob.java |  24 +-
 .../eagle/jpm/spark/entity/SparkStage.java      |  32 +-
 .../eagle/jpm/spark/entity/SparkTask.java       |   4 +-
 .../history/crawler/AbstractJobHistoryDAO.java  |  61 ++--
 .../crawler/DefaultJHFInputStreamCallback.java  |  24 +-
 .../history/crawler/EagleOutputCollector.java   |   6 +-
 .../mr/history/crawler/JHFCrawlerDriver.java    |   4 +-
 .../history/crawler/JHFCrawlerDriverImpl.java   | 193 +++++------
 .../history/crawler/JHFInputStreamCallback.java |   4 +-
 .../crawler/JobHistoryContentFilter.java        |   5 +
 .../crawler/JobHistoryContentFilterBuilder.java |  53 +--
 .../crawler/JobHistoryContentFilterImpl.java    |  42 +--
 .../mr/history/crawler/JobHistoryDAOImpl.java   |  80 ++---
 .../jpm/mr/history/crawler/JobHistoryLCM.java   |  24 +-
 .../JobHistorySpoutCollectorInterceptor.java    |   8 +-
 .../HistoryJobEntityCreationListener.java       |   6 +-
 .../mr/history/parser/JHFEventReaderBase.java   | 319 ++++++++++---------
 .../mr/history/parser/JHFMRVer1EventReader.java |  10 +-
 .../jpm/mr/history/parser/JHFMRVer1Parser.java  |  15 +-
 .../mr/history/parser/JHFMRVer2EventReader.java |   5 +-
 .../jpm/mr/history/parser/JHFMRVer2Parser.java  |  10 +-
 .../jpm/mr/history/parser/JHFParserBase.java    |   5 +-
 .../parser/JHFWriteNotCompletedException.java   |   6 +-
 ...JobConfigurationCreationServiceListener.java |   8 +-
 .../JobEntityCreationEagleServiceListener.java  |   2 +-
 .../parser/JobEntityCreationPublisher.java      |   5 -
 .../parser/JobEntityLifecycleAggregator.java    |  68 ++--
 .../mr/history/parser/TaskFailureListener.java  |  13 +-
 .../eagle/jpm/mr/running/MRRunningJobMain.java  |  11 +-
 .../running/config/MRRunningConfigManager.java  |  13 +-
 .../jpm/mr/running/parser/MRJobParser.java      |  62 ++--
 .../mr/running/recover/MRRunningJobManager.java |   8 +-
 .../running/storm/MRRunningJobFetchSpout.java   |   6 +-
 .../mr/running/storm/MRRunningJobParseBolt.java |   6 +-
 .../history/config/SparkHistoryCrawlConfig.java |   3 +-
 .../status/JobHistoryZKStateManager.java        | 153 +++++----
 .../spark/history/status/ZKStateConstant.java   |   2 +-
 .../history/storm/FinishedSparkJobSpout.java    |  23 +-
 .../history/storm/SparkHistoryTopology.java     |  14 +-
 .../spark/history/storm/SparkJobParseBolt.java  |  16 +-
 .../eagle/jpm/spark/history/storm/TestHDFS.java |   4 +-
 .../jpm/spark/running/SparkRunningJobApp.java   |   9 +-
 .../spark/running/SparkRunningJobAppConfig.java |  17 +-
 .../spark/running/entities/SparkAppEntity.java  |  11 +-
 .../running/entities/SparkExecutorEntity.java   |   2 +-
 .../spark/running/entities/SparkJobEntity.java  |   2 +-
 .../running/entities/SparkStageEntity.java      |  24 +-
 .../spark/running/entities/SparkTaskEntity.java |   2 +-
 .../running/parser/SparkApplicationParser.java  |  44 +--
 .../running/recover/SparkRunningJobManager.java |   7 +-
 .../storm/SparkRunningJobFetchSpout.java        |   6 +-
 .../running/storm/SparkRunningJobParseBolt.java |   6 +-
 .../org/apache/eagle/jpm/util/Constants.java    |  35 +-
 .../org/apache/eagle/jpm/util/HDFSUtil.java     |  12 +-
 .../org/apache/eagle/jpm/util/JSONUtil.java     |  30 +-
 .../eagle/jpm/util/JobNameNormalization.java    | 178 ++++++-----
 .../org/apache/eagle/jpm/util/MRJobTagName.java |   5 +-
 .../eagle/jpm/util/SparkEntityConstant.java     |   4 +-
 .../apache/eagle/jpm/util/SparkJobTagName.java  |   5 +-
 .../java/org/apache/eagle/jpm/util/Utils.java   |  16 +-
 .../util/jobcounter/CounterGroupDictionary.java |  13 +-
 .../jpm/util/jobcounter/CounterGroupKey.java    |   7 +-
 .../eagle/jpm/util/jobcounter/CounterKey.java   |   5 +-
 .../util/jobcounter/JobCounterException.java    |  12 +-
 .../eagle/jpm/util/jobcounter/JobCounters.java  |   2 +-
 .../util/jobcounter/JobCountersSerDeser.java    |   2 +-
 .../jpm/util/jobrecover/RunningJobManager.java  |  15 +-
 .../util/resourceFetch/RMResourceFetcher.java   | 305 +++++++++---------
 .../jpm/util/resourceFetch/ResourceFetcher.java |   4 +-
 .../SparkHistoryServerResourceFetcher.java      |  29 +-
 .../connection/InputStreamUtils.java            |  77 +++--
 .../util/resourceFetch/connection/JobUtils.java |  42 +--
 .../connection/URLConnectionUtils.java          | 121 ++++---
 .../resourceFetch/ha/AbstractURLSelector.java   |  32 +-
 .../util/resourceFetch/ha/HAURLSelector.java    |  14 +-
 .../resourceFetch/ha/HAURLSelectorImpl.java     | 150 ++++-----
 .../jpm/util/resourceFetch/model/AppInfo.java   | 251 ++++++++-------
 .../util/resourceFetch/model/Applications.java  |  20 +-
 .../util/resourceFetch/model/AppsWrapper.java   |  22 +-
 .../util/resourceFetch/model/ClusterInfo.java   |   4 +-
 .../resourceFetch/model/ClusterInfoWrapper.java |   4 +-
 .../resourceFetch/model/JobCounterGroup.java    |   8 +-
 .../resourceFetch/model/JobCounterItem.java     |  11 +-
 .../util/resourceFetch/model/JobCounters.java   |   8 +-
 .../resourceFetch/model/JobCountersWrapper.java |   4 +-
 .../jpm/util/resourceFetch/model/MRJob.java     |   4 +-
 .../util/resourceFetch/model/MRJobsWrapper.java |   4 +-
 .../jpm/util/resourceFetch/model/MRTask.java    |   4 +-
 .../util/resourceFetch/model/MRTaskAttempt.java |   4 +-
 .../model/MRTaskAttemptWrapper.java             |   4 +-
 .../resourceFetch/model/MRTaskAttempts.java     |   4 +-
 .../jpm/util/resourceFetch/model/MRTasks.java   |   4 +-
 .../resourceFetch/model/MRTasksWrapper.java     |   4 +-
 .../jpm/util/resourceFetch/model/MrJobs.java    |   4 +-
 .../resourceFetch/model/SparkApplication.java   |   4 +-
 .../model/SparkApplicationAttempt.java          |   4 +-
 .../model/SparkApplicationWrapper.java          |   4 +-
 .../util/resourceFetch/model/SparkExecutor.java |   4 +-
 .../jpm/util/resourceFetch/model/SparkJob.java  |   4 +-
 .../util/resourceFetch/model/SparkStage.java    |   4 +-
 .../jpm/util/resourceFetch/model/SparkTask.java |   4 +-
 .../model/SparkTaskInputMetrics.java            |   4 +-
 .../resourceFetch/model/SparkTaskMetrics.java   |   4 +-
 .../model/SparkTaskShuffleReadMetrics.java      |   4 +-
 .../model/SparkTaskShuffleWriteMetrics.java     |   4 +-
 .../resourceFetch/model/TaskCounterGroup.java   |   5 +-
 .../resourceFetch/model/TaskCounterItem.java    |   4 +-
 .../util/resourceFetch/model/TaskCounters.java  |   5 +-
 .../model/TaskCountersWrapper.java              |   4 +-
 .../url/JobListServiceURLBuilderImpl.java       |  56 ++--
 .../resourceFetch/url/ServiceURLBuilder.java    |   4 +-
 .../SparkCompleteJobServiceURLBuilderImpl.java  |   2 +-
 .../url/SparkJobServiceURLBuilderImpl.java      |   2 +-
 .../jpm/util/resourceFetch/url/URLUtil.java     |   2 +-
 .../hive/jobrunning/HiveJobFetchSpout.java      |  12 +-
 139 files changed, 1783 insertions(+), 1582 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/0b852cbc/eagle-jpm/eagle-hadoop-queue/src/main/java/org/apache/eagle/hadoop/queue/common/HadoopYarnResourceUtils.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-hadoop-queue/src/main/java/org/apache/eagle/hadoop/queue/common/HadoopYarnResourceUtils.java b/eagle-jpm/eagle-hadoop-queue/src/main/java/org/apache/eagle/hadoop/queue/common/HadoopYarnResourceUtils.java
index f2c4b1f..2802449 100644
--- a/eagle-jpm/eagle-hadoop-queue/src/main/java/org/apache/eagle/hadoop/queue/common/HadoopYarnResourceUtils.java
+++ b/eagle-jpm/eagle-hadoop-queue/src/main/java/org/apache/eagle/hadoop/queue/common/HadoopYarnResourceUtils.java
@@ -20,7 +20,7 @@ package org.apache.eagle.hadoop.queue.common;
 
 import com.typesafe.config.Config;
 import org.apache.eagle.jpm.util.Constants;
-import org.apache.eagle.jpm.util.resourceFetch.connection.InputStreamUtils;
+import org.apache.eagle.jpm.util.resourcefetch.connection.InputStreamUtils;
 import org.codehaus.jackson.JsonParser;
 import org.codehaus.jackson.map.ObjectMapper;
 

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/0b852cbc/eagle-jpm/eagle-hadoop-queue/src/main/java/org/apache/eagle/hadoop/queue/common/YarnURLSelectorImpl.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-hadoop-queue/src/main/java/org/apache/eagle/hadoop/queue/common/YarnURLSelectorImpl.java b/eagle-jpm/eagle-hadoop-queue/src/main/java/org/apache/eagle/hadoop/queue/common/YarnURLSelectorImpl.java
index 05e3be9..02f67d4 100644
--- a/eagle-jpm/eagle-hadoop-queue/src/main/java/org/apache/eagle/hadoop/queue/common/YarnURLSelectorImpl.java
+++ b/eagle-jpm/eagle-hadoop-queue/src/main/java/org/apache/eagle/hadoop/queue/common/YarnURLSelectorImpl.java
@@ -19,7 +19,7 @@
 package org.apache.eagle.hadoop.queue.common;
 
 import org.apache.eagle.jpm.util.Constants;
-import org.apache.eagle.jpm.util.resourceFetch.ha.AbstractURLSelector;
+import org.apache.eagle.jpm.util.resourcefetch.ha.AbstractURLSelector;
 
 public class YarnURLSelectorImpl extends AbstractURLSelector {
 

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/0b852cbc/eagle-jpm/eagle-hadoop-queue/src/main/java/org/apache/eagle/hadoop/queue/storm/HadoopQueueRunningExtractor.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-hadoop-queue/src/main/java/org/apache/eagle/hadoop/queue/storm/HadoopQueueRunningExtractor.java b/eagle-jpm/eagle-hadoop-queue/src/main/java/org/apache/eagle/hadoop/queue/storm/HadoopQueueRunningExtractor.java
index 975e633..3c4391b 100644
--- a/eagle-jpm/eagle-hadoop-queue/src/main/java/org/apache/eagle/hadoop/queue/storm/HadoopQueueRunningExtractor.java
+++ b/eagle-jpm/eagle-hadoop-queue/src/main/java/org/apache/eagle/hadoop/queue/storm/HadoopQueueRunningExtractor.java
@@ -27,7 +27,7 @@ import org.apache.eagle.hadoop.queue.common.HadoopYarnResourceUtils;
 import org.apache.eagle.hadoop.queue.common.YarnClusterResourceURLBuilder;
 import org.apache.eagle.hadoop.queue.common.YarnURLSelectorImpl;
 import org.apache.eagle.jpm.util.Constants;
-import org.apache.eagle.jpm.util.resourceFetch.ha.HAURLSelector;
+import org.apache.eagle.jpm.util.resourcefetch.ha.HAURLSelector;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/0b852cbc/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/mr/historyentity/JobConfig.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/mr/historyentity/JobConfig.java b/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/mr/historyentity/JobConfig.java
index 97ebd50..35f346b 100644
--- a/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/mr/historyentity/JobConfig.java
+++ b/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/mr/historyentity/JobConfig.java
@@ -32,7 +32,7 @@ public final class JobConfig {
         this.config = config;
     }
     
-    public String toString(){
+    public String toString() {
         return config.toString();
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/0b852cbc/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/mr/historyentity/JobConfigSerDeser.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/mr/historyentity/JobConfigSerDeser.java b/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/mr/historyentity/JobConfigSerDeser.java
index 5af4377..cfa50f9 100644
--- a/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/mr/historyentity/JobConfigSerDeser.java
+++ b/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/mr/historyentity/JobConfigSerDeser.java
@@ -35,11 +35,15 @@ public class JobConfigSerDeser implements EntitySerDeser<JobConfig> {
         String sb = Bytes.toString(bytes);
         String[] keyValue = sb.split(",");
         for (String pair : keyValue) {
-            String str[] = pair.split(":");
-            if (pair.equals("") || str[0].equals("")) continue;            
+            String[] str = pair.split(":");
+            if (pair.equals("") || str[0].equals("")) {
+                continue;
+            }
             String key = str[0];
             String value = "";
-            if (str.length == 2) value = str[1];
+            if (str.length == 2) {
+                value = str[1];
+            }
             map.put(key, value);
         }
         return jc;
@@ -49,14 +53,15 @@ public class JobConfigSerDeser implements EntitySerDeser<JobConfig> {
     public byte[] serialize(JobConfig conf) {
         Map<String, String> map = conf.getConfig();
         StringBuilder sb = new StringBuilder();
-        for (Entry<String, String> entry : map.entrySet())
+        for (Entry<String, String> entry : map.entrySet()) {
             sb.append(entry.getKey() + ":" + entry.getValue() + ",");
+        }
         sb.deleteCharAt(sb.length() - 1);
         return sb.toString().getBytes();
     }
 
     @Override
-    public Class<JobConfig> type(){
+    public Class<JobConfig> type() {
         return JobConfig.class;
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/0b852cbc/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/mr/historyentity/JobConfigurationAPIEntity.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/mr/historyentity/JobConfigurationAPIEntity.java b/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/mr/historyentity/JobConfigurationAPIEntity.java
index 3a09c5f..d186fd4 100644
--- a/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/mr/historyentity/JobConfigurationAPIEntity.java
+++ b/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/mr/historyentity/JobConfigurationAPIEntity.java
@@ -22,7 +22,7 @@ import org.apache.eagle.jpm.util.Constants;
 import org.apache.eagle.log.entity.meta.*;
 import org.codehaus.jackson.map.annotate.JsonSerialize;
 
-@JsonSerialize(include=JsonSerialize.Inclusion.NON_NULL)
+@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
 @Table("eaglejpa")
 @ColumnFamily("f")
 @Prefix("jconf")
@@ -30,9 +30,9 @@ import org.codehaus.jackson.map.annotate.JsonSerialize;
 @TimeSeries(true)
 @Partition({"site"})
 @Indexes({
-        @Index(name="Index_1_jobId", columns = { "jobId" }, unique = true),
-        @Index(name="Index_2_jobDefId", columns = { "jobDefId" }, unique = false)
-})
+        @Index(name = "Index_1_jobId", columns = { "jobId" }, unique = true),
+        @Index(name = "Index_2_jobDefId", columns = { "jobDefId" }, unique = false)
+    })
 public class JobConfigurationAPIEntity extends JobBaseAPIEntity {
     
     @Column("a")
@@ -45,20 +45,25 @@ public class JobConfigurationAPIEntity extends JobBaseAPIEntity {
     public JobConfig getJobConfig() {
         return jobConfig;
     }
+
     public void setJobConfig(JobConfig jobConfig) {
         this.jobConfig = jobConfig;
         _pcs.firePropertyChange("jobConfig", null, null);
     }
+
     public String getConfigJobName() {
         return configJobName;
     }
+
     public void setConfigJobName(String configJobName) {
         this.configJobName = configJobName;
         _pcs.firePropertyChange("configJobName", null, null);
     }
+
     public String getAlertEmailList() {
         return alertEmailList;
     }
+
     public void setAlertEmailList(String alertEmailList) {
         this.alertEmailList = alertEmailList;
         _pcs.firePropertyChange("alertEmailList", null, null);

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/0b852cbc/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/mr/historyentity/JobEventAPIEntity.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/mr/historyentity/JobEventAPIEntity.java b/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/mr/historyentity/JobEventAPIEntity.java
index b289a9c..c6bb8e4 100644
--- a/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/mr/historyentity/JobEventAPIEntity.java
+++ b/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/mr/historyentity/JobEventAPIEntity.java
@@ -22,7 +22,7 @@ import org.apache.eagle.jpm.util.Constants;
 import org.apache.eagle.log.entity.meta.*;
 import org.codehaus.jackson.map.annotate.JsonSerialize;
 
-@JsonSerialize(include=JsonSerialize.Inclusion.NON_NULL)
+@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
 @Table("eaglejpa")
 @ColumnFamily("f")
 @Prefix("jevent")
@@ -30,13 +30,13 @@ import org.codehaus.jackson.map.annotate.JsonSerialize;
 @TimeSeries(true)
 @Partition({"site"})
 public class JobEventAPIEntity extends JobBaseAPIEntity {
-
     @Column("a")
     private String eventType;
 
     public String getEventType() {
         return eventType;
     }
+
     public void setEventType(String eventType) {
         this.eventType = eventType;
         _pcs.firePropertyChange("eventType", null, null);

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/0b852cbc/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/mr/historyentity/JobExecutionAPIEntity.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/mr/historyentity/JobExecutionAPIEntity.java b/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/mr/historyentity/JobExecutionAPIEntity.java
index d9093ff..1f75f07 100644
--- a/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/mr/historyentity/JobExecutionAPIEntity.java
+++ b/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/mr/historyentity/JobExecutionAPIEntity.java
@@ -23,7 +23,7 @@ import org.apache.eagle.jpm.util.jobcounter.JobCounters;
 import org.apache.eagle.log.entity.meta.*;
 import org.codehaus.jackson.map.annotate.JsonSerialize;
 
-@JsonSerialize(include=JsonSerialize.Inclusion.NON_NULL)
+@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
 @Table("eaglejpa")
 @ColumnFamily("f")
 @Prefix("jexec")
@@ -31,8 +31,8 @@ import org.codehaus.jackson.map.annotate.JsonSerialize;
 @TimeSeries(true)
 @Partition({"site"})
 @Indexes({
-    @Index(name="Index_1_jobId", columns = { "jobId" }, unique = true),
-    @Index(name="Index_2_jobDefId", columns = { "jobDefId" }, unique = false)
+    @Index(name = "Index_1_jobId", columns = { "jobId" }, unique = true),
+    @Index(name = "Index_2_jobDefId", columns = { "jobDefId" }, unique = false)
     })
 public class JobExecutionAPIEntity extends JobBaseAPIEntity {
     @Column("a")
@@ -85,6 +85,7 @@ public class JobExecutionAPIEntity extends JobBaseAPIEntity {
     public long getDurationTime() {
         return durationTime;
     }
+
     public void setDurationTime(long durationTime) {
         this.durationTime = durationTime;
         valueChanged("durationTime");
@@ -93,59 +94,75 @@ public class JobExecutionAPIEntity extends JobBaseAPIEntity {
     public String getCurrentState() {
         return currentState;
     }
+
     public void setCurrentState(String currentState) {
         this.currentState = currentState;
         _pcs.firePropertyChange("currentState", null, null);
     }
+
     public long getStartTime() {
         return startTime;
     }
+
     public void setStartTime(long startTime) {
         this.startTime = startTime;
         _pcs.firePropertyChange("startTime", null, null);
     }
+
     public long getEndTime() {
         return endTime;
     }
+
     public void setEndTime(long endTime) {
         this.endTime = endTime;
         _pcs.firePropertyChange("endTime", null, null);
     }
+
     public int getNumTotalMaps() {
         return numTotalMaps;
     }
+
     public void setNumTotalMaps(int numTotalMaps) {
         this.numTotalMaps = numTotalMaps;
         _pcs.firePropertyChange("numTotalMaps", null, null);
     }
+
     public int getNumFailedMaps() {
         return numFailedMaps;
     }
+
     public void setNumFailedMaps(int numFailedMaps) {
         this.numFailedMaps = numFailedMaps;
         _pcs.firePropertyChange("numFailedMaps", null, null);
     }
+
     public int getNumFinishedMaps() {
         return numFinishedMaps;
     }
+
     public void setNumFinishedMaps(int numFinishedMaps) {
         this.numFinishedMaps = numFinishedMaps;
         _pcs.firePropertyChange("numFinishedMaps", null, null);
     }
+
     public int getNumTotalReduces() {
         return numTotalReduces;
     }
+
     public void setNumTotalReduces(int numTotalReduces) {
         this.numTotalReduces = numTotalReduces;
         _pcs.firePropertyChange("numTotalReduces", null, null);
     }
+
     public int getNumFailedReduces() {
         return numFailedReduces;
     }
+
     public void setNumFailedReduces(int numFailedReduces) {
         this.numFailedReduces = numFailedReduces;
         _pcs.firePropertyChange("numFailedReduces", null, null);
     }
+
     public int getNumFinishedReduces() {
         return numFinishedReduces;
     }

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/0b852cbc/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/mr/historyentity/JobProcessTimeStampEntity.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/mr/historyentity/JobProcessTimeStampEntity.java b/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/mr/historyentity/JobProcessTimeStampEntity.java
index df57657..6afe347 100644
--- a/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/mr/historyentity/JobProcessTimeStampEntity.java
+++ b/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/mr/historyentity/JobProcessTimeStampEntity.java
@@ -23,7 +23,7 @@ import org.apache.eagle.log.base.taggedlog.TaggedLogAPIEntity;
 import org.apache.eagle.log.entity.meta.*;
 import org.codehaus.jackson.map.annotate.JsonSerialize;
 
-@JsonSerialize(include=JsonSerialize.Inclusion.NON_NULL)
+@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
 @Table("eaglejpa_process")
 @ColumnFamily("f")
 @Prefix("process")
@@ -37,6 +37,7 @@ public class JobProcessTimeStampEntity extends TaggedLogAPIEntity {
     public long getCurrentTimeStamp() {
         return currentTimeStamp;
     }
+
     public void setCurrentTimeStamp(long currentTimeStamp) {
         this.currentTimeStamp = currentTimeStamp;
         _pcs.firePropertyChange("currentTimeStamp", null, null);

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/0b852cbc/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/mr/historyentity/TaskAttemptCounterAPIEntity.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/mr/historyentity/TaskAttemptCounterAPIEntity.java b/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/mr/historyentity/TaskAttemptCounterAPIEntity.java
index 89272bf..e526f45 100644
--- a/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/mr/historyentity/TaskAttemptCounterAPIEntity.java
+++ b/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/mr/historyentity/TaskAttemptCounterAPIEntity.java
@@ -22,7 +22,7 @@ import org.apache.eagle.jpm.util.Constants;
 import org.apache.eagle.log.entity.meta.*;
 import org.codehaus.jackson.map.annotate.JsonSerialize;
 
-@JsonSerialize(include=JsonSerialize.Inclusion.NON_NULL)
+@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
 @Table("eaglejpa_anomaly")
 @ColumnFamily("f")
 @Prefix("tacount")
@@ -40,20 +40,25 @@ public class TaskAttemptCounterAPIEntity extends JobBaseAPIEntity {
     public int getKilledCount() {
         return killedCount;
     }
+
     public void setKilledCount(int killedCount) {
         this.killedCount = killedCount;
         _pcs.firePropertyChange("killedCount", null, null);
     }
+
     public int getFailedCount() {
         return failedCount;
     }
+
     public void setFailedCount(int failedCount) {
         this.failedCount = failedCount;
         _pcs.firePropertyChange("failedCount", null, null);
     }
+
     public int getTotalCount() {
         return totalCount;
     }
+
     public void setTotalCount(int totalCount) {
         this.totalCount = totalCount;
         _pcs.firePropertyChange("totalCount", null, null);

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/0b852cbc/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/mr/historyentity/TaskAttemptExecutionAPIEntity.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/mr/historyentity/TaskAttemptExecutionAPIEntity.java b/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/mr/historyentity/TaskAttemptExecutionAPIEntity.java
index be5566b..620ee1f 100644
--- a/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/mr/historyentity/TaskAttemptExecutionAPIEntity.java
+++ b/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/mr/historyentity/TaskAttemptExecutionAPIEntity.java
@@ -23,7 +23,7 @@ import org.apache.eagle.jpm.util.jobcounter.JobCounters;
 import org.apache.eagle.log.entity.meta.*;
 import org.codehaus.jackson.map.annotate.JsonSerialize;
 
-@JsonSerialize(include=JsonSerialize.Inclusion.NON_NULL)
+@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
 @Table("eaglejpa_task")
 @ColumnFamily("f")
 @Prefix("taexec")
@@ -31,7 +31,7 @@ import org.codehaus.jackson.map.annotate.JsonSerialize;
 @TimeSeries(true)
 @Partition({"site"})
 @Indexes({
-    @Index(name="Index_1_jobId", columns = { "jobID" }, unique = false)
+    @Index(name = "Index_1_jobId", columns = { "jobID" }, unique = false)
     })
 public class TaskAttemptExecutionAPIEntity extends JobBaseAPIEntity {
     @Column("a")
@@ -52,48 +52,61 @@ public class TaskAttemptExecutionAPIEntity extends JobBaseAPIEntity {
     public String getTaskStatus() {
         return taskStatus;
     }
+
     public void setTaskStatus(String taskStatus) {
         this.taskStatus = taskStatus;
         _pcs.firePropertyChange("taskStatus", null, null);
     }
+
     public long getStartTime() {
         return startTime;
     }
+
     public void setStartTime(long startTime) {
         this.startTime = startTime;
         _pcs.firePropertyChange("startTime", null, null);
     }
+
     public long getEndTime() {
         return endTime;
     }
+
     public void setEndTime(long endTime) {
         this.endTime = endTime;
         _pcs.firePropertyChange("endTime", null, null);
     }
+
     public long getDuration() {
         return duration;
     }
+
     public void setDuration(long duration) {
         this.duration = duration;
         _pcs.firePropertyChange("duration", null, null);
     }
+
     public String getError() {
         return error;
     }
+
     public void setError(String error) {
         this.error = error;
         _pcs.firePropertyChange("error", null, null);
     }
+
     public JobCounters getJobCounters() {
         return jobCounters;
     }
+
     public void setJobCounters(JobCounters jobCounters) {
         this.jobCounters = jobCounters;
         _pcs.firePropertyChange("jobCounters", null, null);
     }
+
     public String getTaskAttemptID() {
         return taskAttemptID;
     }
+
     public void setTaskAttemptID(String taskAttemptID) {
         this.taskAttemptID = taskAttemptID;
         _pcs.firePropertyChange("taskAttemptID", null, null);

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/0b852cbc/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/mr/historyentity/TaskExecutionAPIEntity.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/mr/historyentity/TaskExecutionAPIEntity.java b/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/mr/historyentity/TaskExecutionAPIEntity.java
index 9de8b05..bf559d4 100644
--- a/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/mr/historyentity/TaskExecutionAPIEntity.java
+++ b/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/mr/historyentity/TaskExecutionAPIEntity.java
@@ -23,7 +23,7 @@ import org.apache.eagle.jpm.util.jobcounter.JobCounters;
 import org.apache.eagle.log.entity.meta.*;
 import org.codehaus.jackson.map.annotate.JsonSerialize;
 
-@JsonSerialize(include=JsonSerialize.Inclusion.NON_NULL)
+@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
 @Table("eaglejpa_task")
 @ColumnFamily("f")
 @Prefix("texec")
@@ -31,8 +31,8 @@ import org.codehaus.jackson.map.annotate.JsonSerialize;
 @TimeSeries(true)
 @Partition({"site"})
 @Indexes({
-        @Index(name="Index_1_jobId", columns = { "jobId" }, unique = false)
-})
+        @Index(name = "Index_1_jobId", columns = { "jobId" }, unique = false)
+    })
 public class TaskExecutionAPIEntity extends JobBaseAPIEntity {
     @Column("a")
     private String taskStatus;
@@ -50,41 +50,52 @@ public class TaskExecutionAPIEntity extends JobBaseAPIEntity {
     public String getTaskStatus() {
         return taskStatus;
     }
+
     public void setTaskStatus(String taskStatus) {
         this.taskStatus = taskStatus;
         _pcs.firePropertyChange("taskStatus", null, null);
     }
+
     public long getStartTime() {
         return startTime;
     }
+
     public void setStartTime(long startTime) {
         this.startTime = startTime;
         _pcs.firePropertyChange("startTime", null, null);
     }
+
     public long getEndTime() {
         return endTime;
     }
+
     public void setEndTime(long endTime) {
         this.endTime = endTime;
         _pcs.firePropertyChange("endTime", null, null);
     }
+
     public long getDuration() {
         return duration;
     }
+
     public void setDuration(long duration) {
         this.duration = duration;
         _pcs.firePropertyChange("duration", null, null);
     }
+
     public String getError() {
         return error;
     }
+
     public void setError(String error) {
         this.error = error;
         _pcs.firePropertyChange("error", null, null);
     }
+
     public JobCounters getJobCounters() {
         return jobCounters;
     }
+
     public void setJobCounters(JobCounters jobCounters) {
         this.jobCounters = jobCounters;
         _pcs.firePropertyChange("jobCounters", null, null);

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/0b852cbc/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/mr/historyentity/TaskFailureCountAPIEntity.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/mr/historyentity/TaskFailureCountAPIEntity.java b/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/mr/historyentity/TaskFailureCountAPIEntity.java
index 1445a24..31f96da 100644
--- a/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/mr/historyentity/TaskFailureCountAPIEntity.java
+++ b/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/mr/historyentity/TaskFailureCountAPIEntity.java
@@ -22,7 +22,7 @@ import org.apache.eagle.jpm.util.Constants;
 import org.apache.eagle.log.entity.meta.*;
 import org.codehaus.jackson.map.annotate.JsonSerialize;
 
-@JsonSerialize(include=JsonSerialize.Inclusion.NON_NULL)
+@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
 @Table("eaglejpa_anomaly")
 @ColumnFamily("f")
 @Prefix("taskfailurecount")
@@ -37,7 +37,6 @@ public class TaskFailureCountAPIEntity extends JobBaseAPIEntity {
     @Column("c")
     private String taskStatus;
 
-
     public String getTaskStatus() {
         return taskStatus;
     }

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/0b852cbc/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/mr/runningentity/JobExecutionAPIEntity.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/mr/runningentity/JobExecutionAPIEntity.java b/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/mr/runningentity/JobExecutionAPIEntity.java
index 653f1c9..86b6554 100644
--- a/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/mr/runningentity/JobExecutionAPIEntity.java
+++ b/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/mr/runningentity/JobExecutionAPIEntity.java
@@ -20,12 +20,12 @@ package org.apache.eagle.jpm.mr.runningentity;
 
 import org.apache.eagle.jpm.util.Constants;
 import org.apache.eagle.jpm.util.jobcounter.JobCounters;
-import org.apache.eagle.jpm.util.resourceFetch.model.AppInfo;
+import org.apache.eagle.jpm.util.resourcefetch.model.AppInfo;
 import org.apache.eagle.log.base.taggedlog.TaggedLogAPIEntity;
 import org.apache.eagle.log.entity.meta.*;
 import org.codehaus.jackson.map.annotate.JsonSerialize;
 
-@JsonSerialize(include=JsonSerialize.Inclusion.NON_NULL)
+@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
 @Table("eagleMRRunningJobs")
 @ColumnFamily("f")
 @Prefix("jobs")
@@ -33,9 +33,9 @@ import org.codehaus.jackson.map.annotate.JsonSerialize;
 @TimeSeries(true)
 @Partition({"site"})
 @Indexes({
-        @Index(name="Index_1_jobId", columns = { "jobId" }, unique = true),
-        @Index(name="Index_2_jobDefId", columns = { "jobDefId" }, unique = false)
-})
+        @Index(name = "Index_1_jobId", columns = { "jobId" }, unique = true),
+        @Index(name = "Index_2_jobDefId", columns = { "jobDefId" }, unique = false)
+    })
 @Tags({"site", "jobId", "jobName", "jobDefId", "jobType", "user", "queue"})
 public class JobExecutionAPIEntity extends TaggedLogAPIEntity {
     @Column("a")

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/0b852cbc/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/mr/runningentity/TaskAttemptExecutionAPIEntity.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/mr/runningentity/TaskAttemptExecutionAPIEntity.java b/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/mr/runningentity/TaskAttemptExecutionAPIEntity.java
index 11a8b4c..088869f 100644
--- a/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/mr/runningentity/TaskAttemptExecutionAPIEntity.java
+++ b/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/mr/runningentity/TaskAttemptExecutionAPIEntity.java
@@ -23,7 +23,7 @@ import org.apache.eagle.log.base.taggedlog.TaggedLogAPIEntity;
 import org.apache.eagle.log.entity.meta.*;
 import org.codehaus.jackson.map.annotate.JsonSerialize;
 
-@JsonSerialize(include=JsonSerialize.Inclusion.NON_NULL)
+@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
 @Table("eagleMRRunningTasks")
 @ColumnFamily("f")
 @Prefix("tasks_exec_attempt")
@@ -31,8 +31,8 @@ import org.codehaus.jackson.map.annotate.JsonSerialize;
 @TimeSeries(true)
 @Partition({"site"})
 @Indexes({
-        @Index(name="Index_1_jobId", columns = { "jobId" }, unique = false)
-})
+        @Index(name = "Index_1_jobId", columns = { "jobId" }, unique = false)
+    })
 @Tags({"site", "jobId", "JobName", "jobDefId", "jobType", "taskType", "taskId", "user", "queue", "host", "rack"})
 public class TaskAttemptExecutionAPIEntity extends TaggedLogAPIEntity {
     @Column("a")

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/0b852cbc/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/mr/runningentity/TaskExecutionAPIEntity.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/mr/runningentity/TaskExecutionAPIEntity.java b/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/mr/runningentity/TaskExecutionAPIEntity.java
index 50e042f..d1d62ee 100644
--- a/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/mr/runningentity/TaskExecutionAPIEntity.java
+++ b/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/mr/runningentity/TaskExecutionAPIEntity.java
@@ -24,7 +24,7 @@ import org.apache.eagle.log.base.taggedlog.TaggedLogAPIEntity;
 import org.apache.eagle.log.entity.meta.*;
 import org.codehaus.jackson.map.annotate.JsonSerialize;
 
-@JsonSerialize(include=JsonSerialize.Inclusion.NON_NULL)
+@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
 @Table("eagleMRRunningTasks")
 @ColumnFamily("f")
 @Prefix("tasks_exec")
@@ -32,8 +32,8 @@ import org.codehaus.jackson.map.annotate.JsonSerialize;
 @TimeSeries(true)
 @Partition({"site"})
 @Indexes({
-        @Index(name="Index_1_jobId", columns = { "jobId" }, unique = false)
-})
+        @Index(name = "Index_1_jobId", columns = { "jobId" }, unique = false)
+    })
 @Tags({"site", "jobId", "JobName", "jobDefId", "jobType", "taskType", "taskId", "user", "queue", "hostname"})
 public class TaskExecutionAPIEntity extends TaggedLogAPIEntity {
     @Column("a")

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/0b852cbc/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/spark/crawl/JHFInputStreamReader.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/spark/crawl/JHFInputStreamReader.java b/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/spark/crawl/JHFInputStreamReader.java
index feeee7b..8a8d0db 100644
--- a/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/spark/crawl/JHFInputStreamReader.java
+++ b/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/spark/crawl/JHFInputStreamReader.java
@@ -20,6 +20,5 @@ package org.apache.eagle.jpm.spark.crawl;
 import java.io.InputStream;
 
 public interface JHFInputStreamReader {
-    public void read(InputStream is) throws Exception;
-
+    void read(InputStream is) throws Exception;
 }

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/0b852cbc/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/spark/crawl/JHFParserBase.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/spark/crawl/JHFParserBase.java b/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/spark/crawl/JHFParserBase.java
index 48701f7..62ba7d9 100644
--- a/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/spark/crawl/JHFParserBase.java
+++ b/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/spark/crawl/JHFParserBase.java
@@ -21,9 +21,9 @@ import java.io.InputStream;
 
 public interface JHFParserBase {
     /**
-     * this method will ensure to close the inputstream
+     * this method will ensure to close the inputStream.
      * @param is
      * @throws Exception
      */
-    public void parse(InputStream is) throws Exception;
+    void parse(InputStream is) throws Exception;
 }

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/0b852cbc/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/spark/crawl/JHFSparkEventReader.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/spark/crawl/JHFSparkEventReader.java b/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/spark/crawl/JHFSparkEventReader.java
index 1b75e81..e298fa3 100644
--- a/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/spark/crawl/JHFSparkEventReader.java
+++ b/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/spark/crawl/JHFSparkEventReader.java
@@ -17,9 +17,6 @@
 
 package org.apache.eagle.jpm.spark.crawl;
 
-import com.typesafe.config.Config;
-import com.typesafe.config.ConfigFactory;
-import org.apache.eagle.jpm.spark.entity.JobConfig;
 import org.apache.eagle.jpm.spark.entity.*;
 import org.apache.eagle.jpm.util.JSONUtil;
 import org.apache.eagle.jpm.util.JobNameNormalization;
@@ -28,6 +25,8 @@ import org.apache.eagle.jpm.util.SparkJobTagName;
 import org.apache.eagle.log.base.taggedlog.TaggedLogAPIEntity;
 import org.apache.eagle.service.client.impl.EagleServiceBaseClient;
 import org.apache.eagle.service.client.impl.EagleServiceClientImpl;
+import com.typesafe.config.Config;
+import com.typesafe.config.ConfigFactory;
 import org.json.simple.JSONArray;
 import org.json.simple.JSONObject;
 import org.slf4j.Logger;
@@ -116,8 +115,8 @@ public class JHFSparkEventReader {
 
         List<String> jobConfs = conf.getStringList("basic.jobConf.additional.info");
         String[] props = {"spark.yarn.app.id", "spark.executor.memory", "spark.driver.host", "spark.driver.port",
-                "spark.driver.memory", "spark.scheduler.pool", "spark.executor.cores", "spark.yarn.am.memory",
-                "spark.yarn.am.cores", "spark.yarn.executor.memoryOverhead", "spark.yarn.driver.memoryOverhead", "spark.yarn.am.memoryOverhead", "spark.master"};
+            "spark.driver.memory", "spark.scheduler.pool", "spark.executor.cores", "spark.yarn.am.memory",
+            "spark.yarn.am.cores", "spark.yarn.executor.memoryOverhead", "spark.yarn.driver.memoryOverhead", "spark.yarn.am.memoryOverhead", "spark.master"};
         jobConfs.addAll(Arrays.asList(props));
         for (String prop : jobConfs) {
             if (sparkProps.containsKey(prop)) {
@@ -363,9 +362,9 @@ public class JHFSparkEventReader {
         stage.setCompleteTime(JSONUtil.getLong(stageInfo, "Completion Time"));
 
         if (stageInfo.containsKey("Failure Reason")) {
-            stage.setStatus(SparkEntityConstant.SPARK_STAGE_STATUS.FAILED.toString());
+            stage.setStatus(SparkEntityConstant.SparkStageStatus.FAILED.toString());
         } else {
-            stage.setStatus(SparkEntityConstant.SPARK_STAGE_STATUS.COMPLETE.toString());
+            stage.setStatus(SparkEntityConstant.SparkStageStatus.COMPLETE.toString());
         }
     }
 
@@ -383,9 +382,9 @@ public class JHFSparkEventReader {
         JSONObject jobResult = JSONUtil.getJSONObject(event, "Job Result");
         String result = JSONUtil.getString(jobResult, "Result");
         if (result.equalsIgnoreCase("JobSucceeded")) {
-            job.setStatus(SparkEntityConstant.SPARK_JOB_STATUS.SUCCEEDED.toString());
+            job.setStatus(SparkEntityConstant.SparkJobStatus.SUCCEEDED.toString());
         } else {
-            job.setStatus(SparkEntityConstant.SPARK_JOB_STATUS.FAILED.toString());
+            job.setStatus(SparkEntityConstant.SparkJobStatus.FAILED.toString());
         }
     }
 
@@ -429,15 +428,23 @@ public class JHFSparkEventReader {
 
         app.setExecutors(executors.values().size());
         long executorMemory = parseExecutorMemory((String) this.getConfigVal(this.app.getConfig(), "spark.executor.memory", String.class.getName()));
-        long driverMemory = parseExecutorMemory(this.isClientMode(app.getConfig()) ? (String) this.getConfigVal(this.app.getConfig(), "spark.yarn.am.memory", String.class.getName()) : (String) this.getConfigVal(app.getConfig(), "spark.driver.memory", String.class.getName()));
-        int executoreCore = (Integer) this.getConfigVal(app.getConfig(), "spark.executor.cores", Integer.class.getName());
-        int driverCore = this.isClientMode(app.getConfig()) ? (Integer) this.getConfigVal(app.getConfig(), "spark.yarn.am.cores", Integer.class.getName()) : (Integer) this.getConfigVal(app.getConfig(), "spark.driver.cores", Integer.class.getName());
+        long driverMemory = parseExecutorMemory(this.isClientMode(app.getConfig())
+            ? (String) this.getConfigVal(this.app.getConfig(), "spark.yarn.am.memory", String.class.getName())
+            : (String) this.getConfigVal(app.getConfig(), "spark.driver.memory", String.class.getName()));
+
+        int executorCore = (Integer) this.getConfigVal(app.getConfig(), "spark.executor.cores", Integer.class.getName());
+        int driverCore = this.isClientMode(app.getConfig())
+            ? (Integer) this.getConfigVal(app.getConfig(), "spark.yarn.am.cores", Integer.class.getName())
+            : (Integer) this.getConfigVal(app.getConfig(), "spark.driver.cores", Integer.class.getName());
+
         long executorMemoryOverhead = this.getMemoryOverhead(app.getConfig(), executorMemory, "spark.yarn.executor.memoryOverhead");
-        long driverMemoryOverhead = this.isClientMode(app.getConfig()) ? this.getMemoryOverhead(app.getConfig(), driverMemory, "spark.yarn.am.memoryOverhead") : this.getMemoryOverhead(app.getConfig(), driverMemory, "spark.yarn.driver.memoryOverhead");
+        long driverMemoryOverhead = this.isClientMode(app.getConfig())
+            ? this.getMemoryOverhead(app.getConfig(), driverMemory, "spark.yarn.am.memoryOverhead")
+            : this.getMemoryOverhead(app.getConfig(), driverMemory, "spark.yarn.driver.memoryOverhead");
 
         app.setExecMemoryBytes(executorMemory);
         app.setDriveMemoryBytes(driverMemory);
-        app.setExecutorCores(executoreCore);
+        app.setExecutorCores(executorCore);
         app.setDriverCores(driverCore);
         app.setExecutorMemoryOverhead(executorMemoryOverhead);
         app.setDriverMemoryOverhead(driverMemoryOverhead);
@@ -450,11 +457,12 @@ public class JHFSparkEventReader {
                 executor.setMemoryOverhead(driverMemoryOverhead);
             } else {
                 executor.setExecMemoryBytes(executorMemory);
-                executor.setCores(executoreCore);
+                executor.setCores(executorCore);
                 executor.setMemoryOverhead(executorMemoryOverhead);
             }
-            if (executor.getEndTime() == 0)
+            if (executor.getEndTime() == 0) {
                 executor.setEndTime(app.getEndTime());
+            }
             this.aggregateExecutorToApp(executor);
         }
         this.flushEntities(executors.values(), false);
@@ -464,16 +472,16 @@ public class JHFSparkEventReader {
     }
 
     private long getMemoryOverhead(JobConfig config, long executorMemory, String fieldName) {
-        long result = 0l;
+        long result = 0L;
         if (config.getConfig().containsKey(fieldName)) {
             result = this.parseExecutorMemory(config.getConfig().get(fieldName) + "m");
-            if(result  == 0l){
-               result = this.parseExecutorMemory(config.getConfig().get(fieldName));
+            if (result == 0L) {
+                result = this.parseExecutorMemory(config.getConfig().get(fieldName));
             }
         }
 
-        if(result == 0l){
-            result =  Math.max(this.parseExecutorMemory(conf.getString("spark.defaultVal.spark.yarn.overhead.min")), executorMemory * conf.getInt("spark.defaultVal." + fieldName + ".factor") / 100);
+        if (result == 0L) {
+            result = Math.max(this.parseExecutorMemory(conf.getString("spark.defaultVal.spark.yarn.overhead.min")), executorMemory * conf.getInt("spark.defaultVal." + fieldName + ".factor") / 100);
         }
         return result;
     }
@@ -588,7 +596,7 @@ public class JHFSparkEventReader {
         job.setNumTask(job.getNumTask() + stage.getNumTasks());
 
 
-        if (stage.getStatus().equalsIgnoreCase(SparkEntityConstant.SPARK_STAGE_STATUS.COMPLETE.toString())) {
+        if (stage.getStatus().equalsIgnoreCase(SparkEntityConstant.SparkStageStatus.COMPLETE.toString())) {
             //if multiple attempts succeed, just count one
             if (!hasStagePriorAttemptSuccess(stage)) {
                 job.setNumCompletedStages(job.getNumCompletedStages() + 1);
@@ -603,7 +611,7 @@ public class JHFSparkEventReader {
         Integer stageAttemptId = Integer.parseInt(stage.getTags().get(SparkJobTagName.SPARK_STAGE_ATTEMPT_ID.toString()));
         for (Integer i = 0; i < stageAttemptId; i++) {
             SparkStage previousStage = stages.get(this.generateStageKey(stage.getTags().get(SparkJobTagName.SPARK_SATGE_ID.toString()), i.toString()));
-            if (previousStage.getStatus().equalsIgnoreCase(SparkEntityConstant.SPARK_STAGE_STATUS.COMPLETE.toString())) {
+            if (previousStage.getStatus().equalsIgnoreCase(SparkEntityConstant.SparkStageStatus.COMPLETE.toString())) {
                 return true;
             }
         }
@@ -659,22 +667,22 @@ public class JHFSparkEventReader {
 
         if (memory.endsWith("g") || memory.endsWith("G")) {
             int executorGB = Integer.parseInt(memory.substring(0, memory.length() - 1));
-            return 1024l * 1024 * 1024 * executorGB;
+            return 1024L * 1024 * 1024 * executorGB;
         } else if (memory.endsWith("m") || memory.endsWith("M")) {
             int executorMB = Integer.parseInt(memory.substring(0, memory.length() - 1));
-            return 1024l * 1024 * executorMB;
+            return 1024L * 1024 * executorMB;
         } else if (memory.endsWith("k") || memory.endsWith("K")) {
             int executorKB = Integer.parseInt(memory.substring(0, memory.length() - 1));
-            return 1024l * executorKB;
+            return 1024L * executorKB;
         } else if (memory.endsWith("t") || memory.endsWith("T")) {
             int executorTB = Integer.parseInt(memory.substring(0, memory.length() - 1));
-            return 1024l * 1024 * 1024 * 1024 * executorTB;
+            return 1024L * 1024 * 1024 * 1024 * executorTB;
         } else if (memory.endsWith("p") || memory.endsWith("P")) {
             int executorPB = Integer.parseInt(memory.substring(0, memory.length() - 1));
-            return 1024l * 1024 * 1024 * 1024 * 1024 * executorPB;
+            return 1024L * 1024 * 1024 * 1024 * 1024 * executorPB;
         }
-        LOG.info("Cannot parse memory info " +  memory);
-        return 0l;
+        LOG.info("Cannot parse memory info " + memory);
+        return 0L;
     }
 
     private void flushEntities(Object entity, boolean forceFlush) {
@@ -709,20 +717,6 @@ public class JHFSparkEventReader {
 
     private void doFlush(List entities) throws Exception {
         LOG.info("start flushing entities of total number " + entities.size());
-//        client.create(entities);
         LOG.info("finish flushing entities of total number " + entities.size());
-//        for(Object entity: entities){
-//            if(entity instanceof SparkApp){
-//                for (Field field : entity.getClass().getDeclaredFields()) {
-//                    field.setAccessible(true); // You might want to set modifier to public first.
-//                    Object value = field.get(entity);
-//                    if (value != null) {
-//                        System.out.println(field.getName() + "=" + value);
-//                    }
-//                }
-//            }
-//        }
     }
-
-
 }

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/0b852cbc/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/spark/crawl/JHFSparkParser.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/spark/crawl/JHFSparkParser.java b/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/spark/crawl/JHFSparkParser.java
index 171cb0f..da049ea 100644
--- a/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/spark/crawl/JHFSparkParser.java
+++ b/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/spark/crawl/JHFSparkParser.java
@@ -33,7 +33,7 @@ public class JHFSparkParser implements JHFParserBase {
 
     JHFSparkEventReader eventReader;
 
-    public JHFSparkParser(JHFSparkEventReader reader){
+    public JHFSparkParser(JHFSparkEventReader reader) {
         this.eventReader = reader;
     }
 
@@ -41,22 +41,22 @@ public class JHFSparkParser implements JHFParserBase {
     public void parse(InputStream is) throws Exception {
         BufferedReader reader = new BufferedReader(new InputStreamReader(is));
         try {
-            String line = null;
+            String line;
 
             JSONParser parser = new JSONParser();
-            while((line = reader.readLine()) != null){
-                try{
+            while ((line = reader.readLine()) != null) {
+                try {
                     JSONObject eventObj = (JSONObject) parser.parse(line);
                     String eventType = (String) eventObj.get("Event");
                     logger.info("Event type: " + eventType);
                     this.eventReader.read(eventObj);
-                }catch(Exception e){
+                } catch (Exception e) {
                     logger.error(String.format("Invalid json string. Fail to parse %s.", line), e);
                 }
             }
             this.eventReader.clearReader();
         } finally {
-            if(reader != null){
+            if (reader != null) {
                 reader.close();
             }
         }

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/0b852cbc/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/spark/crawl/SparkFilesystemInputStreamReaderImpl.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/spark/crawl/SparkFilesystemInputStreamReaderImpl.java b/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/spark/crawl/SparkFilesystemInputStreamReaderImpl.java
index f1d2cd1..3964454 100644
--- a/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/spark/crawl/SparkFilesystemInputStreamReaderImpl.java
+++ b/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/spark/crawl/SparkFilesystemInputStreamReaderImpl.java
@@ -31,7 +31,7 @@ public class SparkFilesystemInputStreamReaderImpl implements JHFInputStreamReade
     private SparkApplicationInfo app;
 
 
-    public SparkFilesystemInputStreamReaderImpl(String site, SparkApplicationInfo app){
+    public SparkFilesystemInputStreamReaderImpl(String site, SparkApplicationInfo app) {
         this.site = site;
         this.app = app;
     }
@@ -45,7 +45,7 @@ public class SparkFilesystemInputStreamReaderImpl implements JHFInputStreamReade
         parser.parse(is);
     }
 
-    public static void main(String[] args) throws Exception{
+    public static void main(String[] args) throws Exception {
         SparkFilesystemInputStreamReaderImpl impl = new SparkFilesystemInputStreamReaderImpl("apollo-phx", new SparkApplicationInfo());
         impl.read(new FileInputStream(new File("E:\\eagle\\application_1459803563374_535667_1")));
     }

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/0b852cbc/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/spark/entity/JobConfig.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/spark/entity/JobConfig.java b/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/spark/entity/JobConfig.java
index 11c4a22..0664954 100644
--- a/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/spark/entity/JobConfig.java
+++ b/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/spark/entity/JobConfig.java
@@ -32,8 +32,9 @@ public class JobConfig implements Serializable {
     public void setConfig(Map<String, String> config) {
         this.config = config;
     }
+
     @Override
-    public String toString(){
+    public String toString() {
         return config.toString();
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/0b852cbc/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/spark/entity/SparkApp.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/spark/entity/SparkApp.java b/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/spark/entity/SparkApp.java
index 528a91f..58697a1 100644
--- a/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/spark/entity/SparkApp.java
+++ b/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/spark/entity/SparkApp.java
@@ -18,10 +18,10 @@
 
 package org.apache.eagle.jpm.spark.entity;
 
+import org.apache.eagle.jpm.util.Constants;
 import org.apache.eagle.log.base.taggedlog.TaggedLogAPIEntity;
 import org.apache.eagle.log.entity.meta.*;
 import org.codehaus.jackson.annotate.JsonIgnoreProperties;
-import org.apache.eagle.jpm.util.Constants;
 
 @Table("eglesprk_apps")
 @ColumnFamily("f")
@@ -31,7 +31,7 @@ import org.apache.eagle.jpm.util.Constants;
 @TimeSeries(true)
 @Tags({"site","sprkAppId", "sprkAppAttemptId", "sprkAppName", "normSprkAppName","user", "queue"})
 @Partition({"site"})
-public class SparkApp extends TaggedLogAPIEntity{
+public class SparkApp extends TaggedLogAPIEntity {
 
     @Column("a")
     private long  startTime;
@@ -222,11 +222,14 @@ public class SparkApp extends TaggedLogAPIEntity{
         return driveMemoryBytes;
     }
 
-    public int getCompleteTasks(){ return completeTasks;}
+    public int getCompleteTasks() {
+        return completeTasks;
+    }
 
     public JobConfig getConfig() {
         return config;
     }
+
     public void setStartTime(long startTime) {
         this.startTime = startTime;
         valueChanged("startTime");
@@ -377,7 +380,7 @@ public class SparkApp extends TaggedLogAPIEntity{
         valueChanged("driveMemoryBytes");
     }
 
-    public void setCompleteTasks(int completeTasks){
+    public void setCompleteTasks(int completeTasks) {
         this.completeTasks = completeTasks;
         valueChanged("completeTasks");
     }

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/0b852cbc/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/spark/entity/SparkExecutor.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/spark/entity/SparkExecutor.java b/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/spark/entity/SparkExecutor.java
index 366e4aa..4b669ef 100644
--- a/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/spark/entity/SparkExecutor.java
+++ b/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/spark/entity/SparkExecutor.java
@@ -18,10 +18,10 @@
 
 package org.apache.eagle.jpm.spark.entity;
 
+import org.apache.eagle.jpm.util.Constants;
 import org.apache.eagle.log.base.taggedlog.TaggedLogAPIEntity;
 import org.apache.eagle.log.entity.meta.*;
 import org.codehaus.jackson.annotate.JsonIgnoreProperties;
-import org.apache.eagle.jpm.util.Constants;
 
 @Table("eglesprk_executors")
 @ColumnFamily("f")
@@ -31,7 +31,7 @@ import org.apache.eagle.jpm.util.Constants;
 @TimeSeries(true)
 @Tags({"site","sprkAppId", "sprkAppAttemptId", "sprkAppName", "normSprkAppName", "executorId","user", "queue"})
 @Partition({"site"})
-public class SparkExecutor extends TaggedLogAPIEntity{
+public class SparkExecutor extends TaggedLogAPIEntity {
 
     @Column("a")
     private String hostPort;

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/0b852cbc/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/spark/entity/SparkJob.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/spark/entity/SparkJob.java b/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/spark/entity/SparkJob.java
index acecb3a..79ac6da 100644
--- a/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/spark/entity/SparkJob.java
+++ b/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/spark/entity/SparkJob.java
@@ -18,10 +18,10 @@
 
 package org.apache.eagle.jpm.spark.entity;
 
+import org.apache.eagle.jpm.util.Constants;
 import org.apache.eagle.log.base.taggedlog.TaggedLogAPIEntity;
 import org.apache.eagle.log.entity.meta.*;
 import org.codehaus.jackson.annotate.JsonIgnoreProperties;
-import org.apache.eagle.jpm.util.Constants;
 
 @Table("eglesprk_jobs")
 @ColumnFamily("f")
@@ -31,34 +31,34 @@ import org.apache.eagle.jpm.util.Constants;
 @TimeSeries(true)
 @Tags({"site","sprkAppId", "sprkAppAttemptId", "sprkAppName", "normSprkAppName", "jobId","user", "queue"})
 @Partition({"site"})
-public class SparkJob extends TaggedLogAPIEntity{
+public class SparkJob extends TaggedLogAPIEntity {
 
     @Column("a")
     private long  submissionTime;
     @Column("b")
     private long completionTime;
     @Column("c")
-    private int numStages=0;
+    private int numStages = 0;
     @Column("d")
     private String status;
     @Column("e")
-    private int numTask=0;
+    private int numTask = 0;
     @Column("f")
-    private int numActiveTasks=0;
+    private int numActiveTasks = 0;
     @Column("g")
-    private int numCompletedTasks=0;
+    private int numCompletedTasks = 0;
     @Column("h")
-    private int numSkippedTasks=0;
+    private int numSkippedTasks = 0;
     @Column("i")
-    private int numFailedTasks=0;
+    private int numFailedTasks = 0;
     @Column("j")
-    private int numActiveStages=0;
+    private int numActiveStages = 0;
     @Column("k")
-    private int numCompletedStages=0;
+    private int numCompletedStages = 0;
     @Column("l")
-    private int numSkippedStages=0;
+    private int numSkippedStages = 0;
     @Column("m")
-    private int numFailedStages=0;
+    private int numFailedStages = 0;
 
     public long getSubmissionTime() {
         return submissionTime;

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/0b852cbc/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/spark/entity/SparkStage.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/spark/entity/SparkStage.java b/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/spark/entity/SparkStage.java
index fcca889..3f56da6 100644
--- a/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/spark/entity/SparkStage.java
+++ b/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/spark/entity/SparkStage.java
@@ -18,10 +18,10 @@
 
 package org.apache.eagle.jpm.spark.entity;
 
+import org.apache.eagle.jpm.util.Constants;
 import org.apache.eagle.log.base.taggedlog.TaggedLogAPIEntity;
 import org.apache.eagle.log.entity.meta.*;
 import org.codehaus.jackson.annotate.JsonIgnoreProperties;
-import org.apache.eagle.jpm.util.Constants;
 
 @Table("eglesprk_stages")
 @ColumnFamily("f")
@@ -31,38 +31,38 @@ import org.apache.eagle.jpm.util.Constants;
 @TimeSeries(true)
 @Tags({"site","sprkAppId", "sprkAppAttemptId", "sprkAppName", "normSprkAppName", "jobId", "stageId","stageAttemptId","user", "queue"})
 @Partition({"site"})
-public class SparkStage extends TaggedLogAPIEntity{
+public class SparkStage extends TaggedLogAPIEntity {
 
     @Column("a")
     private String status;
     @Column("b")
-    private int numActiveTasks=0;
+    private int numActiveTasks = 0;
     @Column("c")
-    private int numCompletedTasks=0;
+    private int numCompletedTasks = 0;
     @Column("d")
-    private int numFailedTasks=0;
+    private int numFailedTasks = 0;
     @Column("e")
-    private long executorRunTime=0l;
+    private long executorRunTime = 0L;
     @Column("f")
-    private long inputBytes=0l;
+    private long inputBytes = 0L;
     @Column("g")
-    private long inputRecords=0l;
+    private long inputRecords = 0L;
     @Column("h")
-    private long outputBytes=0l;
+    private long outputBytes = 0L;
     @Column("i")
-    private long outputRecords=0l;
+    private long outputRecords = 0L;
     @Column("j")
-    private long shuffleReadBytes=0l;
+    private long shuffleReadBytes = 0L;
     @Column("k")
-    private long shuffleReadRecords=0l;
+    private long shuffleReadRecords = 0L;
     @Column("l")
-    private long shuffleWriteBytes=0l;
+    private long shuffleWriteBytes = 0L;
     @Column("m")
-    private long shuffleWriteRecords=0l;
+    private long shuffleWriteRecords = 0L;
     @Column("n")
-    private long memoryBytesSpilled=0l;
+    private long memoryBytesSpilled = 0L;
     @Column("o")
-    private long diskBytesSpilled=0l;
+    private long diskBytesSpilled = 0L;
     @Column("p")
     private String name;
     @Column("q")

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/0b852cbc/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/spark/entity/SparkTask.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/spark/entity/SparkTask.java b/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/spark/entity/SparkTask.java
index 6ef7c69..fb2fce5 100644
--- a/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/spark/entity/SparkTask.java
+++ b/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/spark/entity/SparkTask.java
@@ -18,10 +18,10 @@
 
 package org.apache.eagle.jpm.spark.entity;
 
+import org.apache.eagle.jpm.util.Constants;
 import org.apache.eagle.log.base.taggedlog.TaggedLogAPIEntity;
 import org.apache.eagle.log.entity.meta.*;
 import org.codehaus.jackson.annotate.JsonIgnoreProperties;
-import org.apache.eagle.jpm.util.Constants;
 
 @Table("eglesprk_tasks")
 @ColumnFamily("f")
@@ -31,7 +31,7 @@ import org.apache.eagle.jpm.util.Constants;
 @TimeSeries(true)
 @Tags({"site","sprkAppId", "sprkAppAttemptId", "sprkAppName", "normSprkAppName", "jobId", "jobName", "stageId","stageAttemptId","taskIndex","taskAttemptId","user", "queue"})
 @Partition({"site"})
-public class SparkTask extends TaggedLogAPIEntity{
+public class SparkTask extends TaggedLogAPIEntity {
 
     @Column("a")
     private int taskId;

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/0b852cbc/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/crawler/AbstractJobHistoryDAO.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/crawler/AbstractJobHistoryDAO.java b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/crawler/AbstractJobHistoryDAO.java
index 5b330fc..74489cd 100644
--- a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/crawler/AbstractJobHistoryDAO.java
+++ b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/crawler/AbstractJobHistoryDAO.java
@@ -30,48 +30,49 @@ import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 
 /**
- * job history is the resource
+ * job history is the resource.
  */
 public abstract class AbstractJobHistoryDAO implements JobHistoryLCM {
     private static final Logger LOG = LoggerFactory.getLogger(AbstractJobHistoryDAO.class);
 
-    private final static String YEAR_URL_FORMAT = "/%4d";
-    private final static String MONTH_URL_FORMAT = "/%02d";
-    private final static String DAY_URL_FORMAT = "/%02d";
-    private final static String YEAR_MONTH_DAY_URL_FORMAT = YEAR_URL_FORMAT + MONTH_URL_FORMAT + DAY_URL_FORMAT;
-    protected final static String SERIAL_URL_FORMAT = "/%06d";
-    protected final static String FILE_URL_FORMAT = "/%s";
+    private static final String YEAR_URL_FORMAT = "/%4d";
+    private static final String MONTH_URL_FORMAT = "/%02d";
+    private static final String DAY_URL_FORMAT = "/%02d";
+    private static final String YEAR_MONTH_DAY_URL_FORMAT = YEAR_URL_FORMAT + MONTH_URL_FORMAT + DAY_URL_FORMAT;
+    protected static final String SERIAL_URL_FORMAT = "/%06d";
+    protected static final String FILE_URL_FORMAT = "/%s";
     private static final Pattern JOBTRACKERNAME_PATTERN = Pattern.compile("^.*_(\\d+)_$");
     protected static final Pattern JOBID_PATTERN = Pattern.compile("job_\\d+_\\d+");
 
-    protected final String m_basePath;
-    protected volatile String m_jobTrackerName;
+    protected final String basePath;
+    protected volatile String jobTrackerName;
 
     public  static final String JOB_CONF_POSTFIX = "_conf.xml";
 
-    private final static Timer timer = new Timer(true);
-    private final static long JOB_TRACKER_SYNC_DURATION = 10 * 60 * 1000; // 10 minutes
+    private static final Timer timer = new Timer(true);
+    private static final long JOB_TRACKER_SYNC_DURATION = 10 * 60 * 1000; // 10 minutes
 
-    private boolean m_pathContainsJobTrackerName;
+    private boolean pathContainsJobTrackerName;
 
     public AbstractJobHistoryDAO(String basePath, boolean pathContainsJobTrackerName, String startingJobTrackerName) throws Exception {
-        m_basePath = basePath;
-        m_pathContainsJobTrackerName = pathContainsJobTrackerName;
-        m_jobTrackerName = startingJobTrackerName;
-        if (m_pathContainsJobTrackerName) {
-            if (startingJobTrackerName == null || startingJobTrackerName.isEmpty())
+        this.basePath = basePath;
+        this.pathContainsJobTrackerName = pathContainsJobTrackerName;
+        jobTrackerName = startingJobTrackerName;
+        if (this.pathContainsJobTrackerName) {
+            if (startingJobTrackerName == null || startingJobTrackerName.isEmpty()) {
                 throw new IllegalStateException("startingJobTrackerName should not be null or empty");
+            }
             // start background thread to check what is current job tracker
-            startThread(m_basePath);
+            startThread(this.basePath);
         }
     }
 
     protected String buildWholePathToYearMonthDay(int year, int month, int day) {
         StringBuilder sb = new StringBuilder();
-        sb.append(m_basePath);
-        if (!m_pathContainsJobTrackerName && m_jobTrackerName != null && !m_jobTrackerName.isEmpty()) {
+        sb.append(basePath);
+        if (!pathContainsJobTrackerName && jobTrackerName != null && !jobTrackerName.isEmpty()) {
             sb.append("/");
-            sb.append(m_jobTrackerName);
+            sb.append(jobTrackerName);
         }
         sb.append(String.format(YEAR_MONTH_DAY_URL_FORMAT, year, month, day));
         return sb.toString();
@@ -105,7 +106,7 @@ public abstract class AbstractJobHistoryDAO implements JobHistoryLCM {
             sb.append(JOB_CONF_POSTFIX);
             return sb.toString();
         }
-        LOG.warn("Illegal job history file name: "+jobHistFileName);
+        LOG.warn("Illegal job history file name: " + jobHistFileName);
         return null;
     }
 
@@ -118,11 +119,11 @@ public abstract class AbstractJobHistoryDAO implements JobHistoryLCM {
                 try {
                     LOG.info("regularly checking current jobTrackerName in background");
                     final String _jobTrackerName = calculateJobTrackerName(basePath);
-                    if (_jobTrackerName != null && !_jobTrackerName.equals(m_jobTrackerName)) {
-                        LOG.info("jobTrackerName changed from " + m_jobTrackerName +" to " + _jobTrackerName);
-                        m_jobTrackerName = _jobTrackerName;
+                    if (_jobTrackerName != null && !_jobTrackerName.equals(jobTrackerName)) {
+                        LOG.info("jobTrackerName changed from " + jobTrackerName + " to " + _jobTrackerName);
+                        jobTrackerName = _jobTrackerName;
                     }
-                    LOG.info("Current jobTrackerName is: " + m_jobTrackerName);
+                    LOG.info("Current jobTrackerName is: " + jobTrackerName);
                 } catch (Exception e) {
                     LOG.error("failed to figure out current job tracker name that is not configured due to: " + e.getMessage(), e);
                 } catch (Throwable t) {
@@ -139,7 +140,7 @@ public abstract class AbstractJobHistoryDAO implements JobHistoryLCM {
         try {
             downloadIs = getJHFFileContentAsStream(year, month, day, serialNumber, jobHistoryFileName);
         } catch (FileNotFoundException ex) {
-            LOG.error("job history file not found " + jobHistoryFileName+", ignore and will NOT process any more");
+            LOG.error("job history file not found " + jobHistoryFileName + ", ignore and will NOT process any more");
             return;
         }
 
@@ -147,7 +148,7 @@ public abstract class AbstractJobHistoryDAO implements JobHistoryLCM {
         try {
             downloadJobConfIs = getJHFConfContentAsStream(year, month, day, serialNumber, jobHistoryFileName);
         } catch (FileNotFoundException ex) {
-            LOG.warn("job configuration file of "+ jobHistoryFileName+" not found , ignore and use empty configuration");
+            LOG.warn("job configuration file of " + jobHistoryFileName + " not found , ignore and use empty configuration");
         }
 
         org.apache.hadoop.conf.Configuration conf = null;
@@ -164,12 +165,12 @@ public abstract class AbstractJobHistoryDAO implements JobHistoryLCM {
         } catch (Exception ex) {
             LOG.error("fail reading job history file", ex);
             throw ex;
-        } catch(Throwable t) {
+        } catch (Throwable t) {
             LOG.error("fail reading job history file", t);
             throw new Exception(t);
         } finally {
             try {
-                if(downloadJobConfIs != null) {
+                if (downloadJobConfIs != null) {
                     downloadJobConfIs.close();
                 }
                 if (downloadIs != null) {

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/0b852cbc/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/crawler/DefaultJHFInputStreamCallback.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/crawler/DefaultJHFInputStreamCallback.java b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/crawler/DefaultJHFInputStreamCallback.java
index aeb35fd..87cd4e0 100644
--- a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/crawler/DefaultJHFInputStreamCallback.java
+++ b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/crawler/DefaultJHFInputStreamCallback.java
@@ -32,30 +32,32 @@ public class DefaultJHFInputStreamCallback implements JHFInputStreamCallback {
     private static final Logger LOG = LoggerFactory.getLogger(DefaultJHFInputStreamCallback.class);
 
 
-    private JobHistoryContentFilter m_filter;
-    private MRHistoryJobConfig m_configManager;
+    private JobHistoryContentFilter filter;
+    private MRHistoryJobConfig configManager;
 
     public DefaultJHFInputStreamCallback(JobHistoryContentFilter filter, MRHistoryJobConfig configManager, EagleOutputCollector eagleCollector) {
-        this.m_filter = filter;
-        this.m_configManager = configManager;
+        this.filter = filter;
+        this.configManager = configManager;
     }
 
     @Override
     public void onInputStream(InputStream jobFileInputStream, org.apache.hadoop.conf.Configuration conf) throws Exception {
-        final MRHistoryJobConfig.JobExtractorConfig jobExtractorConfig = m_configManager.getJobExtractorConfig();
+        final MRHistoryJobConfig.JobExtractorConfig jobExtractorConfig = configManager.getJobExtractorConfig();
         @SuppressWarnings("serial")
-        Map<String, String> baseTags = new HashMap<String, String>() { {
-            put("site", jobExtractorConfig.site);
-        } };
+        Map<String, String> baseTags = new HashMap<String, String>() {
+            {
+                put("site", jobExtractorConfig.site);
+            }
+        };
 
-        if (!m_filter.acceptJobFile()) {
+        if (!filter.acceptJobFile()) {
             // close immediately if we don't need job file
             jobFileInputStream.close();
         } else {
             //get parser and parse, do not need to emit data now
-            JHFParserBase parser = JHFParserFactory.getParser(m_configManager,
+            JHFParserBase parser = JHFParserFactory.getParser(configManager,
                     baseTags,
-                    conf, m_filter);
+                    conf, filter);
             parser.parse(jobFileInputStream);
             jobFileInputStream.close();
         }

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/0b852cbc/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/crawler/EagleOutputCollector.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/crawler/EagleOutputCollector.java b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/crawler/EagleOutputCollector.java
index 693e876..70eab38 100644
--- a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/crawler/EagleOutputCollector.java
+++ b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/crawler/EagleOutputCollector.java
@@ -21,9 +21,9 @@ import org.apache.eagle.dataproc.impl.storm.ValuesArray;
 import java.io.Serializable;
 
 /**
- * expose simple interface for streaming executor to populate output data
+ * expose simple interface for streaming executor to populate output data.
  *
  */
-public interface EagleOutputCollector extends Serializable{
-	void collect(ValuesArray t);
+public interface EagleOutputCollector extends Serializable {
+    void collect(ValuesArray t);
 }

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/0b852cbc/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/crawler/JHFCrawlerDriver.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/crawler/JHFCrawlerDriver.java b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/crawler/JHFCrawlerDriver.java
index 3edde5b..69eb94a 100644
--- a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/crawler/JHFCrawlerDriver.java
+++ b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/crawler/JHFCrawlerDriver.java
@@ -20,8 +20,8 @@ package org.apache.eagle.jpm.mr.history.crawler;
 
 public interface JHFCrawlerDriver {
     /**
-     * return -1 if failed or there is no file to crawl
-     * return modified time of the file if succeed
+     * return -1 if failed or there is no file to crawl.
+     * return modified time of the file if succeed.
      */
     long crawl() throws Exception;
 }


[30/52] [abbrv] incubator-eagle git commit: [EAGLE-498] overwrite equals method for PolicyDefinition should not con…

Posted by yo...@apache.org.
[EAGLE-498] overwrite equals method for PolicyDefinition should not con\u2026

EAGLE-498 overwrite equals method for PolicyDefinition should not contain field description

- Delete the equals for description in method equals.
- Delete append description in method hashCode.

https://issues.apache.org/jira/browse/EAGLE-498

Author: chitin <ch...@gmail.com>

Closes #384 from chitin/EAGLE498.


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

Branch: refs/heads/master
Commit: e77877518ba0cba6cbd3c20ccd0d040c5a3cd86b
Parents: c940f56
Author: chitin <ch...@gmail.com>
Authored: Thu Sep 1 11:31:19 2016 +0800
Committer: Hao Chen <ha...@apache.org>
Committed: Thu Sep 1 11:31:19 2016 +0800

----------------------------------------------------------------------
 .../apache/eagle/alert/engine/coordinator/PolicyDefinition.java    | 2 --
 1 file changed, 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/e7787751/eagle-core/eagle-alert-parent/eagle-alert/alert-common/src/main/java/org/apache/eagle/alert/engine/coordinator/PolicyDefinition.java
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-alert-parent/eagle-alert/alert-common/src/main/java/org/apache/eagle/alert/engine/coordinator/PolicyDefinition.java b/eagle-core/eagle-alert-parent/eagle-alert/alert-common/src/main/java/org/apache/eagle/alert/engine/coordinator/PolicyDefinition.java
index e2dfb6f..0dca247 100644
--- a/eagle-core/eagle-alert-parent/eagle-alert/alert-common/src/main/java/org/apache/eagle/alert/engine/coordinator/PolicyDefinition.java
+++ b/eagle-core/eagle-alert-parent/eagle-alert/alert-common/src/main/java/org/apache/eagle/alert/engine/coordinator/PolicyDefinition.java
@@ -129,7 +129,6 @@ public class PolicyDefinition implements Serializable{
     public int hashCode() {
         return new HashCodeBuilder().
                 append(name).
-                append(description).
                 append(inputStreams).
                 append(outputStreams).
                 append(definition).
@@ -146,7 +145,6 @@ public class PolicyDefinition implements Serializable{
             return false;
         PolicyDefinition another = (PolicyDefinition)that;
         if(another.name.equals(this.name) &&
-                another.description.equals(this.description) &&
                 CollectionUtils.isEqualCollection(another.inputStreams, this.inputStreams) &&
                 CollectionUtils.isEqualCollection(another.outputStreams, this.outputStreams) &&
                 another.definition.equals(this.definition) &&


[12/52] [abbrv] incubator-eagle git commit: EAGLE-492 Fix negative total execution time and make some code optimization for spark history job. 1) negative total execution time It is because the app was killed or terminated unexpectedly without completion

Posted by yo...@apache.org.
EAGLE-492 Fix negative total execution time and make some code optimization for spark history job.
1) negative total execution time
It is because the app was killed or terminated unexpectedly without completion. In the log file, it has only SparkListenerBlockManagerAdded timestamp. "Timestamp":1470768362299
And the code initiated executor starttime with this timestamp.
No executor or app endtime available, so executor endtime = 0L.
Finally, totalExecutionTime = -1470768362299
The code logic does not consider the situation when executor or app Endtime = 0L.
Solution: If totalExecutionTime < 0L, set it = 0L. Stating the app is killed and not completed.
2) Url builder bug for completed jobs

https://issues.apache.org/jira/browse/EAGLE-492

Author: @pkuwm <ih...@gmail.com>
Reviewer: @DadanielZ <da...@apache.org>

Closes #375


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

Branch: refs/heads/master
Commit: 6f5f972c966ea1edc2adf0573dbdb1b0d5ff9671
Parents: 24e6622
Author: DadanielZ <da...@apache.org>
Authored: Wed Aug 24 17:19:22 2016 -0700
Committer: DadanielZ <da...@apache.org>
Committed: Wed Aug 24 17:19:22 2016 -0700

----------------------------------------------------------------------
 .../jpm/spark/crawl/JHFSparkEventReader.java    | 174 +++++++++----------
 .../eagle/jpm/spark/crawl/JHFSparkParser.java   |  45 +++--
 .../history/storm/FinishedSparkJobSpout.java    |  12 +-
 .../spark/history/storm/SparkJobParseBolt.java  |  36 ++--
 .../java/org/apache/eagle/jpm/util/Utils.java   |   5 +-
 .../util/resourceFetch/RMResourceFetcher.java   |  41 ++---
 .../resourceFetch/ha/HAURLSelectorImpl.java     |   4 +-
 .../SparkCompleteJobServiceURLBuilderImpl.java  |  11 +-
 8 files changed, 163 insertions(+), 165 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/6f5f972c/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/spark/crawl/JHFSparkEventReader.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/spark/crawl/JHFSparkEventReader.java b/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/spark/crawl/JHFSparkEventReader.java
index e298fa3..a5e630a 100644
--- a/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/spark/crawl/JHFSparkEventReader.java
+++ b/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/spark/crawl/JHFSparkEventReader.java
@@ -18,10 +18,7 @@
 package org.apache.eagle.jpm.spark.crawl;
 
 import org.apache.eagle.jpm.spark.entity.*;
-import org.apache.eagle.jpm.util.JSONUtil;
-import org.apache.eagle.jpm.util.JobNameNormalization;
-import org.apache.eagle.jpm.util.SparkEntityConstant;
-import org.apache.eagle.jpm.util.SparkJobTagName;
+import org.apache.eagle.jpm.util.*;
 import org.apache.eagle.log.base.taggedlog.TaggedLogAPIEntity;
 import org.apache.eagle.service.client.impl.EagleServiceBaseClient;
 import org.apache.eagle.service.client.impl.EagleServiceClientImpl;
@@ -37,8 +34,9 @@ import java.util.*;
 public class JHFSparkEventReader {
     private static final Logger LOG = LoggerFactory.getLogger(JHFSparkEventReader.class);
 
-    public static final int FLUSH_LIMIT = 500;
+    private static final int FLUSH_LIMIT = 500;
     private long firstTaskLaunchTime;
+    private long lastEventTime;
 
     private Map<String, SparkExecutor> executors;
     private SparkApp app;
@@ -75,7 +73,6 @@ public class JHFSparkEventReader {
 
     public void read(JSONObject eventObj) throws Exception {
         String eventType = (String) eventObj.get("Event");
-        LOG.info("Event type: " + eventType);
         if (eventType.equalsIgnoreCase(EventType.SparkListenerApplicationStart.toString())) {
             handleAppStarted(eventObj);
         } else if (eventType.equalsIgnoreCase(EventType.SparkListenerEnvironmentUpdate.toString())) {
@@ -108,7 +105,6 @@ public class JHFSparkEventReader {
 
     }
 
-
     private void handleEnvironmentSet(JSONObject event) {
         app.setConfig(new JobConfig());
         JSONObject sparkProps = (JSONObject) event.get("Spark Properties");
@@ -142,16 +138,10 @@ public class JHFSparkEventReader {
         }
     }
 
-
     private boolean isClientMode(JobConfig config) {
-        if (config.getConfig().get("spark.master").equalsIgnoreCase("yarn-client")) {
-            return true;
-        } else {
-            return false;
-        }
+        return config.getConfig().get("spark.master").equalsIgnoreCase("yarn-client");
     }
 
-
     private void handleAppStarted(JSONObject event) {
         //need update all entities tag before app start
         List<TaggedLogAPIEntity> entities = new ArrayList<TaggedLogAPIEntity>();
@@ -174,11 +164,14 @@ public class JHFSparkEventReader {
         }
 
         this.app.setStartTime(appStartTime);
+        this.lastEventTime = appStartTime;
     }
 
     private void handleExecutorAdd(JSONObject event) throws Exception {
         String executorID = (String) event.get("Executor ID");
-        SparkExecutor executor = this.initiateExecutor(executorID, JSONUtil.getLong(event, "Timestamp"));
+        long executorAddTime = JSONUtil.getLong(event, "Timestamp");
+        this.lastEventTime = executorAddTime;
+        SparkExecutor executor = this.initiateExecutor(executorID, executorAddTime);
 
         JSONObject executorInfo = JSONUtil.getJSONObject(event, "Executor Info");
 
@@ -187,6 +180,7 @@ public class JHFSparkEventReader {
     private void handleBlockManagerAdd(JSONObject event) throws Exception {
         long maxMemory = JSONUtil.getLong(event, "Maximum Memory");
         long timestamp = JSONUtil.getLong(event, "Timestamp");
+        this.lastEventTime = timestamp;
         JSONObject blockInfo = JSONUtil.getJSONObject(event, "Block Manager ID");
         String executorID = JSONUtil.getString(blockInfo, "Executor ID");
         String hostport = String.format("%s:%s", JSONUtil.getString(blockInfo, "Host"), JSONUtil.getLong(blockInfo, "Port"));
@@ -202,11 +196,9 @@ public class JHFSparkEventReader {
 
     private void handleTaskEnd(JSONObject event) {
         JSONObject taskInfo = JSONUtil.getJSONObject(event, "Task Info");
-        Integer taskId = JSONUtil.getInt(taskInfo, "Task ID");
-        SparkTask task = null;
-        if (tasks.containsKey(taskId)) {
-            task = tasks.get(taskId);
-        } else {
+        int taskId = JSONUtil.getInt(taskInfo, "Task ID");
+        SparkTask task = tasks.get(taskId);
+        if (task == null) {
             return;
         }
 
@@ -261,19 +253,20 @@ public class JHFSparkEventReader {
 
     private SparkTask initializeTask(JSONObject event) {
         SparkTask task = new SparkTask();
-        task.setTags(new HashMap(this.app.getTags()));
+        task.setTags(new HashMap<>(this.app.getTags()));
         task.setTimestamp(app.getTimestamp());
 
-        task.getTags().put(SparkJobTagName.SPARK_SATGE_ID.toString(), JSONUtil.getLong(event, "Stage ID").toString());
-        task.getTags().put(SparkJobTagName.SPARK_STAGE_ATTEMPT_ID.toString(), JSONUtil.getLong(event, "Stage Attempt ID").toString());
+        task.getTags().put(SparkJobTagName.SPARK_SATGE_ID.toString(), Long.toString(JSONUtil.getLong(event, "Stage ID")));
+        task.getTags().put(SparkJobTagName.SPARK_STAGE_ATTEMPT_ID.toString(), Long.toString(JSONUtil.getLong(event, "Stage Attempt ID")));
 
         JSONObject taskInfo = JSONUtil.getJSONObject(event, "Task Info");
         int taskId = JSONUtil.getInt(taskInfo, "Task ID");
         task.setTaskId(taskId);
 
-        task.getTags().put(SparkJobTagName.SPARK_TASK_INDEX.toString(), JSONUtil.getInt(taskInfo, "Index").toString());
-        task.getTags().put(SparkJobTagName.SPARK_TASK_ATTEMPT_ID.toString(), JSONUtil.getInt(taskInfo, "Attempt").toString());
+        task.getTags().put(SparkJobTagName.SPARK_TASK_INDEX.toString(), Integer.toString(JSONUtil.getInt(taskInfo, "Index")));
+        task.getTags().put(SparkJobTagName.SPARK_TASK_ATTEMPT_ID.toString(), Integer.toString(JSONUtil.getInt(taskInfo, "Attempt")));
         long launchTime = JSONUtil.getLong(taskInfo, "Launch Time");
+        this.lastEventTime = launchTime;
         if (taskId == 0) {
             this.setFirstTaskLaunchTime(launchTime);
         }
@@ -295,15 +288,16 @@ public class JHFSparkEventReader {
         return this.firstTaskLaunchTime;
     }
 
-
     private void handleJobStart(JSONObject event) {
         SparkJob job = new SparkJob();
-        job.setTags(new HashMap(this.app.getTags()));
+        job.setTags(new HashMap<>(this.app.getTags()));
         job.setTimestamp(app.getTimestamp());
 
-        Integer jobId = JSONUtil.getInt(event, "Job ID");
-        job.getTags().put(SparkJobTagName.SPARK_JOB_ID.toString(), jobId.toString());
-        job.setSubmissionTime(JSONUtil.getLong(event, "Submission Time"));
+        int jobId = JSONUtil.getInt(event, "Job ID");
+        job.getTags().put(SparkJobTagName.SPARK_JOB_ID.toString(), Integer.toString(jobId));
+        long submissionTime = JSONUtil.getLong(event, "Submission Time");
+        job.setSubmissionTime(submissionTime);
+        this.lastEventTime = submissionTime;
 
         //for complete application, no active stages/tasks
         job.setNumActiveStages(0);
@@ -316,8 +310,8 @@ public class JHFSparkEventReader {
         job.setNumStages(stages.size());
         for (int i = 0; i < stages.size(); i++) {
             JSONObject stageInfo = (JSONObject) stages.get(i);
-            Integer stageId = JSONUtil.getInt(stageInfo, "Stage ID");
-            Integer stageAttemptId = JSONUtil.getInt(stageInfo, "Stage Attempt ID");
+            int stageId = JSONUtil.getInt(stageInfo, "Stage ID");
+            int stageAttemptId = JSONUtil.getInt(stageInfo, "Stage Attempt ID");
             String stageName = JSONUtil.getString(stageInfo, "Stage Name");
             int numTasks = JSONUtil.getInt(stageInfo, "Number of Tasks");
             this.initiateStage(jobId, stageId, stageAttemptId, stageName, numTasks);
@@ -326,14 +320,14 @@ public class JHFSparkEventReader {
 
     private void handleStageSubmit(JSONObject event) {
         JSONObject stageInfo = JSONUtil.getJSONObject(event, "Stage Info");
-        Integer stageId = JSONUtil.getInt(stageInfo, "Stage ID");
-        Integer stageAttemptId = JSONUtil.getInt(stageInfo, "Stage Attempt ID");
-        String key = this.generateStageKey(stageId.toString(), stageAttemptId.toString());
+        int stageId = JSONUtil.getInt(stageInfo, "Stage ID");
+        int stageAttemptId = JSONUtil.getInt(stageInfo, "Stage Attempt ID");
+        String key = this.generateStageKey(Integer.toString(stageId), Integer.toString(stageAttemptId));
         stageTaskStatusMap.put(key, new HashMap<Integer, Boolean>());
 
-        if (!stages.containsKey(this.generateStageKey(stageId.toString(), stageAttemptId.toString()))) {
+        if (!stages.containsKey(this.generateStageKey(Integer.toString(stageId), Integer.toString(stageAttemptId)))) {
             //may be further attempt for one stage
-            String baseAttempt = this.generateStageKey(stageId.toString(), "0");
+            String baseAttempt = this.generateStageKey(Integer.toString(stageId), "0");
             if (stages.containsKey(baseAttempt)) {
                 SparkStage stage = stages.get(baseAttempt);
                 String jobId = stage.getTags().get(SparkJobTagName.SPARK_JOB_ID.toString());
@@ -343,14 +337,13 @@ public class JHFSparkEventReader {
                 this.initiateStage(Integer.parseInt(jobId), stageId, stageAttemptId, stageName, numTasks);
             }
         }
-
     }
 
     private void handleStageComplete(JSONObject event) {
         JSONObject stageInfo = JSONUtil.getJSONObject(event, "Stage Info");
-        Integer stageId = JSONUtil.getInt(stageInfo, "Stage ID");
-        Integer stageAttemptId = JSONUtil.getInt(stageInfo, "Stage Attempt ID");
-        String key = this.generateStageKey(stageId.toString(), stageAttemptId.toString());
+        int stageId = JSONUtil.getInt(stageInfo, "Stage ID");
+        int stageAttemptId = JSONUtil.getInt(stageInfo, "Stage Attempt ID");
+        String key = this.generateStageKey(Integer.toString(stageId), Integer.toString(stageAttemptId));
         SparkStage stage = stages.get(key);
 
         // If "Submission Time" is not available, use the "Launch Time" of "Task ID" = 0.
@@ -359,7 +352,10 @@ public class JHFSparkEventReader {
             submissionTime = this.getFirstTaskLaunchTime();
         }
         stage.setSubmitTime(submissionTime);
-        stage.setCompleteTime(JSONUtil.getLong(stageInfo, "Completion Time"));
+
+        long completeTime = JSONUtil.getLong(stageInfo, "Completion Time");
+        stage.setCompleteTime(completeTime);
+        this.lastEventTime = completeTime;
 
         if (stageInfo.containsKey("Failure Reason")) {
             stage.setStatus(SparkEntityConstant.SparkStageStatus.FAILED.toString());
@@ -371,14 +367,19 @@ public class JHFSparkEventReader {
     private void handleExecutorRemoved(JSONObject event) {
         String executorID = JSONUtil.getString(event, "Executor ID");
         SparkExecutor executor = executors.get(executorID);
-        executor.setEndTime(JSONUtil.getLong(event, "Timestamp"));
-
+        long removedTime = JSONUtil.getLong(event, "Timestamp");
+        executor.setEndTime(removedTime);
+        this.lastEventTime = removedTime;
     }
 
     private void handleJobEnd(JSONObject event) {
-        Integer jobId = JSONUtil.getInt(event, "Job ID");
+        int jobId = JSONUtil.getInt(event, "Job ID");
         SparkJob job = jobs.get(jobId);
-        job.setCompletionTime(JSONUtil.getLong(event, "Completion Time"));
+
+        long completionTime = JSONUtil.getLong(event, "Completion Time");
+        job.setCompletionTime(completionTime);
+        this.lastEventTime = completionTime;
+
         JSONObject jobResult = JSONUtil.getJSONObject(event, "Job Result");
         String result = JSONUtil.getString(jobResult, "Result");
         if (result.equalsIgnoreCase("JobSucceeded")) {
@@ -391,6 +392,7 @@ public class JHFSparkEventReader {
     private void handleAppEnd(JSONObject event) {
         long endTime = JSONUtil.getLong(event, "Timestamp");
         app.setEndTime(endTime);
+        this.lastEventTime = endTime;
     }
 
     public void clearReader() throws Exception {
@@ -405,7 +407,7 @@ public class JHFSparkEventReader {
 
         List<SparkStage> needStoreStages = new ArrayList<>();
         for (SparkStage stage : this.stages.values()) {
-            Integer jobId = Integer.parseInt(stage.getTags().get(SparkJobTagName.SPARK_JOB_ID.toString()));
+            int jobId = Integer.parseInt(stage.getTags().get(SparkJobTagName.SPARK_JOB_ID.toString()));
             if (stage.getSubmitTime() == 0 || stage.getCompleteTime() == 0) {
                 SparkJob job = this.jobs.get(jobId);
                 job.setNumSkippedStages(job.getNumSkippedStages() + 1);
@@ -427,8 +429,9 @@ public class JHFSparkEventReader {
         this.flushEntities(jobs.values(), false);
 
         app.setExecutors(executors.values().size());
-        long executorMemory = parseExecutorMemory((String) this.getConfigVal(this.app.getConfig(), "spark.executor.memory", String.class.getName()));
-        long driverMemory = parseExecutorMemory(this.isClientMode(app.getConfig())
+
+        long executorMemory = Utils.parseMemory((String) this.getConfigVal(this.app.getConfig(), "spark.executor.memory", String.class.getName()));
+        long driverMemory = Utils.parseMemory(this.isClientMode(app.getConfig())
             ? (String) this.getConfigVal(this.app.getConfig(), "spark.yarn.am.memory", String.class.getName())
             : (String) this.getConfigVal(app.getConfig(), "spark.driver.memory", String.class.getName()));
 
@@ -460,7 +463,10 @@ public class JHFSparkEventReader {
                 executor.setCores(executorCore);
                 executor.setMemoryOverhead(executorMemoryOverhead);
             }
-            if (executor.getEndTime() == 0) {
+            if (app.getEndTime() <= 0L) {
+                app.setEndTime(this.lastEventTime);
+            }
+            if (executor.getEndTime() <= 0L) {
                 executor.setEndTime(app.getEndTime());
             }
             this.aggregateExecutorToApp(executor);
@@ -473,21 +479,28 @@ public class JHFSparkEventReader {
 
     private long getMemoryOverhead(JobConfig config, long executorMemory, String fieldName) {
         long result = 0L;
-        if (config.getConfig().containsKey(fieldName)) {
-            result = this.parseExecutorMemory(config.getConfig().get(fieldName) + "m");
+        String fieldValue = config.getConfig().get(fieldName);
+        if (fieldValue != null) {
+            result = Utils.parseMemory(fieldValue + "m");
             if (result == 0L) {
-                result = this.parseExecutorMemory(config.getConfig().get(fieldName));
+               result = Utils.parseMemory(fieldValue);
             }
         }
 
         if (result == 0L) {
-            result = Math.max(this.parseExecutorMemory(conf.getString("spark.defaultVal.spark.yarn.overhead.min")), executorMemory * conf.getInt("spark.defaultVal." + fieldName + ".factor") / 100);
+            result = Math.max(
+                    Utils.parseMemory(conf.getString("spark.defaultVal.spark.yarn.overhead.min")),
+                    executorMemory * conf.getInt("spark.defaultVal." + fieldName + ".factor") / 100);
         }
         return result;
     }
 
     private void aggregateExecutorToApp(SparkExecutor executor) {
-        app.setTotalExecutorTime(app.getTotalExecutorTime() + (executor.getEndTime() - executor.getStartTime()));
+        long totalExecutorTime = app.getTotalExecutorTime() + executor.getEndTime() - executor.getStartTime();
+        if (totalExecutorTime < 0L) {
+            totalExecutorTime = 0L;
+        }
+        app.setTotalExecutorTime(totalExecutorTime);
     }
 
     private void aggregateJobToApp(SparkJob job) {
@@ -589,7 +602,7 @@ public class JHFSparkEventReader {
     }
 
     private void aggregateToJob(SparkStage stage) {
-        Integer jobId = Integer.parseInt(stage.getTags().get(SparkJobTagName.SPARK_JOB_ID.toString()));
+        int jobId = Integer.parseInt(stage.getTags().get(SparkJobTagName.SPARK_JOB_ID.toString()));
         SparkJob job = jobs.get(jobId);
         job.setNumCompletedTasks(job.getNumCompletedTasks() + stage.getNumCompletedTasks());
         job.setNumFailedTasks(job.getNumFailedTasks() + stage.getNumFailedTasks());
@@ -601,16 +614,16 @@ public class JHFSparkEventReader {
             if (!hasStagePriorAttemptSuccess(stage)) {
                 job.setNumCompletedStages(job.getNumCompletedStages() + 1);
             }
-
         } else {
             job.setNumFailedStages(job.getNumFailedStages() + 1);
         }
     }
 
     private boolean hasStagePriorAttemptSuccess(SparkStage stage) {
-        Integer stageAttemptId = Integer.parseInt(stage.getTags().get(SparkJobTagName.SPARK_STAGE_ATTEMPT_ID.toString()));
-        for (Integer i = 0; i < stageAttemptId; i++) {
-            SparkStage previousStage = stages.get(this.generateStageKey(stage.getTags().get(SparkJobTagName.SPARK_SATGE_ID.toString()), i.toString()));
+        int stageAttemptId = Integer.parseInt(stage.getTags().get(SparkJobTagName.SPARK_STAGE_ATTEMPT_ID.toString()));
+        for (int i = 0; i < stageAttemptId; i++) {
+            SparkStage previousStage = stages.get(this.generateStageKey(
+                    stage.getTags().get(SparkJobTagName.SPARK_SATGE_ID.toString()), Integer.toString(i)));
             if (previousStage.getStatus().equalsIgnoreCase(SparkEntityConstant.SparkStageStatus.COMPLETE.toString())) {
                 return true;
             }
@@ -623,19 +636,20 @@ public class JHFSparkEventReader {
         return String.format("%s-%s", stageId, stageAttemptId);
     }
 
-    private void initiateStage(Integer jobId, Integer stageId, Integer stageAttemptId, String name, int numTasks) {
+    private void initiateStage(int jobId, int stageId, int stageAttemptId, String name, int numTasks) {
         SparkStage stage = new SparkStage();
-        stage.setTags(new HashMap(this.app.getTags()));
+        stage.setTags(new HashMap<>(this.app.getTags()));
         stage.setTimestamp(app.getTimestamp());
-        stage.getTags().put(SparkJobTagName.SPARK_JOB_ID.toString(), jobId.toString());
-        stage.getTags().put(SparkJobTagName.SPARK_SATGE_ID.toString(), stageId.toString());
-        stage.getTags().put(SparkJobTagName.SPARK_STAGE_ATTEMPT_ID.toString(), stageAttemptId.toString());
+        stage.getTags().put(SparkJobTagName.SPARK_JOB_ID.toString(), Integer.toString(jobId));
+        stage.getTags().put(SparkJobTagName.SPARK_SATGE_ID.toString(), Integer.toString(stageId));
+        stage.getTags().put(SparkJobTagName.SPARK_STAGE_ATTEMPT_ID.toString(), Integer.toString(stageAttemptId));
         stage.setName(name);
         stage.setNumActiveTasks(0);
         stage.setNumTasks(numTasks);
-        stage.setSchedulingPool(this.app.getConfig().getConfig().get("spark.scheduler.pool") == null ? "default" : this.app.getConfig().getConfig().get("spark.scheduler.pool"));
+        stage.setSchedulingPool(this.app.getConfig().getConfig().get("spark.scheduler.pool") == null ?
+                "default" : this.app.getConfig().getConfig().get("spark.scheduler.pool"));
 
-        String stageKey = this.generateStageKey(stageId.toString(), stageAttemptId.toString());
+        String stageKey = this.generateStageKey(Integer.toString(stageId), Integer.toString(stageAttemptId));
         stages.put(stageKey, stage);
         this.jobStageMap.get(jobId).add(stageKey);
     }
@@ -644,7 +658,7 @@ public class JHFSparkEventReader {
     private SparkExecutor initiateExecutor(String executorID, long startTime) throws Exception {
         if (!executors.containsKey(executorID)) {
             SparkExecutor executor = new SparkExecutor();
-            executor.setTags(new HashMap(this.app.getTags()));
+            executor.setTags(new HashMap<>(this.app.getTags()));
             executor.getTags().put(SparkJobTagName.SPARK_EXECUTOR_ID.toString(), executorID);
             executor.setStartTime(startTime);
             executor.setTimestamp(app.getTimestamp());
@@ -663,28 +677,6 @@ public class JHFSparkEventReader {
         }
     }
 
-    private long parseExecutorMemory(String memory) {
-
-        if (memory.endsWith("g") || memory.endsWith("G")) {
-            int executorGB = Integer.parseInt(memory.substring(0, memory.length() - 1));
-            return 1024L * 1024 * 1024 * executorGB;
-        } else if (memory.endsWith("m") || memory.endsWith("M")) {
-            int executorMB = Integer.parseInt(memory.substring(0, memory.length() - 1));
-            return 1024L * 1024 * executorMB;
-        } else if (memory.endsWith("k") || memory.endsWith("K")) {
-            int executorKB = Integer.parseInt(memory.substring(0, memory.length() - 1));
-            return 1024L * executorKB;
-        } else if (memory.endsWith("t") || memory.endsWith("T")) {
-            int executorTB = Integer.parseInt(memory.substring(0, memory.length() - 1));
-            return 1024L * 1024 * 1024 * 1024 * executorTB;
-        } else if (memory.endsWith("p") || memory.endsWith("P")) {
-            int executorPB = Integer.parseInt(memory.substring(0, memory.length() - 1));
-            return 1024L * 1024 * 1024 * 1024 * 1024 * executorPB;
-        }
-        LOG.info("Cannot parse memory info " + memory);
-        return 0L;
-    }
-
     private void flushEntities(Object entity, boolean forceFlush) {
         this.flushEntities(Arrays.asList(entity), forceFlush);
     }
@@ -719,4 +711,4 @@ public class JHFSparkEventReader {
         LOG.info("start flushing entities of total number " + entities.size());
         LOG.info("finish flushing entities of total number " + entities.size());
     }
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/6f5f972c/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/spark/crawl/JHFSparkParser.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/spark/crawl/JHFSparkParser.java b/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/spark/crawl/JHFSparkParser.java
index da049ea..05cdd7e 100644
--- a/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/spark/crawl/JHFSparkParser.java
+++ b/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/spark/crawl/JHFSparkParser.java
@@ -20,6 +20,7 @@ package org.apache.eagle.jpm.spark.crawl;
 
 import org.json.simple.JSONObject;
 import org.json.simple.parser.JSONParser;
+import org.json.simple.parser.ParseException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -31,7 +32,9 @@ public class JHFSparkParser implements JHFParserBase {
 
     private static final Logger logger = LoggerFactory.getLogger(JHFSparkParser.class);
 
-    JHFSparkEventReader eventReader;
+    private boolean isValidJson;
+
+    private JHFSparkEventReader eventReader;
 
     public JHFSparkParser(JHFSparkEventReader reader) {
         this.eventReader = reader;
@@ -39,26 +42,32 @@ public class JHFSparkParser implements JHFParserBase {
 
     @Override
     public void parse(InputStream is) throws Exception {
-        BufferedReader reader = new BufferedReader(new InputStreamReader(is));
-        try {
-            String line;
-
-            JSONParser parser = new JSONParser();
-            while ((line = reader.readLine()) != null) {
+        try (BufferedReader reader = new BufferedReader(new InputStreamReader(is))) {
+            for (String line = reader.readLine(); line != null; line = reader.readLine()) {
+                isValidJson = true;
                 try {
-                    JSONObject eventObj = (JSONObject) parser.parse(line);
-                    String eventType = (String) eventObj.get("Event");
-                    logger.info("Event type: " + eventType);
-                    this.eventReader.read(eventObj);
-                } catch (Exception e) {
-                    logger.error(String.format("Invalid json string. Fail to parse %s.", line), e);
+                    JSONObject eventObj = parseAndValidateJSON(line);
+                    if (isValidJson) {
+                        this.eventReader.read(eventObj);
+                    }
+                } catch(Exception e) {
+                    logger.error(String.format("Fail to parse %s.", line), e);
                 }
             }
+
             this.eventReader.clearReader();
-        } finally {
-            if (reader != null) {
-                reader.close();
-            }
         }
     }
-}
+
+    private JSONObject parseAndValidateJSON(String line) {
+        JSONObject eventObj = null;
+        JSONParser parser = new JSONParser();
+        try {
+            eventObj = (JSONObject) parser.parse(line);
+        } catch (ParseException ex) {
+            isValidJson = false;
+            logger.error(String.format("Invalid json string. Fail to parse %s.", line), ex);
+        }
+        return eventObj;
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/6f5f972c/eagle-jpm/eagle-jpm-spark-history/src/main/java/org/apache/eagle/jpm/spark/history/storm/FinishedSparkJobSpout.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-spark-history/src/main/java/org/apache/eagle/jpm/spark/history/storm/FinishedSparkJobSpout.java b/eagle-jpm/eagle-jpm-spark-history/src/main/java/org/apache/eagle/jpm/spark/history/storm/FinishedSparkJobSpout.java
index 8965d3d..bf04b55 100644
--- a/eagle-jpm/eagle-jpm-spark-history/src/main/java/org/apache/eagle/jpm/spark/history/storm/FinishedSparkJobSpout.java
+++ b/eagle-jpm/eagle-jpm-spark-history/src/main/java/org/apache/eagle/jpm/spark/history/storm/FinishedSparkJobSpout.java
@@ -76,11 +76,13 @@ public class FinishedSparkJobSpout extends BaseRichSpout {
             if (fetchTime - this.lastFinishAppTime > this.config.stormConfig.spoutCrawlInterval) {
                 List<AppInfo> appInfos = rmFetch.getResource(Constants.ResourceType.COMPLETE_SPARK_JOB, Long.toString(lastFinishAppTime));
                 //List<AppInfo> appInfos = (null != apps ? (List<AppInfo>)apps.get(0):new ArrayList<AppInfo>());
-                LOG.info("Get " + appInfos.size() + " from yarn resource manager.");
-                for (AppInfo app: appInfos) {
-                    String appId = app.getId();
-                    if (!zkState.hasApplication(appId)) {
-                        zkState.addFinishedApplication(appId, app.getQueue(), app.getState(), app.getFinalStatus(), app.getUser(), app.getName());
+                if (appInfos != null) {
+                    LOG.info("Get " + appInfos.size() + " from yarn resource manager.");
+                    for (AppInfo app : appInfos) {
+                        String appId = app.getId();
+                        if (!zkState.hasApplication(appId)) {
+                            zkState.addFinishedApplication(appId, app.getQueue(), app.getState(), app.getFinalStatus(), app.getUser(), app.getName());
+                        }
                     }
                 }
                 this.lastFinishAppTime = fetchTime;

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/6f5f972c/eagle-jpm/eagle-jpm-spark-history/src/main/java/org/apache/eagle/jpm/spark/history/storm/SparkJobParseBolt.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-spark-history/src/main/java/org/apache/eagle/jpm/spark/history/storm/SparkJobParseBolt.java b/eagle-jpm/eagle-jpm-spark-history/src/main/java/org/apache/eagle/jpm/spark/history/storm/SparkJobParseBolt.java
index f00fa1b..c515d32 100644
--- a/eagle-jpm/eagle-jpm-spark-history/src/main/java/org/apache/eagle/jpm/spark/history/storm/SparkJobParseBolt.java
+++ b/eagle-jpm/eagle-jpm-spark-history/src/main/java/org/apache/eagle/jpm/spark/history/storm/SparkJobParseBolt.java
@@ -60,7 +60,7 @@ public class SparkJobParseBolt extends BaseRichBolt {
     @Override
     public void prepare(Map map, TopologyContext topologyContext, OutputCollector outputCollector) {
         this.collector = outputCollector;
-        this.hdfsConf  = new Configuration();
+        this.hdfsConf = new Configuration();
         this.hdfsConf.set("fs.defaultFS", config.hdfsConfig.endpoint);
         this.hdfsConf.setBoolean("fs.hdfs.impl.disable.cache", true);
         this.hdfsConf.set("hdfs.kerberos.principal", config.hdfsConfig.principal);
@@ -73,17 +73,16 @@ public class SparkJobParseBolt extends BaseRichBolt {
     @Override
     public void execute(Tuple tuple) {
         String appId = tuple.getStringByField("appId");
-        FileSystem hdfs = null;
-        try {
-            if (!zkState.hasApplication(appId)) {
-                //may already be processed due to some reason
-                collector.ack(tuple);
-                return;
-            }
+        if (!zkState.hasApplication(appId)) {
+            //may already be processed due to some reason
+            collector.ack(tuple);
+            return;
+        }
 
+        try (FileSystem hdfs = HDFSUtil.getFileSystem(this.hdfsConf)) {
             SparkApplicationInfo info = zkState.getApplicationInfo(appId);
             //first try to get attempts under the application
-            hdfs = HDFSUtil.getFileSystem(this.hdfsConf);
+
             Set<String> inprogressSet = new HashSet<String>();
             List<String> attemptLogNames = this.getAttemptLogNameList(appId, hdfs, inprogressSet);
 
@@ -111,14 +110,6 @@ public class SparkJobParseBolt extends BaseRichBolt {
             LOG.error("Fail to process application {}", appId, e);
             zkState.updateApplicationStatus(appId, ZKStateConstant.AppStatus.FAILED);
             collector.fail(tuple);
-        } finally {
-            if (null != hdfs) {
-                try {
-                    hdfs.close();
-                } catch (Exception e) {
-                    LOG.error("Fail to close hdfs");
-                }
-            }
         }
     }
 
@@ -163,7 +154,18 @@ public class SparkJobParseBolt extends BaseRichBolt {
 
             boolean exists = true;
             while (exists) {
+                // For Yarn version 2.4.x
+                // log name: application_1464382345557_269065_1
                 String attemptIdString = Integer.toString(attemptId);
+
+                // For Yarn version >= 2.7,
+                // log name: "application_1468625664674_0003_appattempt_1468625664674_0003_000001"
+//                String attemptIdFormatted = String.format("%06d", attemptId);
+//
+//                // remove "application_" to get the number part of appID.
+//                String sparkAppIdNum = appId.substring(12);
+//                String attemptIdString = "appattempt_" + sparkAppIdNum + "_" + attemptIdFormatted;
+
                 String appAttemptLogName = this.getAppAttemptLogName(appId, attemptIdString);
                 LOG.info("Attempt ID: {}, App Attempt Log: {}", attemptIdString, appAttemptLogName);
 

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/6f5f972c/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/Utils.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/Utils.java b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/Utils.java
index 2696269..d738439 100644
--- a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/Utils.java
+++ b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/Utils.java
@@ -59,7 +59,7 @@ public class Utils {
         }
 
         if (timestamp == 0L) {
-            LOG.error("Not able to parse date: " + date);
+            LOG.warn("Not able to parse date: " + date);
         }
 
         return timestamp;
@@ -82,7 +82,8 @@ public class Utils {
             int executorPB = Integer.parseInt(memory.substring(0, memory.length() - 1));
             return 1024L * 1024 * 1024 * 1024 * 1024 * executorPB;
         }
-        LOG.info("Cannot parse memory info " + memory);
+        LOG.warn("Cannot parse memory info " +  memory);
+
         return 0L;
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/6f5f972c/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/RMResourceFetcher.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/RMResourceFetcher.java b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/RMResourceFetcher.java
index eb13c3c..b1881ef 100644
--- a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/RMResourceFetcher.java
+++ b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/RMResourceFetcher.java
@@ -91,11 +91,11 @@ public class RMResourceFetcher implements ResourceFetcher<AppInfo> {
     }
 
     private String getSparkRunningJobURL() {
-        StringBuilder sb = new StringBuilder();
-        sb.append(selector.getSelectedUrl()).append("/").append(Constants.V2_APPS_URL);
-        sb.append("?applicationTypes=SPARK&state=RUNNING&");
-        sb.append(Constants.ANONYMOUS_PARAMETER);
-        return sb.toString();
+        return selector.getSelectedUrl()
+                + "/"
+                + Constants.V2_APPS_URL
+                + "?applicationTypes=SPARK&state=RUNNING&"
+                + Constants.ANONYMOUS_PARAMETER;
     }
 
     private String getMRRunningJobURL() {
@@ -105,14 +105,11 @@ public class RMResourceFetcher implements ResourceFetcher<AppInfo> {
             Constants.ANONYMOUS_PARAMETER);
     }
 
-    public String getMRFinishedJobURL(String lastFinishedTime) {
+    private String getMRFinishedJobURL(String lastFinishedTime) {
         String url = URLUtil.removeTrailingSlash(selector.getSelectedUrl());
-        StringBuilder sb = new StringBuilder();
-        sb.append(url).append("/").append(Constants.V2_APPS_URL);
-        sb.append("?applicationTypes=MAPREDUCE&state=FINISHED&finishedTimeBegin=");
-        sb.append(lastFinishedTime).append("&").append(Constants.ANONYMOUS_PARAMETER);
-
-        return sb.toString();
+        return url + "/" + "Constants.V2_APPS_URL"
+                + "?applicationTypes=MAPREDUCE&state=FINISHED&finishedTimeBegin="
+                + lastFinishedTime + "&" + Constants.ANONYMOUS_PARAMETER;
     }
 
     private List<AppInfo> doFetchRunningApplicationsList(String urlString, Constants.CompressionType compressionType) throws Exception {
@@ -139,10 +136,10 @@ public class RMResourceFetcher implements ResourceFetcher<AppInfo> {
         }
     }
 
-    private List<AppInfo> getResource(Constants.ResourceType resoureType, Constants.CompressionType compressionType, Object... parameter) throws Exception {
-        switch (resoureType) {
+    private List<AppInfo> getResource(Constants.ResourceType resourceType, Constants.CompressionType compressionType, Object... parameter) throws Exception {
+        switch (resourceType) {
             case COMPLETE_SPARK_JOB:
-                final String urlString = sparkCompleteJobServiceURLBuilder.build((String) parameter[0]);
+                final String urlString = sparkCompleteJobServiceURLBuilder.build(selector.getSelectedUrl(), (String) parameter[0]);
                 return doFetchFinishApplicationsList(urlString, compressionType);
             case RUNNING_SPARK_JOB:
                 return doFetchRunningApplicationsList(getSparkRunningJobURL(), compressionType);
@@ -151,22 +148,20 @@ public class RMResourceFetcher implements ResourceFetcher<AppInfo> {
             case COMPLETE_MR_JOB:
                 return doFetchFinishApplicationsList(getMRFinishedJobURL((String) parameter[0]), compressionType);
             default:
-                throw new Exception("Not support resourceType :" + resoureType);
+                throw new Exception("Not support resourceType :" + resourceType);
         }
     }
 
-    public List<AppInfo> getResource(Constants.ResourceType resoureType, Object... parameter) throws Exception {
+    public List<AppInfo> getResource(Constants.ResourceType resourceType, Object... parameter) throws Exception {
         try {
-            return getResource(resoureType, Constants.CompressionType.GZIP, parameter);
+            return getResource(resourceType, Constants.CompressionType.GZIP, parameter);
         } catch (java.util.zip.ZipException ex) {
-            return getResource(resoureType, Constants.CompressionType.NONE, parameter);
+            return getResource(resourceType, Constants.CompressionType.NONE, parameter);
         }
     }
 
     private String getClusterInfoURL() {
-        StringBuilder sb = new StringBuilder();
-        sb.append(selector.getSelectedUrl()).append("/").append(Constants.YARN_API_CLUSTER_INFO).append("?" + Constants.ANONYMOUS_PARAMETER);
-        return sb.toString();
+        return selector.getSelectedUrl() + "/" + Constants.YARN_API_CLUSTER_INFO + "?" + Constants.ANONYMOUS_PARAMETER;
     }
 
     public ClusterInfo getClusterInfo() throws Exception {
@@ -191,4 +186,4 @@ public class RMResourceFetcher implements ResourceFetcher<AppInfo> {
             }
         }
     }
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/6f5f972c/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/ha/HAURLSelectorImpl.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/ha/HAURLSelectorImpl.java b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/ha/HAURLSelectorImpl.java
index 7c188c6..a083ef2 100644
--- a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/ha/HAURLSelectorImpl.java
+++ b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/ha/HAURLSelectorImpl.java
@@ -94,7 +94,7 @@ public class HAURLSelectorImpl implements HAURLSelector {
                         }
                         LOG.info("try url " + urlToCheck + "fail for " + (time + 1) + " times, sleep 5 seconds before try again. ");
                         try {
-                            Thread.sleep(1 * 1000);
+                            Thread.sleep(1000);
                         } catch (InterruptedException ex) {
                             LOG.warn("{}", ex);
                         }
@@ -106,4 +106,4 @@ public class HAURLSelectorImpl implements HAURLSelector {
             }
         }
     }
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/6f5f972c/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/url/SparkCompleteJobServiceURLBuilderImpl.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/url/SparkCompleteJobServiceURLBuilderImpl.java b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/url/SparkCompleteJobServiceURLBuilderImpl.java
index 8d959b7..ca6e938 100644
--- a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/url/SparkCompleteJobServiceURLBuilderImpl.java
+++ b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourceFetch/url/SparkCompleteJobServiceURLBuilderImpl.java
@@ -26,11 +26,8 @@ public class SparkCompleteJobServiceURLBuilderImpl implements ServiceURLBuilder
     public String build(String... parameters) {
         String url = URLUtil.removeTrailingSlash(parameters[0]);
 
-        StringBuilder sb = new StringBuilder();
-        sb.append(url).append("/").append(Constants.V2_APPS_URL);
-        sb.append("?applicationTypes=SPARK&state=FINISHED&finishedTimeBegin=");
-        sb.append(parameters[1]).append("&").append(Constants.ANONYMOUS_PARAMETER);
-
-        return sb.toString();
+        return url + "/" + Constants.V2_APPS_URL
+                + "?applicationTypes=SPARK&state=FINISHED&finishedTimeBegin="
+                + parameters[1] + "&" + Constants.ANONYMOUS_PARAMETER;
     }
-}
+}
\ No newline at end of file



[23/52] [abbrv] incubator-eagle git commit: [EAGLE-495] Convert spark history job using application framework.

Posted by yo...@apache.org.
[EAGLE-495] Convert spark history job using application framework.

Author: pkuwm <ih...@gmail.com>

Closes #393 from pkuwm/EAGLE-495.


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

Branch: refs/heads/master
Commit: 4f4fd0c4f606a8bc8ab83c66510aca4226becef8
Parents: a10eeb7
Author: pkuwm <ih...@gmail.com>
Authored: Mon Aug 29 10:19:09 2016 +0800
Committer: Hao Chen <ha...@apache.org>
Committed: Mon Aug 29 10:19:09 2016 +0800

----------------------------------------------------------------------
 .../jpm/spark/crawl/JHFSparkEventReader.java    |  13 +-
 eagle-jpm/eagle-jpm-spark-history/pom.xml       |   5 +
 .../jpm/spark/history/SparkHistoryJobApp.java   |  54 ++++
 .../spark/history/SparkHistoryJobAppConfig.java | 133 +++++++++
 .../history/SparkHistoryJobAppProvider.java     |  27 ++
 .../jpm/spark/history/SparkHistoryJobMain.java  |  25 ++
 .../history/config/SparkHistoryCrawlConfig.java | 123 ---------
 .../status/JobHistoryZKStateManager.java        |   6 +-
 .../history/storm/FinishedSparkJobSpout.java    | 154 -----------
 .../history/storm/SparkHistoryJobParseBolt.java | 201 ++++++++++++++
 .../history/storm/SparkHistoryJobSpout.java     | 154 +++++++++++
 .../history/storm/SparkHistoryTopology.java     |  81 ------
 .../spark/history/storm/SparkJobParseBolt.java  | 201 --------------
 .../eagle/jpm/spark/history/storm/TestHDFS.java |  47 ----
 ...spark.history.SparkHistoryJobAppProvider.xml | 271 +++++++++++++++++++
 ...org.apache.eagle.app.spi.ApplicationProvider |  16 ++
 .../src/main/resources/application.conf         |  34 +--
 .../java/SparkHistoryJobAppProviderTest.java    |  32 +++
 .../running/SparkRunningJobAppProvider.java     |   2 +-
 19 files changed, 947 insertions(+), 632 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/4f4fd0c4/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/spark/crawl/JHFSparkEventReader.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/spark/crawl/JHFSparkEventReader.java b/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/spark/crawl/JHFSparkEventReader.java
index edb3854..1cd5a77 100644
--- a/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/spark/crawl/JHFSparkEventReader.java
+++ b/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/spark/crawl/JHFSparkEventReader.java
@@ -17,6 +17,7 @@
 
 package org.apache.eagle.jpm.spark.crawl;
 
+import org.apache.commons.lang.ArrayUtils;
 import org.apache.eagle.jpm.spark.entity.*;
 import org.apache.eagle.jpm.util.*;
 import org.apache.eagle.log.base.taggedlog.TaggedLogAPIEntity;
@@ -109,12 +110,12 @@ public class JHFSparkEventReader {
         app.setConfig(new JobConfig());
         JSONObject sparkProps = (JSONObject) event.get("Spark Properties");
 
-        List<String> jobConfs = conf.getStringList("basic.jobConf.additional.info");
+        String[] additionalJobConf = conf.getString("basic.jobConf.additional.info").split(",\\s*");
         String[] props = {"spark.yarn.app.id", "spark.executor.memory", "spark.driver.host", "spark.driver.port",
             "spark.driver.memory", "spark.scheduler.pool", "spark.executor.cores", "spark.yarn.am.memory",
             "spark.yarn.am.cores", "spark.yarn.executor.memoryOverhead", "spark.yarn.driver.memoryOverhead", "spark.yarn.am.memoryOverhead", "spark.master"};
-        jobConfs.addAll(Arrays.asList(props));
-        for (String prop : jobConfs) {
+        String[] jobConf = (String[])ArrayUtils.addAll(additionalJobConf, props);
+        for (String prop : jobConf) {
             if (sparkProps.containsKey(prop)) {
                 app.getConfig().getConfig().put(prop, (String) sparkProps.get(prop));
             }
@@ -698,10 +699,10 @@ public class JHFSparkEventReader {
     private EagleServiceBaseClient initiateClient() {
         String host = conf.getString("eagleProps.eagle.service.host");
         int port = conf.getInt("eagleProps.eagle.service.port");
-        String userName = conf.getString("eagleProps.eagle.service.userName");
-        String pwd = conf.getString("eagleProps.eagle.service.pwd");
+        String userName = conf.getString("eagleProps.eagle.service.username");
+        String pwd = conf.getString("eagleProps.eagle.service.password");
         client = new EagleServiceClientImpl(host, port, userName, pwd);
-        int timeout = conf.getInt("eagleProps.eagle.service.read_timeout");
+        int timeout = conf.getInt("eagleProps.eagle.service.read.timeout");
         client.getJerseyClient().setReadTimeout(timeout * 1000);
 
         return client;

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/4f4fd0c4/eagle-jpm/eagle-jpm-spark-history/pom.xml
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-spark-history/pom.xml b/eagle-jpm/eagle-jpm-spark-history/pom.xml
index e144117..1c9c8b4 100644
--- a/eagle-jpm/eagle-jpm-spark-history/pom.xml
+++ b/eagle-jpm/eagle-jpm-spark-history/pom.xml
@@ -100,6 +100,11 @@
             <artifactId>hadoop-mapreduce-client-core</artifactId>
             <version>${hadoop.version}</version>
         </dependency>
+        <dependency>
+            <groupId>org.apache.eagle</groupId>
+            <artifactId>eagle-app-base</artifactId>
+            <version>${project.version}</version>
+        </dependency>
     </dependencies>
     <build>
         <resources>

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/4f4fd0c4/eagle-jpm/eagle-jpm-spark-history/src/main/java/org/apache/eagle/jpm/spark/history/SparkHistoryJobApp.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-spark-history/src/main/java/org/apache/eagle/jpm/spark/history/SparkHistoryJobApp.java b/eagle-jpm/eagle-jpm-spark-history/src/main/java/org/apache/eagle/jpm/spark/history/SparkHistoryJobApp.java
new file mode 100644
index 0000000..180b1e8
--- /dev/null
+++ b/eagle-jpm/eagle-jpm-spark-history/src/main/java/org/apache/eagle/jpm/spark/history/SparkHistoryJobApp.java
@@ -0,0 +1,54 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.eagle.jpm.spark.history;
+
+import backtype.storm.generated.StormTopology;
+import backtype.storm.topology.TopologyBuilder;
+import com.typesafe.config.Config;
+import org.apache.eagle.app.StormApplication;
+import org.apache.eagle.app.environment.impl.StormEnvironment;
+import org.apache.eagle.jpm.spark.history.storm.SparkHistoryJobSpout;
+import org.apache.eagle.jpm.spark.history.storm.SparkHistoryJobParseBolt;
+
+public class SparkHistoryJobApp extends StormApplication {
+    @Override
+    public StormTopology execute(Config config, StormEnvironment environment) {
+        // 1. Init conf
+        SparkHistoryJobAppConfig sparkHistoryJobAppConfig = SparkHistoryJobAppConfig.getInstance(config);
+
+        final String jobFetchSpoutName = SparkHistoryJobAppConfig.SPARK_HISTORY_JOB_FETCH_SPOUT_NAME;
+        final String jobParseBoltName = SparkHistoryJobAppConfig.SPARK_HISTORY_JOB_PARSE_BOLT_NAME;
+
+        // 2. Config topology.
+        TopologyBuilder topologyBuilder = new TopologyBuilder();
+        config = sparkHistoryJobAppConfig.getConfig();
+        topologyBuilder.setSpout(
+                jobFetchSpoutName,
+                new SparkHistoryJobSpout(sparkHistoryJobAppConfig),
+                config.getInt("storm.parallelismConfig." + jobFetchSpoutName)
+        ).setNumTasks(config.getInt("storm.tasks." + jobFetchSpoutName));
+
+        topologyBuilder.setBolt(
+                jobParseBoltName,
+                new SparkHistoryJobParseBolt(sparkHistoryJobAppConfig),
+                config.getInt("storm.parallelismConfig." + jobParseBoltName)
+        ).setNumTasks(config.getInt("storm.tasks." + jobParseBoltName)).shuffleGrouping(jobFetchSpoutName);
+
+        return topologyBuilder.createTopology();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/4f4fd0c4/eagle-jpm/eagle-jpm-spark-history/src/main/java/org/apache/eagle/jpm/spark/history/SparkHistoryJobAppConfig.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-spark-history/src/main/java/org/apache/eagle/jpm/spark/history/SparkHistoryJobAppConfig.java b/eagle-jpm/eagle-jpm-spark-history/src/main/java/org/apache/eagle/jpm/spark/history/SparkHistoryJobAppConfig.java
new file mode 100644
index 0000000..ed499db
--- /dev/null
+++ b/eagle-jpm/eagle-jpm-spark-history/src/main/java/org/apache/eagle/jpm/spark/history/SparkHistoryJobAppConfig.java
@@ -0,0 +1,133 @@
+/*
+ *
+ *  * 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.eagle.jpm.spark.history;
+
+import com.typesafe.config.Config;
+import com.typesafe.config.ConfigFactory;
+
+import java.io.Serializable;
+
+public class SparkHistoryJobAppConfig implements Serializable {
+    final static String SPARK_HISTORY_JOB_FETCH_SPOUT_NAME = "sparkHistoryJobFetchSpout";
+    final static String SPARK_HISTORY_JOB_PARSE_BOLT_NAME = "sparkHistoryJobParseBolt";
+
+    public ZKStateConfig zkStateConfig;
+    public JobHistoryEndpointConfig jobHistoryConfig;
+    public HDFSConfig hdfsConfig;
+    public BasicInfo info;
+    public EagleInfo eagleInfo;
+    public StormConfig stormConfig;
+
+    private Config config;
+
+    private static SparkHistoryJobAppConfig manager = new SparkHistoryJobAppConfig();
+    
+    public Config getConfig() {
+        return config;
+    }
+
+    public SparkHistoryJobAppConfig() {
+        this.zkStateConfig = new ZKStateConfig();
+        this.jobHistoryConfig = new JobHistoryEndpointConfig();
+        this.hdfsConfig = new HDFSConfig();
+        this.info = new BasicInfo();
+        this.eagleInfo = new EagleInfo();
+        this.stormConfig = new StormConfig();
+    }
+
+    public static SparkHistoryJobAppConfig getInstance(Config config) {
+        manager.init(config);
+        return manager;
+    }
+
+    private void init(Config config) {
+        this.config = config;
+
+        this.zkStateConfig.zkQuorum = config.getString("dataSourceConfig.zkQuorum");
+        this.zkStateConfig.zkRetryInterval = config.getInt("dataSourceConfig.zkRetryInterval");
+        this.zkStateConfig.zkRetryTimes = config.getInt("dataSourceConfig.zkRetryTimes");
+        this.zkStateConfig.zkSessionTimeoutMs = config.getInt("dataSourceConfig.zkSessionTimeoutMs");
+        this.zkStateConfig.zkRoot = config.getString("dataSourceConfig.zkRoot");
+
+        jobHistoryConfig.historyServerUrl = config.getString("dataSourceConfig.spark.history.server.url");
+        jobHistoryConfig.historyServerUserName = config.getString("dataSourceConfig.spark.history.server.username");
+        jobHistoryConfig.historyServerUserPwd = config.getString("dataSourceConfig.spark.history.server.password");
+        jobHistoryConfig.rms = config.getString("dataSourceConfig.rm.url").split(",\\s*");
+
+        this.hdfsConfig.baseDir = config.getString("dataSourceConfig.hdfs.eventLog");
+        this.hdfsConfig.endpoint = config.getString("dataSourceConfig.hdfs.endPoint");
+        this.hdfsConfig.principal = config.getString("dataSourceConfig.hdfs.principal");
+        this.hdfsConfig.keytab = config.getString("dataSourceConfig.hdfs.keytab");
+
+        info.site = config.getString("basic.cluster") + "-" + config.getString("basic.dataCenter");
+        info.jobConf = config.getString("basic.jobConf.additional.info").split(",\\s*");
+
+        this.eagleInfo.host = config.getString("eagleProps.eagle.service.host");
+        this.eagleInfo.port = config.getInt("eagleProps.eagle.service.port");
+
+        this.stormConfig.mode = config.getString("storm.mode");
+        this.stormConfig.topologyName = config.getString("storm.name");
+        this.stormConfig.workerNo = config.getInt("storm.worker.num");
+        this.stormConfig.timeoutSec = config.getInt("storm.messageTimeoutSec");
+        this.stormConfig.spoutPending = config.getInt("storm.pendingSpout");
+        this.stormConfig.spoutCrawlInterval = config.getInt("storm.spoutCrawlInterval");
+    }
+
+    public static class ZKStateConfig implements Serializable {
+        public String zkQuorum;
+        public String zkRoot;
+        public int zkSessionTimeoutMs;
+        public int zkRetryTimes;
+        public int zkRetryInterval;
+    }
+
+    public static class JobHistoryEndpointConfig implements Serializable {
+        public String[] rms;
+        public String historyServerUrl;
+        public String historyServerUserName;
+        public String historyServerUserPwd;
+    }
+
+    public static class HDFSConfig implements Serializable {
+        public String endpoint;
+        public String baseDir;
+        public String principal;
+        public String keytab;
+    }
+
+    public static class BasicInfo implements Serializable {
+        public String site;
+        public String[] jobConf;
+    }
+
+    public static class StormConfig implements Serializable {
+        public String mode;
+        public int workerNo;
+        public int timeoutSec;
+        public String topologyName;
+        public int spoutPending;
+        public int spoutCrawlInterval;
+    }
+
+    public static class EagleInfo implements Serializable {
+        public String host;
+        public int port;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/4f4fd0c4/eagle-jpm/eagle-jpm-spark-history/src/main/java/org/apache/eagle/jpm/spark/history/SparkHistoryJobAppProvider.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-spark-history/src/main/java/org/apache/eagle/jpm/spark/history/SparkHistoryJobAppProvider.java b/eagle-jpm/eagle-jpm-spark-history/src/main/java/org/apache/eagle/jpm/spark/history/SparkHistoryJobAppProvider.java
new file mode 100644
index 0000000..343d9c2
--- /dev/null
+++ b/eagle-jpm/eagle-jpm-spark-history/src/main/java/org/apache/eagle/jpm/spark/history/SparkHistoryJobAppProvider.java
@@ -0,0 +1,27 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.eagle.jpm.spark.history;
+
+import org.apache.eagle.app.spi.AbstractApplicationProvider;
+
+public class SparkHistoryJobAppProvider extends AbstractApplicationProvider<SparkHistoryJobApp> {
+    @Override
+    public SparkHistoryJobApp getApplication() {
+        return new SparkHistoryJobApp();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/4f4fd0c4/eagle-jpm/eagle-jpm-spark-history/src/main/java/org/apache/eagle/jpm/spark/history/SparkHistoryJobMain.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-spark-history/src/main/java/org/apache/eagle/jpm/spark/history/SparkHistoryJobMain.java b/eagle-jpm/eagle-jpm-spark-history/src/main/java/org/apache/eagle/jpm/spark/history/SparkHistoryJobMain.java
new file mode 100644
index 0000000..e47e5b2
--- /dev/null
+++ b/eagle-jpm/eagle-jpm-spark-history/src/main/java/org/apache/eagle/jpm/spark/history/SparkHistoryJobMain.java
@@ -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.
+*/
+
+package org.apache.eagle.jpm.spark.history;
+
+public class SparkHistoryJobMain {
+    public static void main(String[] args) {
+        new SparkHistoryJobApp().run(args);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/4f4fd0c4/eagle-jpm/eagle-jpm-spark-history/src/main/java/org/apache/eagle/jpm/spark/history/config/SparkHistoryCrawlConfig.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-spark-history/src/main/java/org/apache/eagle/jpm/spark/history/config/SparkHistoryCrawlConfig.java b/eagle-jpm/eagle-jpm-spark-history/src/main/java/org/apache/eagle/jpm/spark/history/config/SparkHistoryCrawlConfig.java
deleted file mode 100644
index e6cd2f6..0000000
--- a/eagle-jpm/eagle-jpm-spark-history/src/main/java/org/apache/eagle/jpm/spark/history/config/SparkHistoryCrawlConfig.java
+++ /dev/null
@@ -1,123 +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.eagle.jpm.spark.history.config;
-
-
-import com.typesafe.config.Config;
-import com.typesafe.config.ConfigFactory;
-
-import java.io.Serializable;
-
-public class SparkHistoryCrawlConfig implements Serializable {
-    public ZKStateConfig zkStateConfig;
-    public JobHistoryEndpointConfig jobHistoryConfig;
-    public HDFSConfig hdfsConfig;
-    public BasicInfo info;
-    public EagleInfo eagleInfo;
-    public StormConfig stormConfig;
-
-    private Config config;
-    
-    public Config getConfig() {
-        return config;
-    }
-
-    public SparkHistoryCrawlConfig() {
-        this.config = ConfigFactory.load();
-
-        this.zkStateConfig = new ZKStateConfig();
-        this.zkStateConfig.zkQuorum = config.getString("dataSourceConfig.zkQuorum");
-        this.zkStateConfig.zkRetryInterval = config.getInt("dataSourceConfig.zkRetryInterval");
-        this.zkStateConfig.zkRetryTimes = config.getInt("dataSourceConfig.zkRetryTimes");
-        this.zkStateConfig.zkSessionTimeoutMs = config.getInt("dataSourceConfig.zkSessionTimeoutMs");
-        this.zkStateConfig.zkRoot = config.getString("dataSourceConfig.zkRoot");
-
-        this.jobHistoryConfig = new JobHistoryEndpointConfig();
-        jobHistoryConfig.historyServerUrl = config.getString("dataSourceConfig.spark.history.server.url");
-        jobHistoryConfig.historyServerUserName = config.getString("dataSourceConfig.spark.history.server.username");
-        jobHistoryConfig.historyServerUserPwd = config.getString("dataSourceConfig.spark.history.server.pwd");
-        jobHistoryConfig.rms = config.getStringList("dataSourceConfig.rm.url").toArray(new String[0]);
-
-        this.hdfsConfig = new HDFSConfig();
-        this.hdfsConfig.baseDir = config.getString("dataSourceConfig.hdfs.baseDir");
-        this.hdfsConfig.endpoint = config.getString("dataSourceConfig.hdfs.endPoint");
-        this.hdfsConfig.principal = config.getString("dataSourceConfig.hdfs.principal");
-        this.hdfsConfig.keytab = config.getString("dataSourceConfig.hdfs.keytab");
-
-        this.info = new BasicInfo();
-        info.site = String.format("%s-%s",config.getString("basic.cluster"),config.getString("basic.datacenter"));
-        info.jobConf = config.getStringList("basic.jobConf.additional.info").toArray(new String[0]);
-
-        this.eagleInfo = new EagleInfo();
-        this.eagleInfo.host = config.getString("eagleProps.eagle.service.host");
-        this.eagleInfo.port = config.getInt("eagleProps.eagle.service.port");
-
-        this.stormConfig = new StormConfig();
-        this.stormConfig.mode = config.getString("storm.mode");
-        this.stormConfig.topologyName = config.getString("storm.name");
-        this.stormConfig.workerNo = config.getInt("storm.workerNo");
-        this.stormConfig.timeoutSec = config.getInt("storm.messageTimeoutSec");
-        this.stormConfig.spoutPending = config.getInt("storm.pendingSpout");
-        this.stormConfig.spoutCrawlInterval = config.getInt("storm.spoutCrawlInterval");
-    }
-
-    public static class ZKStateConfig implements Serializable {
-        public String zkQuorum;
-        public String zkRoot;
-        public int zkSessionTimeoutMs;
-        public int zkRetryTimes;
-        public int zkRetryInterval;
-    }
-
-    public static class JobHistoryEndpointConfig implements Serializable {
-        public String[] rms;
-        public String historyServerUrl;
-        public String historyServerUserName;
-        public String historyServerUserPwd;
-    }
-
-    public static class HDFSConfig implements Serializable {
-        public String endpoint;
-        public String baseDir;
-        public String principal;
-        public String keytab;
-    }
-
-    public static class BasicInfo implements Serializable {
-        public String site;
-        public String[] jobConf;
-    }
-
-    public static class StormConfig implements Serializable {
-        public String mode;
-        public int workerNo;
-        public int timeoutSec;
-        public String topologyName;
-        public int spoutPending;
-        public int spoutCrawlInterval;
-    }
-
-    public static class EagleInfo implements Serializable {
-        public String host;
-        public int port;
-    }
-
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/4f4fd0c4/eagle-jpm/eagle-jpm-spark-history/src/main/java/org/apache/eagle/jpm/spark/history/status/JobHistoryZKStateManager.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-spark-history/src/main/java/org/apache/eagle/jpm/spark/history/status/JobHistoryZKStateManager.java b/eagle-jpm/eagle-jpm-spark-history/src/main/java/org/apache/eagle/jpm/spark/history/status/JobHistoryZKStateManager.java
index 382375f..7a95e56 100644
--- a/eagle-jpm/eagle-jpm-spark-history/src/main/java/org/apache/eagle/jpm/spark/history/status/JobHistoryZKStateManager.java
+++ b/eagle-jpm/eagle-jpm-spark-history/src/main/java/org/apache/eagle/jpm/spark/history/status/JobHistoryZKStateManager.java
@@ -20,7 +20,7 @@
 package org.apache.eagle.jpm.spark.history.status;
 
 import org.apache.eagle.jpm.spark.crawl.SparkApplicationInfo;
-import org.apache.eagle.jpm.spark.history.config.SparkHistoryCrawlConfig;
+import org.apache.eagle.jpm.spark.history.SparkHistoryJobAppConfig;
 import org.apache.curator.framework.CuratorFramework;
 import org.apache.curator.framework.CuratorFrameworkFactory;
 import org.apache.curator.framework.api.transaction.CuratorTransactionBridge;
@@ -40,7 +40,7 @@ public class JobHistoryZKStateManager {
     private CuratorFramework curator;
     private static String START_TIMESTAMP = "lastAppTime";
 
-    private CuratorFramework newCurator(SparkHistoryCrawlConfig config) throws Exception {
+    private CuratorFramework newCurator(SparkHistoryJobAppConfig config) throws Exception {
         return CuratorFrameworkFactory.newClient(
                 config.zkStateConfig.zkQuorum,
                 config.zkStateConfig.zkSessionTimeoutMs,
@@ -49,7 +49,7 @@ public class JobHistoryZKStateManager {
         );
     }
 
-    public JobHistoryZKStateManager(SparkHistoryCrawlConfig config) {
+    public JobHistoryZKStateManager(SparkHistoryJobAppConfig config) {
         this.zkRoot = config.zkStateConfig.zkRoot + "/" + config.info.site;
 
         try {

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/4f4fd0c4/eagle-jpm/eagle-jpm-spark-history/src/main/java/org/apache/eagle/jpm/spark/history/storm/FinishedSparkJobSpout.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-spark-history/src/main/java/org/apache/eagle/jpm/spark/history/storm/FinishedSparkJobSpout.java b/eagle-jpm/eagle-jpm-spark-history/src/main/java/org/apache/eagle/jpm/spark/history/storm/FinishedSparkJobSpout.java
deleted file mode 100644
index bf04b55..0000000
--- a/eagle-jpm/eagle-jpm-spark-history/src/main/java/org/apache/eagle/jpm/spark/history/storm/FinishedSparkJobSpout.java
+++ /dev/null
@@ -1,154 +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.eagle.jpm.spark.history.storm;
-
-import org.apache.eagle.jpm.spark.history.config.SparkHistoryCrawlConfig;
-import org.apache.eagle.jpm.spark.history.status.JobHistoryZKStateManager;
-import org.apache.eagle.jpm.spark.history.status.ZKStateConstant;
-import org.apache.eagle.jpm.util.Constants;
-import org.apache.eagle.jpm.util.resourcefetch.RMResourceFetcher;
-import org.apache.eagle.jpm.util.resourcefetch.ResourceFetcher;
-import org.apache.eagle.jpm.util.resourcefetch.model.AppInfo;
-import backtype.storm.spout.SpoutOutputCollector;
-import backtype.storm.task.TopologyContext;
-import backtype.storm.topology.OutputFieldsDeclarer;
-import backtype.storm.topology.base.BaseRichSpout;
-import backtype.storm.tuple.Fields;
-import backtype.storm.tuple.Values;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import java.util.Calendar;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-public class FinishedSparkJobSpout extends BaseRichSpout {
-    private static final Logger LOG = LoggerFactory.getLogger(FinishedSparkJobSpout.class);
-    private SpoutOutputCollector collector;
-    private JobHistoryZKStateManager zkState;
-    private SparkHistoryCrawlConfig config;
-    private ResourceFetcher rmFetch;
-    private long lastFinishAppTime = 0;
-    private Map<String, Integer> failTimes;
-
-    private static final int FAIL_MAX_TIMES = 5;
-
-    public FinishedSparkJobSpout(SparkHistoryCrawlConfig config) {
-        this.config = config;
-    }
-
-    @Override
-    public void open(Map map, TopologyContext topologyContext, SpoutOutputCollector spoutOutputCollector) {
-        rmFetch = new RMResourceFetcher(config.jobHistoryConfig.rms);
-        this.failTimes = new HashMap<>();
-        this.collector = spoutOutputCollector;
-        this.zkState = new JobHistoryZKStateManager(config);
-        this.lastFinishAppTime = zkState.readLastFinishedTimestamp();
-        zkState.resetApplications();
-    }
-
-
-    @Override
-    public void nextTuple() {
-        LOG.info("Start to run tuple");
-        try {
-            Calendar calendar = Calendar.getInstance();
-            long fetchTime = calendar.getTimeInMillis();
-            calendar.setTimeInMillis(this.lastFinishAppTime);
-            LOG.info("Last finished time = {}", calendar.getTime());
-            if (fetchTime - this.lastFinishAppTime > this.config.stormConfig.spoutCrawlInterval) {
-                List<AppInfo> appInfos = rmFetch.getResource(Constants.ResourceType.COMPLETE_SPARK_JOB, Long.toString(lastFinishAppTime));
-                //List<AppInfo> appInfos = (null != apps ? (List<AppInfo>)apps.get(0):new ArrayList<AppInfo>());
-                if (appInfos != null) {
-                    LOG.info("Get " + appInfos.size() + " from yarn resource manager.");
-                    for (AppInfo app : appInfos) {
-                        String appId = app.getId();
-                        if (!zkState.hasApplication(appId)) {
-                            zkState.addFinishedApplication(appId, app.getQueue(), app.getState(), app.getFinalStatus(), app.getUser(), app.getName());
-                        }
-                    }
-                }
-                this.lastFinishAppTime = fetchTime;
-                zkState.updateLastUpdateTime(fetchTime);
-            }
-
-            List<String> appIds = zkState.loadApplications(10);
-            for (String appId: appIds) {
-                collector.emit(new Values(appId), appId);
-                LOG.info("emit " + appId);
-                zkState.updateApplicationStatus(appId, ZKStateConstant.AppStatus.SENT_FOR_PARSE);
-            }
-            LOG.info("{} apps sent.", appIds.size());
-
-            if (appIds.isEmpty()) {
-                this.takeRest(60);
-            }
-        } catch (Exception e) {
-            LOG.error("Fail to run next tuple", e);
-        }
-    }
-
-    private void takeRest(int seconds) {
-        try {
-            Thread.sleep(seconds * 1000);
-        } catch (InterruptedException e) {
-            LOG.warn("exception found {}", e);
-        }
-    }
-
-    @Override
-    public void declareOutputFields(OutputFieldsDeclarer outputFieldsDeclarer) {
-        outputFieldsDeclarer.declare(new Fields("appId"));
-    }
-
-    @Override
-    public void fail(Object msgId) {
-        String appId = (String) msgId;
-        int failTimes = 0;
-        if (this.failTimes.containsKey(appId)) {
-            failTimes = this.failTimes.get(appId);
-        }
-        failTimes++;
-        if (failTimes >= FAIL_MAX_TIMES) {
-            this.failTimes.remove(appId);
-            zkState.updateApplicationStatus(appId, ZKStateConstant.AppStatus.FINISHED);
-            LOG.error(String.format("Application %s has failed for over %s times, drop it.", appId, FAIL_MAX_TIMES));
-        } else {
-            this.failTimes.put(appId, failTimes);
-            collector.emit(new Values(appId), appId);
-            zkState.updateApplicationStatus(appId, ZKStateConstant.AppStatus.SENT_FOR_PARSE);
-        }
-    }
-
-    @Override
-    public void ack(Object msgId) {
-        String appId = (String) msgId;
-        if (this.failTimes.containsKey(appId)) {
-            this.failTimes.remove(appId);
-        }
-
-    }
-
-    @Override
-    public void close() {
-        super.close();
-        zkState.close();
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/4f4fd0c4/eagle-jpm/eagle-jpm-spark-history/src/main/java/org/apache/eagle/jpm/spark/history/storm/SparkHistoryJobParseBolt.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-spark-history/src/main/java/org/apache/eagle/jpm/spark/history/storm/SparkHistoryJobParseBolt.java b/eagle-jpm/eagle-jpm-spark-history/src/main/java/org/apache/eagle/jpm/spark/history/storm/SparkHistoryJobParseBolt.java
new file mode 100644
index 0000000..e88c62f
--- /dev/null
+++ b/eagle-jpm/eagle-jpm-spark-history/src/main/java/org/apache/eagle/jpm/spark/history/storm/SparkHistoryJobParseBolt.java
@@ -0,0 +1,201 @@
+/*
+ *
+ *  * 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.eagle.jpm.spark.history.storm;
+
+import org.apache.eagle.jpm.spark.crawl.JHFInputStreamReader;
+import org.apache.eagle.jpm.spark.crawl.SparkApplicationInfo;
+import org.apache.eagle.jpm.spark.crawl.SparkFilesystemInputStreamReaderImpl;
+import org.apache.eagle.jpm.spark.history.SparkHistoryJobAppConfig;
+import org.apache.eagle.jpm.spark.history.status.JobHistoryZKStateManager;
+import org.apache.eagle.jpm.spark.history.status.ZKStateConstant;
+import org.apache.eagle.jpm.util.HDFSUtil;
+import org.apache.eagle.jpm.util.resourcefetch.ResourceFetcher;
+import org.apache.eagle.jpm.util.resourcefetch.SparkHistoryServerResourceFetcher;
+import org.apache.eagle.jpm.util.resourcefetch.model.SparkApplication;
+import backtype.storm.task.OutputCollector;
+import backtype.storm.task.TopologyContext;
+import backtype.storm.topology.OutputFieldsDeclarer;
+import backtype.storm.topology.base.BaseRichBolt;
+import backtype.storm.tuple.Tuple;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.*;
+
+public class SparkHistoryJobParseBolt extends BaseRichBolt {
+
+    private static final Logger LOG = LoggerFactory.getLogger(SparkHistoryJobParseBolt.class);
+
+    private OutputCollector collector;
+    private ResourceFetcher historyServerFetcher;
+    private SparkHistoryJobAppConfig config;
+    private JobHistoryZKStateManager zkState;
+    private Configuration hdfsConf;
+
+    public SparkHistoryJobParseBolt(SparkHistoryJobAppConfig config) {
+        this.config = config;
+    }
+
+    @Override
+    public void prepare(Map map, TopologyContext topologyContext, OutputCollector outputCollector) {
+        this.collector = outputCollector;
+        this.hdfsConf = new Configuration();
+        this.hdfsConf.set("fs.defaultFS", config.hdfsConfig.endpoint);
+        this.hdfsConf.setBoolean("fs.hdfs.impl.disable.cache", true);
+        this.hdfsConf.set("hdfs.kerberos.principal", config.hdfsConfig.principal);
+        this.hdfsConf.set("hdfs.keytab.file", config.hdfsConfig.keytab);
+        this.historyServerFetcher = new SparkHistoryServerResourceFetcher(config.jobHistoryConfig.historyServerUrl,
+                config.jobHistoryConfig.historyServerUserName, config.jobHistoryConfig.historyServerUserPwd);
+        this.zkState = new JobHistoryZKStateManager(config);
+    }
+
+    @Override
+    public void execute(Tuple tuple) {
+        String appId = tuple.getStringByField("appId");
+        if (!zkState.hasApplication(appId)) {
+            //may already be processed due to some reason
+            collector.ack(tuple);
+            return;
+        }
+
+        try (FileSystem hdfs = HDFSUtil.getFileSystem(this.hdfsConf)) {
+            SparkApplicationInfo info = zkState.getApplicationInfo(appId);
+            //first try to get attempts under the application
+
+            Set<String> inprogressSet = new HashSet<String>();
+            List<String> attemptLogNames = this.getAttemptLogNameList(appId, hdfs, inprogressSet);
+
+            if (attemptLogNames.isEmpty()) {
+                LOG.info("Application:{}( Name:{}, user: {}, queue: {}) not found on history server.",
+                    appId, info.getName(), info.getUser(), info.getQueue());
+            } else {
+                for (String attemptLogName : attemptLogNames) {
+                    String extension = "";
+                    if (inprogressSet.contains(attemptLogName)) {
+                        extension = ".inprogress";
+                    }
+                    LOG.info("Attempt log name: " + attemptLogName + extension);
+
+                    Path attemptFile = getFilePath(attemptLogName, extension);
+                    JHFInputStreamReader reader = new SparkFilesystemInputStreamReaderImpl(config.info.site, info);
+                    reader.read(hdfs.open(attemptFile));
+                }
+            }
+
+            zkState.updateApplicationStatus(appId, ZKStateConstant.AppStatus.FINISHED);
+            LOG.info("Successfully parse application {}", appId);
+            collector.ack(tuple);
+        } catch (Exception e) {
+            LOG.error("Fail to process application {}", appId, e);
+            zkState.updateApplicationStatus(appId, ZKStateConstant.AppStatus.FAILED);
+            collector.fail(tuple);
+        }
+    }
+
+    @Override
+    public void declareOutputFields(OutputFieldsDeclarer outputFieldsDeclarer) {
+
+    }
+
+    private String getAppAttemptLogName(String appId, String attemptId) {
+        if (attemptId.equals("0")) {
+            return appId;
+        }
+        return appId + "_" + attemptId;
+    }
+
+    private Path getFilePath(String appAttemptLogName, String extension) {
+        String attemptLogDir = this.config.hdfsConfig.baseDir + "/" + appAttemptLogName + extension;
+        return new Path(attemptLogDir);
+    }
+
+    private List<String> getAttemptLogNameList(String appId, FileSystem hdfs, Set<String> inprogressSet)
+            throws IOException {
+        List<String> attempts = new ArrayList<String>();
+        SparkApplication app = null;
+        /*try {
+            List apps = this.historyServerFetcher.getResource(Constants.ResourceType.SPARK_JOB_DETAIL, appId);
+            if (apps != null) {
+                app = (SparkApplication) apps.get(0);
+                attempts = app.getAttempts();
+            }
+        } catch (Exception e) {
+            LOG.warn("Fail to get application detail from history server for appId " + appId, e);
+        }*/
+
+
+        if (null == app) {
+            // history server may not have the info, just double check.
+            // TODO: if attemptId is not "1, 2, 3,...", we should change the logic.
+            // Use getResourceManagerVersion() to compare YARN/RM versions.
+            // attemptId might be: "appId_000001"
+            int attemptId = 0;
+
+            boolean exists = true;
+            while (exists) {
+                // For Yarn version 2.4.x
+                // log name: application_1464382345557_269065_1
+                String attemptIdString = Integer.toString(attemptId);
+
+                // For Yarn version >= 2.7,
+                // log name: "application_1468625664674_0003_appattempt_1468625664674_0003_000001"
+//                String attemptIdFormatted = String.format("%06d", attemptId);
+//
+//                // remove "application_" to get the number part of appID.
+//                String sparkAppIdNum = appId.substring(12);
+//                String attemptIdString = "appattempt_" + sparkAppIdNum + "_" + attemptIdFormatted;
+
+                String appAttemptLogName = this.getAppAttemptLogName(appId, attemptIdString);
+                LOG.info("Attempt ID: {}, App Attempt Log: {}", attemptIdString, appAttemptLogName);
+
+                String extension = "";
+                Path attemptFile = getFilePath(appAttemptLogName, extension);
+                extension = ".inprogress";
+                Path inprogressFile = getFilePath(appAttemptLogName, extension);
+                Path logFile = null;
+                // Check if history log exists.
+                if (hdfs.exists(attemptFile)) {
+                    logFile = attemptFile;
+                } else if (hdfs.exists(inprogressFile)) {
+                    logFile = inprogressFile;
+                    inprogressSet.add(appAttemptLogName);
+                } else if (attemptId > 0) {
+                    exists = false;
+                }
+
+                if (logFile != null) {
+                    attempts.add(appAttemptLogName);
+                }
+                attemptId++;
+            }
+        }
+        return attempts;
+    }
+
+    @Override
+    public void cleanup() {
+        super.cleanup();
+        zkState.close();
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/4f4fd0c4/eagle-jpm/eagle-jpm-spark-history/src/main/java/org/apache/eagle/jpm/spark/history/storm/SparkHistoryJobSpout.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-spark-history/src/main/java/org/apache/eagle/jpm/spark/history/storm/SparkHistoryJobSpout.java b/eagle-jpm/eagle-jpm-spark-history/src/main/java/org/apache/eagle/jpm/spark/history/storm/SparkHistoryJobSpout.java
new file mode 100644
index 0000000..db60744
--- /dev/null
+++ b/eagle-jpm/eagle-jpm-spark-history/src/main/java/org/apache/eagle/jpm/spark/history/storm/SparkHistoryJobSpout.java
@@ -0,0 +1,154 @@
+/*
+ *
+ *  * 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.eagle.jpm.spark.history.storm;
+
+import org.apache.eagle.jpm.spark.history.SparkHistoryJobAppConfig;
+import org.apache.eagle.jpm.spark.history.status.JobHistoryZKStateManager;
+import org.apache.eagle.jpm.spark.history.status.ZKStateConstant;
+import org.apache.eagle.jpm.util.Constants;
+import org.apache.eagle.jpm.util.resourcefetch.RMResourceFetcher;
+import org.apache.eagle.jpm.util.resourcefetch.ResourceFetcher;
+import org.apache.eagle.jpm.util.resourcefetch.model.AppInfo;
+import backtype.storm.spout.SpoutOutputCollector;
+import backtype.storm.task.TopologyContext;
+import backtype.storm.topology.OutputFieldsDeclarer;
+import backtype.storm.topology.base.BaseRichSpout;
+import backtype.storm.tuple.Fields;
+import backtype.storm.tuple.Values;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import java.util.Calendar;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+public class SparkHistoryJobSpout extends BaseRichSpout {
+    private static final Logger LOG = LoggerFactory.getLogger(SparkHistoryJobSpout.class);
+    private SpoutOutputCollector collector;
+    private JobHistoryZKStateManager zkState;
+    private SparkHistoryJobAppConfig config;
+    private ResourceFetcher rmFetch;
+    private long lastFinishAppTime = 0;
+    private Map<String, Integer> failTimes;
+
+    private static final int FAIL_MAX_TIMES = 5;
+
+    public SparkHistoryJobSpout(SparkHistoryJobAppConfig config) {
+        this.config = config;
+    }
+
+    @Override
+    public void open(Map map, TopologyContext topologyContext, SpoutOutputCollector spoutOutputCollector) {
+        rmFetch = new RMResourceFetcher(config.jobHistoryConfig.rms);
+        this.failTimes = new HashMap<>();
+        this.collector = spoutOutputCollector;
+        this.zkState = new JobHistoryZKStateManager(config);
+        this.lastFinishAppTime = zkState.readLastFinishedTimestamp();
+        zkState.resetApplications();
+    }
+
+    @SuppressWarnings("unchecked")
+    @Override
+    public void nextTuple() {
+        LOG.info("Start to run tuple");
+        try {
+            Calendar calendar = Calendar.getInstance();
+            long fetchTime = calendar.getTimeInMillis();
+            calendar.setTimeInMillis(this.lastFinishAppTime);
+            LOG.info("Last finished time = {}", calendar.getTime());
+            if (fetchTime - this.lastFinishAppTime > this.config.stormConfig.spoutCrawlInterval) {
+                List<AppInfo> appInfos = rmFetch.getResource(Constants.ResourceType.COMPLETE_SPARK_JOB, Long.toString(lastFinishAppTime));
+                //List<AppInfo> appInfos = (null != apps ? (List<AppInfo>)apps.get(0):new ArrayList<AppInfo>());
+                if (appInfos != null) {
+                    LOG.info("Get " + appInfos.size() + " from yarn resource manager.");
+                    for (AppInfo app : appInfos) {
+                        String appId = app.getId();
+                        if (!zkState.hasApplication(appId)) {
+                            zkState.addFinishedApplication(appId, app.getQueue(), app.getState(), app.getFinalStatus(), app.getUser(), app.getName());
+                        }
+                    }
+                }
+                this.lastFinishAppTime = fetchTime;
+                zkState.updateLastUpdateTime(fetchTime);
+            }
+
+            List<String> appIds = zkState.loadApplications(10);
+            for (String appId: appIds) {
+                collector.emit(new Values(appId), appId);
+                LOG.info("emit " + appId);
+                zkState.updateApplicationStatus(appId, ZKStateConstant.AppStatus.SENT_FOR_PARSE);
+            }
+            LOG.info("{} apps sent.", appIds.size());
+
+            if (appIds.isEmpty()) {
+                this.takeRest(60);
+            }
+        } catch (Exception e) {
+            LOG.error("Fail to run next tuple", e);
+        }
+    }
+
+    private void takeRest(int seconds) {
+        try {
+            Thread.sleep(seconds * 1000);
+        } catch (InterruptedException e) {
+            LOG.warn("exception found {}", e);
+        }
+    }
+
+    @Override
+    public void declareOutputFields(OutputFieldsDeclarer outputFieldsDeclarer) {
+        outputFieldsDeclarer.declare(new Fields("appId"));
+    }
+
+    @Override
+    public void fail(Object msgId) {
+        String appId = (String) msgId;
+        int failTimes = 0;
+        if (this.failTimes.containsKey(appId)) {
+            failTimes = this.failTimes.get(appId);
+        }
+        failTimes++;
+        if (failTimes >= FAIL_MAX_TIMES) {
+            this.failTimes.remove(appId);
+            zkState.updateApplicationStatus(appId, ZKStateConstant.AppStatus.FINISHED);
+            LOG.error(String.format("Application %s has failed for over %s times, drop it.", appId, FAIL_MAX_TIMES));
+        } else {
+            this.failTimes.put(appId, failTimes);
+            collector.emit(new Values(appId), appId);
+            zkState.updateApplicationStatus(appId, ZKStateConstant.AppStatus.SENT_FOR_PARSE);
+        }
+    }
+
+    @Override
+    public void ack(Object msgId) {
+        String appId = (String) msgId;
+        if (this.failTimes.containsKey(appId)) {
+            this.failTimes.remove(appId);
+        }
+
+    }
+
+    @Override
+    public void close() {
+        super.close();
+        zkState.close();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/4f4fd0c4/eagle-jpm/eagle-jpm-spark-history/src/main/java/org/apache/eagle/jpm/spark/history/storm/SparkHistoryTopology.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-spark-history/src/main/java/org/apache/eagle/jpm/spark/history/storm/SparkHistoryTopology.java b/eagle-jpm/eagle-jpm-spark-history/src/main/java/org/apache/eagle/jpm/spark/history/storm/SparkHistoryTopology.java
deleted file mode 100644
index 423dbef..0000000
--- a/eagle-jpm/eagle-jpm-spark-history/src/main/java/org/apache/eagle/jpm/spark/history/storm/SparkHistoryTopology.java
+++ /dev/null
@@ -1,81 +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.eagle.jpm.spark.history.storm;
-
-import org.apache.eagle.jpm.spark.history.config.SparkHistoryCrawlConfig;
-import backtype.storm.Config;
-import backtype.storm.LocalCluster;
-import backtype.storm.StormSubmitter;
-import backtype.storm.topology.TopologyBuilder;
-
-public class SparkHistoryTopology {
-
-    private SparkHistoryCrawlConfig sparkHistoryCrawlConfig;
-
-    public SparkHistoryTopology(SparkHistoryCrawlConfig config) {
-        this.sparkHistoryCrawlConfig = config;
-    }
-
-    public TopologyBuilder getBuilder() {
-        TopologyBuilder builder = new TopologyBuilder();
-        String spoutName = "sparkHistoryJobSpout";
-        String boltName = "sparkHistoryJobBolt";
-        com.typesafe.config.Config config = this.sparkHistoryCrawlConfig.getConfig();
-        builder.setSpout(spoutName,
-                new FinishedSparkJobSpout(sparkHistoryCrawlConfig),
-                config.getInt("storm.parallelismConfig." + spoutName)
-        ).setNumTasks(config.getInt("storm.tasks." + spoutName));
-
-        builder.setBolt(boltName,
-                new SparkJobParseBolt(sparkHistoryCrawlConfig),
-                config.getInt("storm.parallelismConfig." + boltName)
-        ).setNumTasks(config.getInt("storm.tasks." + boltName)).shuffleGrouping(spoutName);
-        return builder;
-    }
-
-
-    public static void main(String[] args) {
-        try {
-            SparkHistoryCrawlConfig crawlConfig = new SparkHistoryCrawlConfig();
-
-            Config conf = new Config();
-            conf.setNumWorkers(crawlConfig.stormConfig.workerNo);
-            conf.setMessageTimeoutSecs(crawlConfig.stormConfig.timeoutSec);
-            //conf.setMaxSpoutPending(crawlConfig.stormConfig.spoutPending);
-            //conf.put(Config.TOPOLOGY_DEBUG, true);
-
-
-            if (crawlConfig.stormConfig.mode.equals("local")) {
-                LocalCluster cluster = new LocalCluster();
-                cluster.submitTopology(
-                        crawlConfig.stormConfig.topologyName,
-                        conf,
-                        new SparkHistoryTopology(crawlConfig).getBuilder().createTopology());
-            } else {
-                StormSubmitter.submitTopology(
-                        crawlConfig.stormConfig.topologyName,
-                        conf,
-                        new SparkHistoryTopology(crawlConfig).getBuilder().createTopology());
-            }
-        } catch (Exception e) {
-            e.printStackTrace();
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/4f4fd0c4/eagle-jpm/eagle-jpm-spark-history/src/main/java/org/apache/eagle/jpm/spark/history/storm/SparkJobParseBolt.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-spark-history/src/main/java/org/apache/eagle/jpm/spark/history/storm/SparkJobParseBolt.java b/eagle-jpm/eagle-jpm-spark-history/src/main/java/org/apache/eagle/jpm/spark/history/storm/SparkJobParseBolt.java
deleted file mode 100644
index c515d32..0000000
--- a/eagle-jpm/eagle-jpm-spark-history/src/main/java/org/apache/eagle/jpm/spark/history/storm/SparkJobParseBolt.java
+++ /dev/null
@@ -1,201 +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.eagle.jpm.spark.history.storm;
-
-import org.apache.eagle.jpm.spark.crawl.JHFInputStreamReader;
-import org.apache.eagle.jpm.spark.crawl.SparkApplicationInfo;
-import org.apache.eagle.jpm.spark.crawl.SparkFilesystemInputStreamReaderImpl;
-import org.apache.eagle.jpm.spark.history.config.SparkHistoryCrawlConfig;
-import org.apache.eagle.jpm.spark.history.status.JobHistoryZKStateManager;
-import org.apache.eagle.jpm.spark.history.status.ZKStateConstant;
-import org.apache.eagle.jpm.util.HDFSUtil;
-import org.apache.eagle.jpm.util.resourcefetch.ResourceFetcher;
-import org.apache.eagle.jpm.util.resourcefetch.SparkHistoryServerResourceFetcher;
-import org.apache.eagle.jpm.util.resourcefetch.model.SparkApplication;
-import backtype.storm.task.OutputCollector;
-import backtype.storm.task.TopologyContext;
-import backtype.storm.topology.OutputFieldsDeclarer;
-import backtype.storm.topology.base.BaseRichBolt;
-import backtype.storm.tuple.Tuple;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-import java.util.*;
-
-public class SparkJobParseBolt extends BaseRichBolt {
-
-    private static final Logger LOG = LoggerFactory.getLogger(SparkJobParseBolt.class);
-
-    private OutputCollector collector;
-    private ResourceFetcher historyServerFetcher;
-    private SparkHistoryCrawlConfig config;
-    private JobHistoryZKStateManager zkState;
-    private Configuration hdfsConf;
-
-    public SparkJobParseBolt(SparkHistoryCrawlConfig config) {
-        this.config = config;
-    }
-
-    @Override
-    public void prepare(Map map, TopologyContext topologyContext, OutputCollector outputCollector) {
-        this.collector = outputCollector;
-        this.hdfsConf = new Configuration();
-        this.hdfsConf.set("fs.defaultFS", config.hdfsConfig.endpoint);
-        this.hdfsConf.setBoolean("fs.hdfs.impl.disable.cache", true);
-        this.hdfsConf.set("hdfs.kerberos.principal", config.hdfsConfig.principal);
-        this.hdfsConf.set("hdfs.keytab.file", config.hdfsConfig.keytab);
-        this.historyServerFetcher = new SparkHistoryServerResourceFetcher(config.jobHistoryConfig.historyServerUrl,
-                config.jobHistoryConfig.historyServerUserName, config.jobHistoryConfig.historyServerUserPwd);
-        this.zkState = new JobHistoryZKStateManager(config);
-    }
-
-    @Override
-    public void execute(Tuple tuple) {
-        String appId = tuple.getStringByField("appId");
-        if (!zkState.hasApplication(appId)) {
-            //may already be processed due to some reason
-            collector.ack(tuple);
-            return;
-        }
-
-        try (FileSystem hdfs = HDFSUtil.getFileSystem(this.hdfsConf)) {
-            SparkApplicationInfo info = zkState.getApplicationInfo(appId);
-            //first try to get attempts under the application
-
-            Set<String> inprogressSet = new HashSet<String>();
-            List<String> attemptLogNames = this.getAttemptLogNameList(appId, hdfs, inprogressSet);
-
-            if (attemptLogNames.isEmpty()) {
-                LOG.info("Application:{}( Name:{}, user: {}, queue: {}) not found on history server.",
-                    appId, info.getName(), info.getUser(), info.getQueue());
-            } else {
-                for (String attemptLogName : attemptLogNames) {
-                    String extension = "";
-                    if (inprogressSet.contains(attemptLogName)) {
-                        extension = ".inprogress";
-                    }
-                    LOG.info("Attempt log name: " + attemptLogName + extension);
-
-                    Path attemptFile = getFilePath(attemptLogName, extension);
-                    JHFInputStreamReader reader = new SparkFilesystemInputStreamReaderImpl(config.info.site, info);
-                    reader.read(hdfs.open(attemptFile));
-                }
-            }
-
-            zkState.updateApplicationStatus(appId, ZKStateConstant.AppStatus.FINISHED);
-            LOG.info("Successfully parse application {}", appId);
-            collector.ack(tuple);
-        } catch (Exception e) {
-            LOG.error("Fail to process application {}", appId, e);
-            zkState.updateApplicationStatus(appId, ZKStateConstant.AppStatus.FAILED);
-            collector.fail(tuple);
-        }
-    }
-
-    @Override
-    public void declareOutputFields(OutputFieldsDeclarer outputFieldsDeclarer) {
-
-    }
-
-    private String getAppAttemptLogName(String appId, String attemptId) {
-        if (attemptId.equals("0")) {
-            return appId;
-        }
-        return appId + "_" + attemptId;
-    }
-
-    private Path getFilePath(String appAttemptLogName, String extension) {
-        String attemptLogDir = this.config.hdfsConfig.baseDir + "/" + appAttemptLogName + extension;
-        return new Path(attemptLogDir);
-    }
-
-    private List<String> getAttemptLogNameList(String appId, FileSystem hdfs, Set<String> inprogressSet)
-            throws IOException {
-        List<String> attempts = new ArrayList<String>();
-        SparkApplication app = null;
-        /*try {
-            List apps = this.historyServerFetcher.getResource(Constants.ResourceType.SPARK_JOB_DETAIL, appId);
-            if (apps != null) {
-                app = (SparkApplication) apps.get(0);
-                attempts = app.getAttempts();
-            }
-        } catch (Exception e) {
-            LOG.warn("Fail to get application detail from history server for appId " + appId, e);
-        }*/
-
-
-        if (null == app) {
-            // history server may not have the info, just double check.
-            // TODO: if attemptId is not "1, 2, 3,...", we should change the logic.
-            // Use getResourceManagerVersion() to compare YARN/RM versions.
-            // attemptId might be: "appId_000001"
-            int attemptId = 0;
-
-            boolean exists = true;
-            while (exists) {
-                // For Yarn version 2.4.x
-                // log name: application_1464382345557_269065_1
-                String attemptIdString = Integer.toString(attemptId);
-
-                // For Yarn version >= 2.7,
-                // log name: "application_1468625664674_0003_appattempt_1468625664674_0003_000001"
-//                String attemptIdFormatted = String.format("%06d", attemptId);
-//
-//                // remove "application_" to get the number part of appID.
-//                String sparkAppIdNum = appId.substring(12);
-//                String attemptIdString = "appattempt_" + sparkAppIdNum + "_" + attemptIdFormatted;
-
-                String appAttemptLogName = this.getAppAttemptLogName(appId, attemptIdString);
-                LOG.info("Attempt ID: {}, App Attempt Log: {}", attemptIdString, appAttemptLogName);
-
-                String extension = "";
-                Path attemptFile = getFilePath(appAttemptLogName, extension);
-                extension = ".inprogress";
-                Path inprogressFile = getFilePath(appAttemptLogName, extension);
-                Path logFile = null;
-                // Check if history log exists.
-                if (hdfs.exists(attemptFile)) {
-                    logFile = attemptFile;
-                } else if (hdfs.exists(inprogressFile)) {
-                    logFile = inprogressFile;
-                    inprogressSet.add(appAttemptLogName);
-                } else if (attemptId > 0) {
-                    exists = false;
-                }
-
-                if (logFile != null) {
-                    attempts.add(appAttemptLogName);
-                }
-                attemptId++;
-            }
-        }
-        return attempts;
-    }
-
-    @Override
-    public void cleanup() {
-        super.cleanup();
-        zkState.close();
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/4f4fd0c4/eagle-jpm/eagle-jpm-spark-history/src/main/java/org/apache/eagle/jpm/spark/history/storm/TestHDFS.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-spark-history/src/main/java/org/apache/eagle/jpm/spark/history/storm/TestHDFS.java b/eagle-jpm/eagle-jpm-spark-history/src/main/java/org/apache/eagle/jpm/spark/history/storm/TestHDFS.java
deleted file mode 100644
index f4284e1..0000000
--- a/eagle-jpm/eagle-jpm-spark-history/src/main/java/org/apache/eagle/jpm/spark/history/storm/TestHDFS.java
+++ /dev/null
@@ -1,47 +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.eagle.jpm.spark.history.storm;
-
-import org.apache.eagle.jpm.spark.history.config.SparkHistoryCrawlConfig;
-import org.apache.eagle.jpm.util.HDFSUtil;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-public class TestHDFS {
-    private static final Logger LOG = LoggerFactory.getLogger(TestHDFS.class);
-
-    public static void main(String[] args) throws Exception {
-        SparkHistoryCrawlConfig config = new SparkHistoryCrawlConfig();
-
-        Configuration conf  = new Configuration();
-        conf.set("fs.defaultFS", config.hdfsConfig.endpoint);
-        conf.set("hdfs.kerberos.principal", config.hdfsConfig.principal);
-        conf.set("hdfs.keytab.file", config.hdfsConfig.keytab);
-
-        FileSystem hdfs = HDFSUtil.getFileSystem(conf);
-        Path path = new Path("/logs/spark-events/local-1463002514438");
-        boolean exists = hdfs.exists(path);
-        LOG.info("File exists:{}", exists);
-
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/4f4fd0c4/eagle-jpm/eagle-jpm-spark-history/src/main/resources/META-INF/providers/org.apache.eagle.jpm.spark.history.SparkHistoryJobAppProvider.xml
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-spark-history/src/main/resources/META-INF/providers/org.apache.eagle.jpm.spark.history.SparkHistoryJobAppProvider.xml b/eagle-jpm/eagle-jpm-spark-history/src/main/resources/META-INF/providers/org.apache.eagle.jpm.spark.history.SparkHistoryJobAppProvider.xml
new file mode 100644
index 0000000..26842b8
--- /dev/null
+++ b/eagle-jpm/eagle-jpm-spark-history/src/main/resources/META-INF/providers/org.apache.eagle.jpm.spark.history.SparkHistoryJobAppProvider.xml
@@ -0,0 +1,271 @@
+<?xml version="1.0" encoding="UTF-8" ?>
+<!--
+  ~ Licensed to the Apache Software Foundation (ASF) under one or more
+  ~ contributor license agreements.  See the NOTICE file distributed with
+  ~ this work for additional information regarding copyright ownership.
+  ~ The ASF licenses this file to You under the Apache License, Version 2.0
+  ~ (the "License"); you may not use this file except in compliance with
+  ~ the License.  You may obtain a copy of the License at
+  ~
+  ~    http://www.apache.org/licenses/LICENSE-2.0
+  ~
+  ~ Unless required by applicable law or agreed to in writing, software
+  ~ distributed under the License is distributed on an "AS IS" BASIS,
+  ~ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  ~ See the License for the specific language governing permissions and
+  ~ limitations under the License.
+  -->
+
+<application>
+    <type>SPARK_HISTORY_JOB_APP</type>
+    <name>Spark History Job Monitoring</name>
+    <version>0.5.0-incubating</version>
+    <appClass>org.apache.eagle.jpm.spark.history.SparkHistoryJobApp</appClass>
+    <viewPath>/apps/jpm</viewPath>
+    <configuration>
+        <!-- org.apache.eagle.jpm.spark.history.SparkHistoryJobAppConfig -->
+        <property>
+            <name>basic.cluster</name>
+            <displayName>cluster</displayName>
+            <description>Cluster Name</description>
+            <value>sandbox</value>
+        </property>
+        <property>
+            <name>basic.dataCenter</name>
+            <displayName>dataCenter</displayName>
+            <description>Data Center</description>
+            <value>sandbox</value>
+        </property>
+        <property>
+            <name>basic.jobConf.additional.info</name>
+            <displayName>jobConf.additional.info</displayName>
+            <description>Additional info in Job Configs</description>
+            <value></value>
+        </property>
+        <property>
+            <name>dataSourceConfig.zkQuorum</name>
+            <displayName>zkQuorum</displayName>
+            <description>Zookeeper Quorum</description>
+            <value>sandbox.hortonworks.com:2181</value>
+        </property>
+        <property>
+            <name>dataSourceConfig.zkRoot</name>
+            <displayName>zkRoot</displayName>
+            <description>Zookeeper Root</description>
+            <value>/sparkHistoryJob</value>
+        </property>
+        <property>
+            <name>dataSourceConfig.zkPort</name>
+            <displayName>zkPort</displayName>
+            <description>Zookeeper Port</description>
+            <value>2181</value>
+        </property>
+        <property>
+            <name>dataSourceConfig.zkSessionTimeoutMs</name>
+            <displayName>zkSessionTimeoutMs</displayName>
+            <description>Zookeeper session timeoutMs</description>
+            <value>15000</value>
+        </property>
+        <property>
+            <name>zookeeperConfig.zkRetryTimes</name>
+            <displayName>zkRetryTimes</displayName>
+            <description>zookeeperConfig.zkRetryTimes</description>
+            <value>3</value>
+        </property>
+        <property>
+            <name>zookeeperConfig.zkRetryInterval</name>
+            <displayName>zkRetryInterval</displayName>
+            <description>zookeeperConfig.zkRetryInterval</description>
+            <value>20000</value>
+        </property>
+        <property>
+            <name>dataSourceConfig.spark.history.server.url</name>
+            <displayName>spark.history.server.url</displayName>
+            <description>Spark History Server URL</description>
+            <value>http://sandbox.hortonworks.com:18080</value>
+        </property>
+        <property>
+            <name>dataSourceConfig.spark.history.server.username</name>
+            <displayName>spark.history.server.username</displayName>
+            <description>Spark History Server Auth Username</description>
+            <value></value>
+        </property>
+        <property>
+            <name>dataSourceConfig.spark.history.server.password</name>
+            <displayName>spark.history.server.password</displayName>
+            <description>Spark History Server Auth Password</description>
+            <value></value>
+        </property>
+        <property>
+            <name>eagleProps.eagle.service.host</name>
+            <description>eagleProps.eagle.service.host</description>
+            <value>sandbox.hortonworks.com</value>
+        </property>
+        <property>
+            <name>eagleProps.eagle.service.port</name>
+            <description>eagleProps.eagle.service.port</description>
+            <value>9099</value>
+        </property>
+        <property>
+            <name>eagleProps.eagle.service.username</name>
+            <description>eagleProps.eagle.service.username</description>
+            <value>admin</value>
+        </property>
+        <property>
+            <name>eagleProps.eagle.service.password</name>
+            <description>eagleProps.eagle.service.password</description>
+            <value>secret</value>
+        </property>
+        <property>
+            <name>eagleProps.eagle.service.read.timeout</name>
+            <displayName>eagleProps.eagle.service.read.timeout</displayName>
+            <description>The maximum amount of time (in seconds) the app is trying to read from eagle service</description>
+            <value>2</value>
+        </property>
+        <property>
+            <name>eagleProps.eagleService.maxFlushNum</name>
+            <displayName>eagleProps.eagleService.maxFlushNum</displayName>
+            <value>500</value>
+        </property>
+        <property>
+            <name>dataSourceConfig.hdfs.eventLog</name>
+            <displayName>dataSourceConfig.hdfs.eventLog</displayName>
+            <value>/spark-history</value>
+        </property>
+        <property>
+            <name>dataSourceConfig.hdfs.endpoint</name>
+            <displayName>dataSourceConfig.hdfs.endpoint</displayName>
+            <value>hdfs://sandbox.hortonworks.com:8020</value>
+        </property>
+        <property>
+            <name>dataSourceConfig.hdfs.keytab</name>
+            <displayName>dataSourceConfig.hdfs.keytab</displayName>
+            <value></value>
+        </property>
+        <property>
+            <name>dataSourceConfig.hdfs.principal</name>
+            <displayName>dataSourceConfig.hdfs.principal</displayName>
+            <value></value>
+        </property>
+        <property>
+            <name>dataSourceConfig.rmUrl</name>
+            <displayName>dataSourceConfig.rmUrl</displayName>
+            <value>http://sandbox.hortonworks.com:8088</value>
+        </property>
+        <property>
+            <name>storm.mode</name>
+            <displayName>mode</displayName>
+            <description>Storm Mode: local or cluster</description>
+            <value>local</value>
+        </property>
+        <property>
+            <name>storm.worker.num</name>
+            <displayName>worker.num</displayName>
+            <description>The number of workers</description>
+            <value>2</value>
+        </property>
+        <property>
+            <name>name</name>
+            <displayName>name</displayName>
+            <description>Name of the topology</description>
+            <value>sparkHistoryJob</value>
+        </property>
+        <property>
+            <name>storm.messageTimeoutSec</name>
+            <displayName>messageTimeoutSec</displayName>
+            <description>Message timeout (in seconds)</description>
+            <value>3000</value>
+        </property>
+        <property>
+            <name>storm.pendingSpout</name>
+            <displayName>pendingSpout</displayName>
+            <value>1000</value>
+        </property>
+        <property>
+            <name>storm.spoutCrawlInterval</name>
+            <displayName>spoutCrawlInterval</displayName>
+            <description>Spout crawl interval (in milliseconds)</description>
+            <value>10000</value>
+        </property>
+        <property>
+            <name>storm.parallelismConfig.sparkHistoryJobFetchSpout</name>
+            <displayName>parallelismConfig.sparkHistoryJobFetchSpout</displayName>
+            <description>Parallelism of sparkHistoryJobFetchSpout </description>
+            <value>1</value>
+        </property>
+        <property>
+            <name>storm.tasks.sparkHistoryJobFetchSpout</name>
+            <displayName>tasks.sparkHistoryJobFetchSpout</displayName>
+            <description>Tasks Num of sparkHistoryJobFetchSpout </description>
+            <value>4</value>
+        </property>
+        <property>
+            <name>storm.parallelismConfig.sparkHistoryJobParseBolt</name>
+            <displayName>parallelismConfig.sparkHistoryJobParseBolt</displayName>
+            <description>Parallelism of sparkHistoryJobParseBolt </description>
+            <value>1</value>
+        </property>
+        <property>
+            <name>storm.tasks.sparkHistoryJobParseBolt</name>
+            <displayName>parallelismConfig.sparkHistoryJobParseBolt</displayName>
+            <description>Tasks Num of sparkHistoryJobParseBolt</description>
+            <value>4</value>
+        </property>
+        <property>
+            <name>spark.defaultVal.spark.executor.memory</name>
+            <displayName>spark.executor.memory</displayName>
+            <value>1g</value>
+        </property>
+        <property>
+            <name>spark.defaultVal.spark.driver.memory</name>
+            <displayName>spark.driver.memory</displayName>
+            <value>1g</value>
+        </property>
+        <property>
+            <name>spark.defaultVal.spark.driver.cores</name>
+            <displayName>spark.driver.cores</displayName>
+            <value>1</value>
+        </property>
+        <property>
+            <name>spark.defaultVal.spark.executor.cores</name>
+            <displayName>spark.executor.cores</displayName>
+            <value>1</value>
+        </property>
+        <property>
+            <name>spark.defaultVal.spark.yarn.am.memory</name>
+            <displayName>spark.yarn.am.memory</displayName>
+            <value>512m</value>
+        </property>
+        <property>
+            <name>spark.defaultVal.spark.yarn.am.cores</name>
+            <displayName>spark.yarn.am.cores</displayName>
+            <value>1</value>
+        </property>
+        <property>
+            <name>spark.defaultVal.spark.yarn.executor.memoryOverhead.factor</name>
+            <displayName>spark.yarn.executor.memoryOverhead.factor</displayName>
+            <value>10</value>
+        </property>
+        <property>
+            <name>spark.defaultVal.spark.yarn.driver.memoryOverhead.factor</name>
+            <displayName>spark.yarn.driver.memoryOverhead.factor</displayName>
+            <value>10</value>
+        </property>
+        <property>
+            <name>spark.defaultVal.spark.yarn.am.memoryOverhead.factor</name>
+            <displayName>spark.yarn.am.memoryOverhead.factor</displayName>
+            <value>10</value>
+        </property>
+        <property>
+            <name>spark.defaultVal.spark.yarn.overhead.min</name>
+            <displayName>spark.yarn.overhead.min</displayName>
+            <value>384m</value>
+        </property>
+    </configuration>
+    <docs>
+        <install>
+        </install>
+        <uninstall>
+        </uninstall>
+    </docs>
+</application>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/4f4fd0c4/eagle-jpm/eagle-jpm-spark-history/src/main/resources/META-INF/services/org.apache.eagle.app.spi.ApplicationProvider
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-spark-history/src/main/resources/META-INF/services/org.apache.eagle.app.spi.ApplicationProvider b/eagle-jpm/eagle-jpm-spark-history/src/main/resources/META-INF/services/org.apache.eagle.app.spi.ApplicationProvider
new file mode 100644
index 0000000..7adb50c
--- /dev/null
+++ b/eagle-jpm/eagle-jpm-spark-history/src/main/resources/META-INF/services/org.apache.eagle.app.spi.ApplicationProvider
@@ -0,0 +1,16 @@
+# 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.
+
+org.apache.eagle.jpm.spark.history.SparkHistoryJobAppProvider
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/4f4fd0c4/eagle-jpm/eagle-jpm-spark-history/src/main/resources/application.conf
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-spark-history/src/main/resources/application.conf b/eagle-jpm/eagle-jpm-spark-history/src/main/resources/application.conf
index 65aaa36..289c6f7 100644
--- a/eagle-jpm/eagle-jpm-spark-history/src/main/resources/application.conf
+++ b/eagle-jpm/eagle-jpm-spark-history/src/main/resources/application.conf
@@ -18,14 +18,14 @@
   "basic":{
     "cluster":"sandbox",
     "datacenter":"sandbox",
-    jobConf.additional.info: []
+    jobConf.additional.info: ""
   },
   "eagleProps":{
     eagle.service.host:"sandbox.hortonworks.com",
     eagle.service.port: 9099,
-    eagle.service.userName: "admin",
-    eagle.service.pwd : "secret",
-    eagle.service.read_timeout : 2
+    eagle.service.username: "admin",
+    eagle.service.password : "secret",
+    eagle.service.read.timeout : 2
   },
   "dataSourceConfig":{
     "zkQuorum" : "sandbox.hortonworks.com:2181",
@@ -35,29 +35,29 @@
     "zkRetryInterval" : 20000,
     spark.history.server.url : "http://sandbox.hortonworks.com:18080",
     spark.history.server.username : "",
-    spark.history.server.pwd : "",
-    rm.url:["http://sandbox.hortonworks.com:8088"] ,
+    spark.history.server.password : "",
+    rm.url: "http://sandbox.hortonworks.com:8088",
     "hdfs": {
-      "baseDir": "/spark-history",
+      "eventLog": "/spark-history",
       "endPoint": "hdfs://sandbox.hortonworks.com:8020",
       "principal": "",
       "keytab" : ""
-      }
+    }
   },
   "storm":{
+    worker.num: 2,
     "mode": "local",
-    "workerNo": 2,
-    "name":"sparkHistory",
-    "messageTimeoutSec":  3000,
+    "name":"sparkHistoryJob",
+    "messageTimeoutSec": 3000,
     "pendingSpout": 1000,
     "spoutCrawlInterval": 10000,#in ms
     "parallelismConfig" : {
-      "sparkHistoryJobSpout" : 1,
-      "sparkHistoryJobBolt" : 6
+      "sparkHistoryJobFetchSpout" : 1,
+      "sparkHistoryJobParseBolt" : 4
     },
     "tasks" : {
-      "sparkHistoryJobSpout" : 1,
-      "sparkHistoryJobBolt" : 6
+      "sparkHistoryJobFetchSpout" : 1,
+      "sparkHistoryJobParseBolt" : 4
     }
   },
   "spark":{
@@ -73,5 +73,7 @@
       spark.yarn.am.memoryOverhead.factor: 10,
       spark.yarn.overhead.min: "384m"
     }
-  }
+  },
+  "appId": "sparkHistoryJob",
+  "mode": "LOCAL"
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/4f4fd0c4/eagle-jpm/eagle-jpm-spark-history/src/test/java/SparkHistoryJobAppProviderTest.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-spark-history/src/test/java/SparkHistoryJobAppProviderTest.java b/eagle-jpm/eagle-jpm-spark-history/src/test/java/SparkHistoryJobAppProviderTest.java
new file mode 100644
index 0000000..cf6e932
--- /dev/null
+++ b/eagle-jpm/eagle-jpm-spark-history/src/test/java/SparkHistoryJobAppProviderTest.java
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import com.google.inject.Inject;
+import org.apache.eagle.app.test.ApplicationSimulator;
+import org.apache.eagle.app.test.ApplicationTestBase;
+import org.apache.eagle.jpm.spark.history.SparkHistoryJobAppProvider;
+import org.junit.Test;
+
+public class SparkHistoryJobAppProviderTest extends ApplicationTestBase {
+    @Inject
+    ApplicationSimulator simulator;
+
+    @Test
+    public void testRunWithProvider(){
+        simulator.start(SparkHistoryJobAppProvider.class);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/4f4fd0c4/eagle-jpm/eagle-jpm-spark-running/src/main/java/org/apache/eagle/jpm/spark/running/SparkRunningJobAppProvider.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-spark-running/src/main/java/org/apache/eagle/jpm/spark/running/SparkRunningJobAppProvider.java b/eagle-jpm/eagle-jpm-spark-running/src/main/java/org/apache/eagle/jpm/spark/running/SparkRunningJobAppProvider.java
index 3d20af7..dc79b15 100644
--- a/eagle-jpm/eagle-jpm-spark-running/src/main/java/org/apache/eagle/jpm/spark/running/SparkRunningJobAppProvider.java
+++ b/eagle-jpm/eagle-jpm-spark-running/src/main/java/org/apache/eagle/jpm/spark/running/SparkRunningJobAppProvider.java
@@ -23,4 +23,4 @@ public class SparkRunningJobAppProvider extends AbstractApplicationProvider<Spar
     public SparkRunningJobApp getApplication() {
         return new SparkRunningJobApp();
     }
-}
\ No newline at end of file
+}


[46/52] [abbrv] incubator-eagle git commit: [EAGLE-520] Fix and decouple co-processor from eagle aggreation query service

Posted by yo...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/21187b55/eagle-core/eagle-query/eagle-storage-hbase/src/test/java/org/apache/eagle/storage/hbase/aggregate/coprocessor/TestGroupAggregateTimeSeriesClient.java
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-query/eagle-storage-hbase/src/test/java/org/apache/eagle/storage/hbase/aggregate/coprocessor/TestGroupAggregateTimeSeriesClient.java b/eagle-core/eagle-query/eagle-storage-hbase/src/test/java/org/apache/eagle/storage/hbase/aggregate/coprocessor/TestGroupAggregateTimeSeriesClient.java
index d61c974..f93b68e 100755
--- a/eagle-core/eagle-query/eagle-storage-hbase/src/test/java/org/apache/eagle/storage/hbase/aggregate/coprocessor/TestGroupAggregateTimeSeriesClient.java
+++ b/eagle-core/eagle-query/eagle-storage-hbase/src/test/java/org/apache/eagle/storage/hbase/aggregate/coprocessor/TestGroupAggregateTimeSeriesClient.java
@@ -1,220 +1,220 @@
-/*
- * 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.eagle.storage.hbase.aggregate.coprocessor;
-
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.HashMap;
-import java.util.List;
-
-import org.apache.eagle.common.config.EagleConfigFactory;
-import org.junit.Assert;
-
-import org.apache.eagle.storage.hbase.query.coprocessor.AggregateClient;
-import org.apache.hadoop.hbase.client.HTableInterface;
-import org.apache.hadoop.hbase.client.Scan;
-import org.apache.hadoop.io.BytesWritable;
-import org.apache.hadoop.io.DoubleWritable;
-import org.junit.Before;
-import org.junit.Ignore;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import org.apache.eagle.log.entity.GenericEntityWriter;
-import org.apache.eagle.log.entity.meta.EntityDefinition;
-import org.apache.eagle.log.entity.meta.EntityDefinitionManager;
-import org.apache.eagle.log.entity.test.TestTimeSeriesAPIEntity;
-import org.apache.eagle.query.ListQueryCompiler;
-import org.apache.eagle.query.aggregate.AggregateFunctionType;
-import org.apache.eagle.query.aggregate.raw.GroupbyKey;
-import org.apache.eagle.query.aggregate.raw.GroupbyKeyValue;
-import org.apache.eagle.query.aggregate.raw.GroupbyValue;
-import org.apache.eagle.service.hbase.TestHBaseBase;
-import org.apache.eagle.storage.hbase.query.coprocessor.impl.AggregateClientImpl;
-
-/**
- * @since : 11/10/14,2014
- */
-@Ignore
-public class TestGroupAggregateTimeSeriesClient extends TestHBaseBase {
-
-	private final static Logger LOG = LoggerFactory.getLogger(TestGroupAggregateTimeSeriesClient.class);
-
-	HTableInterface table;
-	long startTime;
-	long endTime;
-	List<String> rowkeys;
-	AggregateClient client;
-	Scan scan;
-	EntityDefinition ed;
-
-	@Before
-	public void setUp() throws IllegalAccessException, InstantiationException {
-		ed = EntityDefinitionManager.getEntityDefinitionByEntityClass(TestTimeSeriesAPIEntity.class);
-		hbase.createTable("unittest", "f");
-		table = EagleConfigFactory.load().getHTable("unittest");
-		startTime = System.currentTimeMillis();
-		try {
-			rowkeys = prepareData(1000);
-		} catch (Exception e) {
-			e.printStackTrace();
-			Assert.fail(e.getMessage());
-		}
-		endTime = System.currentTimeMillis();
-
-		client = new AggregateClientImpl();
-		scan = new Scan();
-		ListQueryCompiler compiler = null;
-		try {
-			compiler = new ListQueryCompiler("TestTimeSeriesAPIEntity[@cluster=\"test4UT\" and @datacenter = \"dc1\"]{@field1,@field2}");
-		} catch (Exception e) {
-			Assert.fail(e.getMessage());
-		}
-		scan.setFilter(compiler.filter());
-//		scan.setStartRow(EagleBase64Wrapper.decode(rowkeys.get(0)));
-//		scan.setStopRow(EagleBase64Wrapper.decode(rowkeys.get(rowkeys.size()-1)));
-	}
-
-	private List<String> prepareData(int count) throws Exception {
-		List<TestTimeSeriesAPIEntity> list = new ArrayList<TestTimeSeriesAPIEntity>();
-
-		if (ed == null) {
-			EntityDefinitionManager.registerEntity(TestTimeSeriesAPIEntity.class);
-			ed = EntityDefinitionManager.getEntityDefinitionByEntityClass(TestTimeSeriesAPIEntity.class);
-		}
-
-		for(int i=0;i<count;i++){
-			TestTimeSeriesAPIEntity e = new TestTimeSeriesAPIEntity();
-			e.setTimestamp(System.currentTimeMillis());
-			e.setField1(1);
-			e.setField2(2);
-			e.setField3(3);
-			e.setField4(4L);
-			e.setField5(5.0);
-			e.setField6(5.0);
-			e.setField7("7");
-			e.setTags(new HashMap<String, String>());
-			e.getTags().put("cluster", "test4UT");
-			e.getTags().put("datacenter", "dc1");
-			e.getTags().put("index", ""+i);
-			e.getTags().put("jobId", "job_"+System.currentTimeMillis());
-			list.add(e);
-		}
-
-		GenericEntityWriter writer = new GenericEntityWriter(ed.getService());
-		List<String> result = writer.write(list);
-		return result;
-	}
-
-
-	//@Test
-	public void testGroupTimeSeriesAggCountClient(){
-		try {
-			List<GroupbyKeyValue> result = client.aggregate(table,ed,scan, Arrays.asList("cluster","datacenter"),Arrays.asList(AggregateFunctionType.count),Arrays.asList("count"),true,startTime,System.currentTimeMillis(),10).getKeyValues();
-			if(LOG.isDebugEnabled()) LOG.debug("COUNT");
-			logGroupbyKeyValue(result);
-			Assert.assertNotNull(result);
-		} catch (Exception e) {
-			e.printStackTrace();
-			Assert.fail(e.getMessage());
-		}
-	}
-
-	//@Test
-	public void testGroupTimeSeriesAggMaxClient(){
-		try {
-			List<GroupbyKeyValue> result = client.aggregate(table,ed,scan, Arrays.asList("cluster","datacenter"),Arrays.asList(AggregateFunctionType.max),Arrays.asList("field2"),true,startTime,System.currentTimeMillis(),10).getKeyValues();
-			if(LOG.isDebugEnabled()) LOG.debug("MAX");
-			logGroupbyKeyValue(result);
-			Assert.assertNotNull(result);
-		} catch (Exception e) {
-			e.printStackTrace();
-			Assert.fail(e.getMessage());
-		}
-	}
-
-	//@Test
-	public void testGroupTimeSeriesAggMinClient(){
-		try {
-			List<GroupbyKeyValue> result = client.aggregate(table,ed,scan, Arrays.asList("cluster","datacenter"),Arrays.asList(AggregateFunctionType.min),Arrays.asList("field2"),true,startTime,System.currentTimeMillis(),10).getKeyValues();
-			if(LOG.isDebugEnabled()) LOG.debug("MIN");
-			logGroupbyKeyValue(result);
-			Assert.assertNotNull(result);
-		} catch (Exception e) {
-			e.printStackTrace();
-			Assert.fail(e.getMessage());
-		}
-	}
-
-	//@Test
-	public void testGroupTimeSeriesAggAvgClient(){
-		try {
-			List<GroupbyKeyValue> result = client.aggregate(table,ed,scan, Arrays.asList("cluster","datacenter"),Arrays.asList(AggregateFunctionType.min),Arrays.asList("field2"),true,startTime,System.currentTimeMillis(),10).getKeyValues();
-			if(LOG.isDebugEnabled()) LOG.debug("MIN");
-			logGroupbyKeyValue(result);
-			Assert.assertNotNull(result);
-		} catch (Exception e) {
-			e.printStackTrace();
-			Assert.fail(e.getMessage());
-		}
-	}
-
-	//@Test
-	public void testGroupTimeSeriesAggSumClient(){
-		try {
-			List<GroupbyKeyValue> result = client.aggregate(table,ed,scan, Arrays.asList("cluster","datacenter"),Arrays.asList(AggregateFunctionType.sum),Arrays.asList("field2"),true,startTime,System.currentTimeMillis(),10).getKeyValues();
-			if(LOG.isDebugEnabled()) LOG.debug("SUM");
-			logGroupbyKeyValue(result);
-			Assert.assertNotNull(result);
-		} catch (Exception e) {
-			e.printStackTrace();
-			Assert.fail(e.getMessage());
-		}
-	}
-
-	//@Test
-	public void testGroupTimeSeriesAggMultipleClient(){
-		try {
-			List<GroupbyKeyValue> result = client.aggregate(table,ed,scan,
-					Arrays.asList("cluster","datacenter"),
-					Arrays.asList(AggregateFunctionType.max,AggregateFunctionType.min,AggregateFunctionType.avg,AggregateFunctionType.sum,AggregateFunctionType.count),
-					Arrays.asList("field2","field2","field2","field2","field2"),true,startTime,System.currentTimeMillis(),16).getKeyValues();
-			if(LOG.isDebugEnabled()) LOG.debug("MUTILPLE");
-			logGroupbyKeyValue(result);
-			Assert.assertNotNull(result);
-		} catch (Exception e) {
-			e.printStackTrace();
-			Assert.fail(e.getMessage());
-		}
-	}
-
-	private void logGroupbyKeyValue(List<GroupbyKeyValue> keyValues){
-		for(GroupbyKeyValue keyValue:keyValues){
-			GroupbyKey key = keyValue.getKey();
-			List<String> keys = new ArrayList<String>();
-			for(BytesWritable bytes:key.getValue()){
-				keys.add(new String(bytes.copyBytes()));
-			}
-			List<Double> vals = new ArrayList<Double>();
-			GroupbyValue val = keyValue.getValue();
-			for(DoubleWritable dw:val.getValue()){
-				vals.add(dw.get());
-			}
-			if(LOG.isDebugEnabled()) LOG.debug("KEY: " + keys + ", VALUE: " + vals);
-		}
-	}
-}
+/*
+ * 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.eagle.storage.hbase.aggregate.coprocessor;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+
+import org.apache.eagle.common.config.EagleConfigFactory;
+import org.junit.Assert;
+
+import org.apache.eagle.storage.hbase.query.coprocessor.AggregateClient;
+import org.apache.hadoop.hbase.client.HTableInterface;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.io.BytesWritable;
+import org.apache.hadoop.io.DoubleWritable;
+import org.junit.Before;
+import org.junit.Ignore;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.eagle.log.entity.GenericEntityWriter;
+import org.apache.eagle.log.entity.meta.EntityDefinition;
+import org.apache.eagle.log.entity.meta.EntityDefinitionManager;
+import org.apache.eagle.log.entity.test.TestTimeSeriesAPIEntity;
+import org.apache.eagle.query.ListQueryCompiler;
+import org.apache.eagle.query.aggregate.AggregateFunctionType;
+import org.apache.eagle.query.aggregate.raw.GroupbyKey;
+import org.apache.eagle.query.aggregate.raw.GroupbyKeyValue;
+import org.apache.eagle.query.aggregate.raw.GroupbyValue;
+import org.apache.eagle.service.hbase.TestHBaseBase;
+import org.apache.eagle.storage.hbase.query.coprocessor.impl.AggregateClientImpl;
+
+/**
+ * @since : 11/10/14,2014
+ */
+@Ignore
+public class TestGroupAggregateTimeSeriesClient extends TestHBaseBase {
+
+    private final static Logger LOG = LoggerFactory.getLogger(TestGroupAggregateTimeSeriesClient.class);
+
+    HTableInterface table;
+    long startTime;
+    long endTime;
+    List<String> rowkeys;
+    AggregateClient client;
+    Scan scan;
+    EntityDefinition ed;
+
+    @Before
+    public void setUp() throws IllegalAccessException, InstantiationException {
+        ed = EntityDefinitionManager.getEntityDefinitionByEntityClass(TestTimeSeriesAPIEntity.class);
+        hbase.createTable("unittest", "f");
+        table = EagleConfigFactory.load().getHTable("unittest");
+        startTime = System.currentTimeMillis();
+        try {
+            rowkeys = prepareData(1000);
+        } catch (Exception e) {
+            e.printStackTrace();
+            Assert.fail(e.getMessage());
+        }
+        endTime = System.currentTimeMillis();
+
+        client = new AggregateClientImpl();
+        scan = new Scan();
+        ListQueryCompiler compiler = null;
+        try {
+            compiler = new ListQueryCompiler("TestTimeSeriesAPIEntity[@cluster=\"test4UT\" and @datacenter = \"dc1\"]{@field1,@field2}");
+        } catch (Exception e) {
+            Assert.fail(e.getMessage());
+        }
+        scan.setFilter(compiler.filter());
+//		scan.setStartRow(EagleBase64Wrapper.decode(rowkeys.get(0)));
+//		scan.setStopRow(EagleBase64Wrapper.decode(rowkeys.get(rowkeys.size()-1)));
+    }
+
+    private List<String> prepareData(int count) throws Exception {
+        List<TestTimeSeriesAPIEntity> list = new ArrayList<TestTimeSeriesAPIEntity>();
+
+        if (ed == null) {
+            EntityDefinitionManager.registerEntity(TestTimeSeriesAPIEntity.class);
+            ed = EntityDefinitionManager.getEntityDefinitionByEntityClass(TestTimeSeriesAPIEntity.class);
+        }
+
+        for (int i = 0; i < count; i++) {
+            TestTimeSeriesAPIEntity e = new TestTimeSeriesAPIEntity();
+            e.setTimestamp(System.currentTimeMillis());
+            e.setField1(1);
+            e.setField2(2);
+            e.setField3(3);
+            e.setField4(4L);
+            e.setField5(5.0);
+            e.setField6(5.0);
+            e.setField7("7");
+            e.setTags(new HashMap<String, String>());
+            e.getTags().put("cluster", "test4UT");
+            e.getTags().put("datacenter", "dc1");
+            e.getTags().put("index", "" + i);
+            e.getTags().put("jobId", "job_" + System.currentTimeMillis());
+            list.add(e);
+        }
+
+        GenericEntityWriter writer = new GenericEntityWriter(ed.getService());
+        List<String> result = writer.write(list);
+        return result;
+    }
+
+
+    //@Test
+    public void testGroupTimeSeriesAggCountClient() {
+        try {
+            List<GroupbyKeyValue> result = client.aggregate(table, ed, scan, Arrays.asList("cluster", "datacenter"), Arrays.asList(AggregateFunctionType.count), Arrays.asList("count"), true, startTime, System.currentTimeMillis(), 10).getKeyValues();
+            if (LOG.isDebugEnabled()) LOG.debug("COUNT");
+            logGroupbyKeyValue(result);
+            Assert.assertNotNull(result);
+        } catch (Exception e) {
+            e.printStackTrace();
+            Assert.fail(e.getMessage());
+        }
+    }
+
+    //@Test
+    public void testGroupTimeSeriesAggMaxClient() {
+        try {
+            List<GroupbyKeyValue> result = client.aggregate(table, ed, scan, Arrays.asList("cluster", "datacenter"), Arrays.asList(AggregateFunctionType.max), Arrays.asList("field2"), true, startTime, System.currentTimeMillis(), 10).getKeyValues();
+            if (LOG.isDebugEnabled()) LOG.debug("MAX");
+            logGroupbyKeyValue(result);
+            Assert.assertNotNull(result);
+        } catch (Exception e) {
+            e.printStackTrace();
+            Assert.fail(e.getMessage());
+        }
+    }
+
+    //@Test
+    public void testGroupTimeSeriesAggMinClient() {
+        try {
+            List<GroupbyKeyValue> result = client.aggregate(table, ed, scan, Arrays.asList("cluster", "datacenter"), Arrays.asList(AggregateFunctionType.min), Arrays.asList("field2"), true, startTime, System.currentTimeMillis(), 10).getKeyValues();
+            if (LOG.isDebugEnabled()) LOG.debug("MIN");
+            logGroupbyKeyValue(result);
+            Assert.assertNotNull(result);
+        } catch (Exception e) {
+            e.printStackTrace();
+            Assert.fail(e.getMessage());
+        }
+    }
+
+    //@Test
+    public void testGroupTimeSeriesAggAvgClient() {
+        try {
+            List<GroupbyKeyValue> result = client.aggregate(table, ed, scan, Arrays.asList("cluster", "datacenter"), Arrays.asList(AggregateFunctionType.min), Arrays.asList("field2"), true, startTime, System.currentTimeMillis(), 10).getKeyValues();
+            if (LOG.isDebugEnabled()) LOG.debug("MIN");
+            logGroupbyKeyValue(result);
+            Assert.assertNotNull(result);
+        } catch (Exception e) {
+            e.printStackTrace();
+            Assert.fail(e.getMessage());
+        }
+    }
+
+    //@Test
+    public void testGroupTimeSeriesAggSumClient() {
+        try {
+            List<GroupbyKeyValue> result = client.aggregate(table, ed, scan, Arrays.asList("cluster", "datacenter"), Arrays.asList(AggregateFunctionType.sum), Arrays.asList("field2"), true, startTime, System.currentTimeMillis(), 10).getKeyValues();
+            if (LOG.isDebugEnabled()) LOG.debug("SUM");
+            logGroupbyKeyValue(result);
+            Assert.assertNotNull(result);
+        } catch (Exception e) {
+            e.printStackTrace();
+            Assert.fail(e.getMessage());
+        }
+    }
+
+    //@Test
+    public void testGroupTimeSeriesAggMultipleClient() {
+        try {
+            List<GroupbyKeyValue> result = client.aggregate(table, ed, scan,
+                    Arrays.asList("cluster", "datacenter"),
+                    Arrays.asList(AggregateFunctionType.max, AggregateFunctionType.min, AggregateFunctionType.avg, AggregateFunctionType.sum, AggregateFunctionType.count),
+                    Arrays.asList("field2", "field2", "field2", "field2", "field2"), true, startTime, System.currentTimeMillis(), 16).getKeyValues();
+            if (LOG.isDebugEnabled()) LOG.debug("MUTILPLE");
+            logGroupbyKeyValue(result);
+            Assert.assertNotNull(result);
+        } catch (Exception e) {
+            e.printStackTrace();
+            Assert.fail(e.getMessage());
+        }
+    }
+
+    private void logGroupbyKeyValue(List<GroupbyKeyValue> keyValues) {
+        for (GroupbyKeyValue keyValue : keyValues) {
+            GroupbyKey key = keyValue.getKey();
+            List<String> keys = new ArrayList<String>();
+            for (BytesWritable bytes : key.getValue()) {
+                keys.add(new String(bytes.copyBytes()));
+            }
+            List<Double> vals = new ArrayList<Double>();
+            GroupbyValue val = keyValue.getValue();
+            for (DoubleWritable dw : val.getValue()) {
+                vals.add(dw.get());
+            }
+            if (LOG.isDebugEnabled()) LOG.debug("KEY: " + keys + ", VALUE: " + vals);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/21187b55/eagle-core/eagle-query/eagle-storage-hbase/src/test/java/org/apache/eagle/storage/hbase/integrationtest/CoprocessorITSuite.java
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-query/eagle-storage-hbase/src/test/java/org/apache/eagle/storage/hbase/integrationtest/CoprocessorITSuite.java b/eagle-core/eagle-query/eagle-storage-hbase/src/test/java/org/apache/eagle/storage/hbase/integrationtest/CoprocessorITSuite.java
new file mode 100644
index 0000000..d380418
--- /dev/null
+++ b/eagle-core/eagle-query/eagle-storage-hbase/src/test/java/org/apache/eagle/storage/hbase/integrationtest/CoprocessorITSuite.java
@@ -0,0 +1,246 @@
+/*
+ * 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.eagle.storage.hbase.integrationtest;
+
+import org.apache.eagle.common.config.EagleConfigFactory;
+import org.apache.eagle.log.base.taggedlog.TaggedLogAPIEntity;
+import org.apache.eagle.log.entity.GenericEntityWriter;
+import org.apache.eagle.log.entity.meta.EntityDefinition;
+import org.apache.eagle.log.entity.meta.EntityDefinitionManager;
+import org.apache.eagle.log.entity.test.TestLogAPIEntity;
+import org.apache.eagle.query.ListQueryCompiler;
+import org.apache.eagle.query.aggregate.AggregateFunctionType;
+import org.apache.eagle.query.aggregate.raw.GroupbyKey;
+import org.apache.eagle.query.aggregate.raw.GroupbyKeyValue;
+import org.apache.eagle.query.aggregate.raw.GroupbyValue;
+import org.apache.eagle.storage.hbase.query.coprocessor.AggregateClient;
+import org.apache.eagle.storage.hbase.query.coprocessor.impl.AggregateClientImpl;
+import org.apache.hadoop.hbase.client.HTableInterface;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.io.BytesWritable;
+import org.apache.hadoop.io.DoubleWritable;
+import org.junit.*;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.*;
+
+@Ignore("Coprocessor Aggregate Client Integration Test.")
+public class CoprocessorITSuite {
+    private HTableInterface table;
+    private long startTime;
+    private long endTime;
+    private List<String> rowkeys;
+    private AggregateClient client;
+    private Scan scan;
+    private int num = 200;
+
+    private static final Logger LOG = LoggerFactory.getLogger(CoprocessorITSuite.class);
+
+    @BeforeClass
+    public static void before() {
+        System.setProperty("config.resource", "/application-sandbox.conf");
+    }
+
+    @Before
+    public void setUp() {
+        startTime = System.currentTimeMillis();
+        try {
+            rowkeys = prepareData(num);
+        } catch (Exception e) {
+            e.printStackTrace();
+            Assert.fail(e.getMessage());
+        }
+        endTime = System.currentTimeMillis();
+        table = EagleConfigFactory.load().getHTable("unittest");
+        client = new AggregateClientImpl();
+        scan = new Scan();
+        scan.setCaching(200);
+
+        ListQueryCompiler compiler = null;
+        try {
+            compiler = new ListQueryCompiler("TestLogAPIEntity[@cluster=\"test4UT\" and @datacenter=\"dc1\"]{@field1,@field2}");
+        } catch (Exception e) {
+            Assert.fail(e.getMessage());
+        }
+        scan.setFilter(compiler.filter());
+    }
+
+    private List<String> prepareData(int count) throws Exception {
+        List<TaggedLogAPIEntity> list = new ArrayList<TaggedLogAPIEntity>();
+        EntityDefinition ed = EntityDefinitionManager.getEntityDefinitionByEntityClass(TestLogAPIEntity.class);
+
+        if (ed == null) {
+            EntityDefinitionManager.registerEntity(TestLogAPIEntity.class);
+            ed = EntityDefinitionManager.getEntityDefinitionByEntityClass(TestLogAPIEntity.class);
+        }
+        ed.setTimeSeries(true);
+        for (int i = 0; i < count; i++) {
+            TestLogAPIEntity e = new TestLogAPIEntity();
+            e.setTags(new HashMap<String, String>());
+            e.getTags().put("cluster", "test4UT");
+            e.getTags().put("datacenter", "dc1");
+            e.getTags().put("index", "" + i);
+            e.getTags().put("jobId", "job_" + System.currentTimeMillis());
+            e.setTimestamp(System.currentTimeMillis());
+            e.setField1(1);
+            e.setField2(2);
+            e.setField3(3);
+            e.setField4(4L);
+            e.setField5(5.0);
+            e.setField6(5.0);
+            e.setField7("7");
+            list.add(e);
+        }
+        GenericEntityWriter writer = new GenericEntityWriter(ed.getService());
+        LOG.info("Writing {} TestLogAPIEntity entities", list.size());
+        List<String> result = writer.write(list);
+        LOG.info("Finish writing test entities");
+        return result;
+    }
+
+    @Test
+    public void testGroupAggregateCountClient() {
+        try {
+            EntityDefinition ed = EntityDefinitionManager.getEntityByServiceName("TestLogAPIEntity");
+            final List<GroupbyKeyValue> result = client.aggregate(table, ed,
+                    scan, Arrays.asList("cluster", "datacenter"),
+                    Collections.singletonList(AggregateFunctionType.count),
+                    Collections.singletonList("field2")).getKeyValues();
+            if (LOG.isDebugEnabled()) {
+                LOG.debug("COUNT");
+            }
+            logGroupbyKeyValue(result);
+            Assert.assertNotNull(result);
+            Assert.assertTrue(result.size() > 0);
+        } catch (Exception e) {
+            Assert.fail(e.getMessage());
+        }
+    }
+
+    @Test
+    public void testGroupAggregateAvgClient() {
+        try {
+            EntityDefinition ed = EntityDefinitionManager.getEntityByServiceName("TestLogAPIEntity");
+            List<GroupbyKeyValue> result = client.aggregate(table, ed, scan, Arrays.asList("cluster", "datacenter"), Arrays.asList(AggregateFunctionType.avg), Arrays.asList("field2")).getKeyValues();
+            if (LOG.isDebugEnabled()) {
+                LOG.debug("AVG");
+            }
+            logGroupbyKeyValue(result);
+            Assert.assertNotNull(result);
+            Assert.assertTrue(result.size() > 0);
+        } catch (Exception e) {
+            e.printStackTrace();
+            Assert.fail(e.getMessage());
+        }
+    }
+
+    @Test
+    public void testGroupAggregateMaxClient() {
+        try {
+            EntityDefinition ed = EntityDefinitionManager.getEntityByServiceName("TestLogAPIEntity");
+            List<GroupbyKeyValue> result = client.aggregate(table, ed, scan, Arrays.asList("cluster", "datacenter"), Arrays.asList(AggregateFunctionType.max), Arrays.asList("field1")).getKeyValues();
+            if (LOG.isDebugEnabled()) {
+                LOG.debug("MAX");
+            }
+            logGroupbyKeyValue(result);
+            Assert.assertNotNull(result);
+            Assert.assertTrue(result.size() > 0);
+        } catch (Exception e) {
+            e.printStackTrace();
+            Assert.fail(e.getMessage());
+        }
+    }
+
+    @Test
+    public void testGroupAggregateSumClient() {
+        try {
+            EntityDefinition ed = EntityDefinitionManager.getEntityByServiceName("TestLogAPIEntity");
+            List<GroupbyKeyValue> result = client.aggregate(table, ed, scan, Arrays.asList("cluster", "datacenter"), Arrays.asList(AggregateFunctionType.sum), Arrays.asList("field2")).getKeyValues();
+            if (LOG.isDebugEnabled()) {
+                LOG.debug("MAX");
+            }
+            logGroupbyKeyValue(result);
+            Assert.assertNotNull(result);
+            Assert.assertTrue(result.size() > 0);
+        } catch (Exception e) {
+            e.printStackTrace();
+            Assert.fail(e.getMessage());
+        }
+    }
+
+    @Test
+    public void testGroupAggregateMinClient() {
+
+        try {
+            EntityDefinition ed = EntityDefinitionManager.getEntityByServiceName("TestLogAPIEntity");
+            List<GroupbyKeyValue> result = client.aggregate(table, ed, scan, Arrays.asList("cluster", "datacenter"), Arrays.asList(AggregateFunctionType.min), Arrays.asList("field2")).getKeyValues();
+            if (LOG.isDebugEnabled()) {
+                LOG.debug("MIN");
+            }
+            logGroupbyKeyValue(result);
+            Assert.assertNotNull(result);
+            Assert.assertTrue(result.size() > 0);
+        } catch (Exception e) {
+            e.printStackTrace();
+            Assert.fail(e.getMessage());
+        }
+    }
+
+    @Test
+    public void testGroupAggregateMultipleClient() {
+        try {
+            EntityDefinition ed = EntityDefinitionManager.getEntityByServiceName("TestLogAPIEntity");
+            final List<GroupbyKeyValue> result = client.aggregate(table, ed, scan, Arrays.asList("cluster", "datacenter"),
+                    Arrays.asList(AggregateFunctionType.min,
+                            AggregateFunctionType.max,
+                            AggregateFunctionType.avg,
+                            AggregateFunctionType.count,
+                            AggregateFunctionType.sum),
+                    Arrays.asList("field2", "field2", "field2", "field2", "field2")).getKeyValues();
+            logGroupbyKeyValue(result);
+            Assert.assertNotNull(result);
+            Assert.assertTrue(result.size() > 0);
+            Assert.assertEquals(2.0, result.get(0).getValue().get(0).get(), 0.00001);
+            Assert.assertEquals(2.0, result.get(0).getValue().get(1).get(), 0.00001);
+            Assert.assertEquals(2.0, result.get(0).getValue().get(2).get(), 0.00001);
+            Assert.assertTrue(num <= result.get(0).getValue().get(3).get());
+            Assert.assertTrue(2.0 * num <= result.get(0).getValue().get(4).get());
+            Assert.assertEquals("test4UT", new String(result.get(0).getKey().getValue().get(0).copyBytes()));
+            Assert.assertEquals("dc1", new String(result.get(0).getKey().getValue().get(1).copyBytes()));
+        } catch (Exception e) {
+            e.printStackTrace();
+            Assert.fail(e.getMessage());
+        }
+    }
+
+    private void logGroupbyKeyValue(List<GroupbyKeyValue> keyValues) {
+        for (GroupbyKeyValue keyValue : keyValues) {
+            GroupbyKey key = keyValue.getKey();
+            List<String> keys = new ArrayList<>();
+            for (BytesWritable bytes : key.getValue()) {
+                keys.add(new String(bytes.copyBytes()));
+            }
+            List<Double> vals = new ArrayList<>();
+            GroupbyValue val = keyValue.getValue();
+            for (DoubleWritable dw : val.getValue()) {
+                vals.add(dw.get());
+            }
+            if (LOG.isDebugEnabled()) LOG.debug("KEY: " + keys + ", VALUE: " + vals);
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/21187b55/eagle-core/eagle-query/eagle-storage-hbase/src/test/java/org/apache/eagle/storage/hbase/integrationtest/CoprocessorJarUtils.java
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-query/eagle-storage-hbase/src/test/java/org/apache/eagle/storage/hbase/integrationtest/CoprocessorJarUtils.java b/eagle-core/eagle-query/eagle-storage-hbase/src/test/java/org/apache/eagle/storage/hbase/integrationtest/CoprocessorJarUtils.java
new file mode 100644
index 0000000..bc7a022
--- /dev/null
+++ b/eagle-core/eagle-query/eagle-storage-hbase/src/test/java/org/apache/eagle/storage/hbase/integrationtest/CoprocessorJarUtils.java
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.eagle.storage.hbase.integrationtest;
+
+import org.apache.commons.io.FileUtils;
+import org.apache.commons.io.filefilter.TrueFileFilter;
+import org.apache.commons.io.filefilter.WildcardFileFilter;
+
+import java.io.File;
+import java.util.Collection;
+
+public class CoprocessorJarUtils {
+    public static File getCoprocessorJarFile() {
+        String projectRootDir = System.getProperty("user.dir");
+        String targetDirPath = projectRootDir + "/target/";
+        File targetDirFile = new File(targetDirPath);
+        if (!targetDirFile.exists()) {
+            throw new IllegalStateException(targetDirPath + " not found, please execute 'mvn install -DskipTests' under " + projectRootDir + " to build the project firstly and retry");
+        }
+        String jarFileNameWildCard = "eagle-storage-hbase-*-coprocessor.jar";
+        Collection<File> jarFiles = FileUtils.listFiles(targetDirFile, new WildcardFileFilter(jarFileNameWildCard), TrueFileFilter.INSTANCE);
+        if (jarFiles.size() == 0) {
+            throw new IllegalStateException("jar is not found, please execute 'mvn package -DskipTests' from project root firstly and retry");
+        }
+        return jarFiles.iterator().next();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/21187b55/eagle-core/eagle-query/eagle-storage-hbase/src/test/java/org/apache/eagle/storage/hbase/integrationtest/CoprocessorToolITSuite.java
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-query/eagle-storage-hbase/src/test/java/org/apache/eagle/storage/hbase/integrationtest/CoprocessorToolITSuite.java b/eagle-core/eagle-query/eagle-storage-hbase/src/test/java/org/apache/eagle/storage/hbase/integrationtest/CoprocessorToolITSuite.java
new file mode 100644
index 0000000..8e7497b
--- /dev/null
+++ b/eagle-core/eagle-query/eagle-storage-hbase/src/test/java/org/apache/eagle/storage/hbase/integrationtest/CoprocessorToolITSuite.java
@@ -0,0 +1,92 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.eagle.storage.hbase.integrationtest;
+
+import org.apache.eagle.storage.hbase.tools.CoprocessorTool;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HColumnDescriptor;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.junit.*;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+
+@Ignore("Coprocessor CLI Tool Integration Test.")
+public class CoprocessorToolITSuite {
+    private static final String remoteJarPath = "/tmp/eagle-storage-hbase-latest-coprocessor.jar";
+    private static String localJarPath = null;
+    private static final Logger LOGGER = LoggerFactory.getLogger(CoprocessorToolITSuite.class);
+    private static final String toolITTableName = "coprocessor_it_table";
+
+    static {
+        Configuration.addDefaultResource("hbase-site-sandbox.xml");
+        localJarPath = CoprocessorJarUtils.getCoprocessorJarFile().getPath();
+    }
+
+    private void testRegisterCoprocessor(String tableName) throws Exception {
+        CoprocessorTool.main(new String[]{
+                "--register",
+                "--config", "hbase-site-sandbox.xml",
+                "--table", tableName,
+                "--jar", remoteJarPath,
+                "--localJar", localJarPath});
+    }
+
+    private void testUnregisterCoprocessor(String tableName) throws Exception {
+        CoprocessorTool.main(new String[]{
+                "--unregister",
+                "--config", "hbase-site-sandbox.xml",
+                "--table", tableName
+        });
+    }
+
+    private void ensureTable() throws IOException {
+        LOGGER.info("Creating table {}", toolITTableName);
+        HBaseAdmin admin = new HBaseAdmin(new Configuration());
+        HTableDescriptor hTableDescriptor = new HTableDescriptor(TableName.valueOf(toolITTableName));
+        hTableDescriptor.addFamily(new HColumnDescriptor("f"));
+        admin.createTable(hTableDescriptor);
+        admin.close();
+        LOGGER.info("Created table {}", toolITTableName);
+    }
+
+    @Test
+    public void testRegisterAndUnregisterCoprocessor() throws Exception {
+        try {
+            ensureTable();
+            testRegisterCoprocessor(toolITTableName);
+            testUnregisterCoprocessor(toolITTableName);
+        } finally {
+            deleteTable();
+        }
+    }
+
+    private void deleteTable() throws IOException {
+        HBaseAdmin admin = new HBaseAdmin(new Configuration());
+        admin.disableTable(TableName.valueOf(toolITTableName));
+        admin.deleteTable(TableName.valueOf(toolITTableName));
+        admin.close();
+    }
+
+    @Test
+    public void testRegisterCoprocessor() throws Exception {
+        testRegisterCoprocessor("unittest");
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/21187b55/eagle-core/eagle-query/eagle-storage-hbase/src/test/java/org/apache/eagle/storage/hbase/spi/TestHBaseStorage.java
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-query/eagle-storage-hbase/src/test/java/org/apache/eagle/storage/hbase/spi/TestHBaseStorage.java b/eagle-core/eagle-query/eagle-storage-hbase/src/test/java/org/apache/eagle/storage/hbase/spi/TestHBaseStorage.java
deleted file mode 100644
index cc5913a..0000000
--- a/eagle-core/eagle-query/eagle-storage-hbase/src/test/java/org/apache/eagle/storage/hbase/spi/TestHBaseStorage.java
+++ /dev/null
@@ -1,29 +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.eagle.storage.hbase.spi;
-
-import org.junit.Test;
-
-/**
- * @since 3/23/15
- */
-public class TestHBaseStorage {
-    @Test
-    public void testCreate(){
-
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/21187b55/eagle-core/eagle-query/eagle-storage-hbase/src/test/java/org/apache/eagle/storage/hbase/spi/TestHBaseStorageLoader.java
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-query/eagle-storage-hbase/src/test/java/org/apache/eagle/storage/hbase/spi/TestHBaseStorageLoader.java b/eagle-core/eagle-query/eagle-storage-hbase/src/test/java/org/apache/eagle/storage/hbase/spi/TestHBaseStorageLoader.java
index 393dfd5..a4c0e23 100644
--- a/eagle-core/eagle-query/eagle-storage-hbase/src/test/java/org/apache/eagle/storage/hbase/spi/TestHBaseStorageLoader.java
+++ b/eagle-core/eagle-query/eagle-storage-hbase/src/test/java/org/apache/eagle/storage/hbase/spi/TestHBaseStorageLoader.java
@@ -27,7 +27,8 @@ import org.junit.Test;
  * @since 3/20/15
  */
 public class TestHBaseStorageLoader {
-    @Test @Ignore("TODO: Add back after refactoring hbase related unit test cases")
+    @Test
+    @Ignore("TODO: Add back after refactoring hbase related unit test cases")
     public void testHBaseStorageLoader() {
         try {
             assert DataStorageManager.getDataStorageByEagleConfig() instanceof HBaseStorage;

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/21187b55/eagle-core/eagle-query/eagle-storage-hbase/src/test/resources/application-sandbox.conf
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-query/eagle-storage-hbase/src/test/resources/application-sandbox.conf b/eagle-core/eagle-query/eagle-storage-hbase/src/test/resources/application-sandbox.conf
new file mode 100644
index 0000000..9ef869c
--- /dev/null
+++ b/eagle-core/eagle-query/eagle-storage-hbase/src/test/resources/application-sandbox.conf
@@ -0,0 +1,23 @@
+# 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.
+
+storage {
+  storage-type = "hbase"
+  table-name-prefixed-with-environment = false
+  coprocessor-enabled = true
+  hbase-zookeeper-quorum = "sandbox.hortonworks.com"
+  hbase-zookeeper-property-clientPort = 2181
+  zookeeper-znode-parent = "/hbase-unsecure"
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/21187b55/eagle-core/eagle-query/eagle-storage-hbase/src/test/resources/application.conf
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-query/eagle-storage-hbase/src/test/resources/application.conf b/eagle-core/eagle-query/eagle-storage-hbase/src/test/resources/application.conf
index 25257c5..d51e3f4 100644
--- a/eagle-core/eagle-query/eagle-storage-hbase/src/test/resources/application.conf
+++ b/eagle-core/eagle-query/eagle-storage-hbase/src/test/resources/application.conf
@@ -13,16 +13,14 @@
 # See the License for the specific language governing permissions and
 # limitations under the License.
 
-eagle{
-	service{
-		env="dev"
-		host="localhost"
-		port=8080
-		storage-type="hbase"
-		table-name-prefixed-with-environment=false
-		coprocessor-enabled=false
-		hbase-zookeeper-quorum="localhost"
-		hbase-zookeeper-property-clientPort=2181
-		zookeeper-znode-parent="/hbase-unsecure"
-	}
+storage {
+  env = "dev"
+  host = "localhost"
+  port = 8080
+  storage-type = "hbase"
+  table-name-prefixed-with-environment = false
+  coprocessor-enabled = false
+  hbase-zookeeper-quorum = "localhost"
+  hbase-zookeeper-property-clientPort = 2181
+  zookeeper-znode-parent = "/hbase-unsecure"
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/21187b55/eagle-core/eagle-query/eagle-storage-hbase/src/test/resources/hbase-site-sandbox.xml
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-query/eagle-storage-hbase/src/test/resources/hbase-site-sandbox.xml b/eagle-core/eagle-query/eagle-storage-hbase/src/test/resources/hbase-site-sandbox.xml
new file mode 100644
index 0000000..17e40a1
--- /dev/null
+++ b/eagle-core/eagle-query/eagle-storage-hbase/src/test/resources/hbase-site-sandbox.xml
@@ -0,0 +1,40 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  ~ Licensed to the Apache Software Foundation (ASF) under one or more
+  ~ contributor license agreements.  See the NOTICE file distributed with
+  ~ this work for additional information regarding copyright ownership.
+  ~ The ASF licenses this file to You under the Apache License, Version 2.0
+  ~ (the "License"); you may not use this file except in compliance with
+  ~ the License.  You may obtain a copy of the License at
+  ~
+  ~    http://www.apache.org/licenses/LICENSE-2.0
+  ~
+  ~ Unless required by applicable law or agreed to in writing, software
+  ~ distributed under the License is distributed on an "AS IS" BASIS,
+  ~ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  ~ See the License for the specific language governing permissions and
+  ~ limitations under the License.
+  -->
+
+<configuration>
+    <property>
+        <name>fs.defaultFS</name>
+        <value>hdfs://sandbox.hortonworks.com:8020</value>
+    </property>
+    <property>
+        <name>hbase.zookeeper.property.clientPort</name>
+        <value>2181</value>
+    </property>
+    <property>
+        <name>hbase.zookeeper.quorum</name>
+        <value>sandbox.hortonworks.com</value>
+    </property>
+    <property>
+        <name>zookeeper.session.timeout</name>
+        <value>30000</value>
+    </property>
+    <property>
+        <name>zookeeper.znode.parent</name>
+        <value>/hbase-unsecure</value>
+    </property>
+</configuration>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/21187b55/eagle-core/eagle-query/eagle-storage-hbase/src/test/resources/log4j.properties
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-query/eagle-storage-hbase/src/test/resources/log4j.properties b/eagle-core/eagle-query/eagle-storage-hbase/src/test/resources/log4j.properties
index fb13ad5..ba06033 100644
--- a/eagle-core/eagle-query/eagle-storage-hbase/src/test/resources/log4j.properties
+++ b/eagle-core/eagle-query/eagle-storage-hbase/src/test/resources/log4j.properties
@@ -12,9 +12,7 @@
 # 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.
-
 log4j.rootLogger=DEBUG, stdout
-
 # standard output
 log4j.appender.stdout=org.apache.log4j.ConsoleAppender
 log4j.appender.stdout.layout=org.apache.log4j.PatternLayout

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/21187b55/eagle-core/eagle-query/eagle-storage-jdbc/src/main/java/org/apache/eagle/storage/jdbc/JdbcConstants.java
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-query/eagle-storage-jdbc/src/main/java/org/apache/eagle/storage/jdbc/JdbcConstants.java b/eagle-core/eagle-query/eagle-storage-jdbc/src/main/java/org/apache/eagle/storage/jdbc/JdbcConstants.java
index a6d0c93..84bf86e 100644
--- a/eagle-core/eagle-query/eagle-storage-jdbc/src/main/java/org/apache/eagle/storage/jdbc/JdbcConstants.java
+++ b/eagle-core/eagle-query/eagle-storage-jdbc/src/main/java/org/apache/eagle/storage/jdbc/JdbcConstants.java
@@ -31,14 +31,14 @@ public class JdbcConstants {
     public static final int DEFAULT_VARCHAR_SIZE =30000;
 
     // Eagle JDBC Storage Configuration
-    public final static String EAGLE_DB_USERNAME = "eagle.service.storage-username";
-    public final static String EAGLE_DB_PASSWORD = "eagle.service.storage-password";
-    public final static String EAGLE_CONN_URL= "eagle.service.storage-connection-url";
-    public final static String EAGLE_CONN_PROPS= "eagle.service.storage-connection-props";
-    public final static String EAGLE_ADAPTER= "eagle.service.storage-adapter";
-    public final static String EAGLE_DATABASE= "eagle.service.storage-database";
-    public final static String EAGLE_DRIVER_CLASS= "eagle.service.storage-driver-class";
-    public final static String EAGLE_CONN_MAX_SIZE= "eagle.service.storage-connection-max";
+    public final static String EAGLE_DB_USERNAME = "storage.storage-username";
+    public final static String EAGLE_DB_PASSWORD = "storage.storage-password";
+    public final static String EAGLE_CONN_URL= "storage.storage-connection-url";
+    public final static String EAGLE_CONN_PROPS= "storage.storage-connection-props";
+    public final static String EAGLE_ADAPTER= "storage.storage-adapter";
+    public final static String EAGLE_DATABASE= "storage.storage-database";
+    public final static String EAGLE_DRIVER_CLASS= "storage.storage-driver-class";
+    public final static String EAGLE_CONN_MAX_SIZE= "storage.storage-connection-max";
 
     public static final boolean isReservedField(String columnName){
         return TIMESTAMP_COLUMN_NAME.equals(columnName) || METRIC_NAME_COLUMN_NAME.equals(columnName) || ROW_KEY_COLUMN_NAME.equals(columnName);

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/21187b55/eagle-core/eagle-query/eagle-storage-jdbc/src/main/java/org/apache/eagle/storage/jdbc/schema/JdbcEntitySchemaManager.java
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-query/eagle-storage-jdbc/src/main/java/org/apache/eagle/storage/jdbc/schema/JdbcEntitySchemaManager.java b/eagle-core/eagle-query/eagle-storage-jdbc/src/main/java/org/apache/eagle/storage/jdbc/schema/JdbcEntitySchemaManager.java
index 4cd1967..9e47ac3 100644
--- a/eagle-core/eagle-query/eagle-storage-jdbc/src/main/java/org/apache/eagle/storage/jdbc/schema/JdbcEntitySchemaManager.java
+++ b/eagle-core/eagle-query/eagle-storage-jdbc/src/main/java/org/apache/eagle/storage/jdbc/schema/JdbcEntitySchemaManager.java
@@ -1,6 +1,4 @@
-package org.apache.eagle.storage.jdbc.schema;
-
-/**
+/*
  * 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.
@@ -17,15 +15,18 @@ package org.apache.eagle.storage.jdbc.schema;
  * limitations under the License.
  */
 
-import org.apache.ddlutils.Platform;
-import org.apache.ddlutils.PlatformFactory;
-import org.apache.ddlutils.model.*;
+package org.apache.eagle.storage.jdbc.schema;
+
 import org.apache.eagle.log.entity.GenericMetricEntity;
 import org.apache.eagle.log.entity.meta.Qualifier;
 import org.apache.eagle.storage.jdbc.JdbcConstants;
 import org.apache.eagle.storage.jdbc.conn.ConnectionConfig;
 import org.apache.eagle.storage.jdbc.conn.ConnectionConfigFactory;
 import org.apache.eagle.storage.jdbc.conn.ConnectionManagerFactory;
+
+import org.apache.ddlutils.Platform;
+import org.apache.ddlutils.PlatformFactory;
+import org.apache.ddlutils.model.*;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/21187b55/eagle-core/eagle-query/eagle-storage-jdbc/src/main/java/org/apache/eagle/storage/jdbc/schema/serializer/MetricJdbcSerDeser.java
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-query/eagle-storage-jdbc/src/main/java/org/apache/eagle/storage/jdbc/schema/serializer/MetricJdbcSerDeser.java b/eagle-core/eagle-query/eagle-storage-jdbc/src/main/java/org/apache/eagle/storage/jdbc/schema/serializer/MetricJdbcSerDeser.java
index 8162955..53ba528 100644
--- a/eagle-core/eagle-query/eagle-storage-jdbc/src/main/java/org/apache/eagle/storage/jdbc/schema/serializer/MetricJdbcSerDeser.java
+++ b/eagle-core/eagle-query/eagle-storage-jdbc/src/main/java/org/apache/eagle/storage/jdbc/schema/serializer/MetricJdbcSerDeser.java
@@ -1,16 +1,4 @@
-package org.apache.eagle.storage.jdbc.schema.serializer;
-
-import org.apache.eagle.log.entity.meta.Qualifier;
-import org.apache.eagle.storage.jdbc.JdbcConstants;
-import org.apache.eagle.storage.jdbc.schema.JdbcEntityDefinitionManager;
-import org.apache.torque.util.JdbcTypedValue;
-
-import java.io.IOException;
-import java.sql.ResultSet;
-import java.sql.SQLException;
-import java.sql.Types;
-
-/**
+/*
  * 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.
@@ -26,6 +14,19 @@ import java.sql.Types;
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+
+package org.apache.eagle.storage.jdbc.schema.serializer;
+
+import org.apache.eagle.log.entity.meta.Qualifier;
+import org.apache.eagle.storage.jdbc.JdbcConstants;
+import org.apache.eagle.storage.jdbc.schema.JdbcEntityDefinitionManager;
+import org.apache.torque.util.JdbcTypedValue;
+
+import java.io.IOException;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Types;
+
 public class MetricJdbcSerDeser implements JdbcSerDeser<double[]> {
     @Override
     public double[] toJavaTypedValue(ResultSet result, Class<?> fieldType, String fieldName, Qualifier qualifier) throws IOException {

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/21187b55/eagle-core/eagle-query/eagle-storage-jdbc/src/test/resources/application-derby.conf
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-query/eagle-storage-jdbc/src/test/resources/application-derby.conf b/eagle-core/eagle-query/eagle-storage-jdbc/src/test/resources/application-derby.conf
index 689dc91..6b4aad1 100644
--- a/eagle-core/eagle-query/eagle-storage-jdbc/src/test/resources/application-derby.conf
+++ b/eagle-core/eagle-query/eagle-storage-jdbc/src/test/resources/application-derby.conf
@@ -13,8 +13,7 @@
 # See the License for the specific language governing permissions and
 # limitations under the License.
 
-eagle {
-	service {
+storage {
 		storage-type="jdbc"
 		storage-adapter="derby"
 		storage-username="eagle"
@@ -24,5 +23,4 @@ eagle {
 		storage-connection-props="encoding=UTF-8"
 		storage-driver-class="org.apache.derby.jdbc.EmbeddedDriver"
 		storage-connection-max=8
-	}
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/21187b55/eagle-core/eagle-query/eagle-storage-jdbc/src/test/resources/application-mysql.conf
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-query/eagle-storage-jdbc/src/test/resources/application-mysql.conf b/eagle-core/eagle-query/eagle-storage-jdbc/src/test/resources/application-mysql.conf
index 61a899f..565c42b 100644
--- a/eagle-core/eagle-query/eagle-storage-jdbc/src/test/resources/application-mysql.conf
+++ b/eagle-core/eagle-query/eagle-storage-jdbc/src/test/resources/application-mysql.conf
@@ -13,8 +13,7 @@
 # See the License for the specific language governing permissions and
 # limitations under the License.
 
-eagle {
-	service {
+storage{
 		storage-type="jdbc"
 		storage-adapter="mysql"
 		storage-username="eagle"
@@ -24,5 +23,4 @@ eagle {
 		storage-connection-props="encoding=UTF-8"
 		storage-driver-class="com.mysql.jdbc.Driver"
 		storage-connection-max=8
-	}
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/21187b55/eagle-core/eagle-query/eagle-storage-jdbc/src/test/resources/application.conf
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-query/eagle-storage-jdbc/src/test/resources/application.conf b/eagle-core/eagle-query/eagle-storage-jdbc/src/test/resources/application.conf
index 689dc91..6b4aad1 100644
--- a/eagle-core/eagle-query/eagle-storage-jdbc/src/test/resources/application.conf
+++ b/eagle-core/eagle-query/eagle-storage-jdbc/src/test/resources/application.conf
@@ -13,8 +13,7 @@
 # See the License for the specific language governing permissions and
 # limitations under the License.
 
-eagle {
-	service {
+storage {
 		storage-type="jdbc"
 		storage-adapter="derby"
 		storage-username="eagle"
@@ -24,5 +23,4 @@ eagle {
 		storage-connection-props="encoding=UTF-8"
 		storage-driver-class="org.apache.derby.jdbc.EmbeddedDriver"
 		storage-connection-max=8
-	}
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/21187b55/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/mr/historyentity/JobConfigurationAPIEntity.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/mr/historyentity/JobConfigurationAPIEntity.java b/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/mr/historyentity/JobConfigurationAPIEntity.java
index d186fd4..bd40c48 100644
--- a/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/mr/historyentity/JobConfigurationAPIEntity.java
+++ b/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/mr/historyentity/JobConfigurationAPIEntity.java
@@ -48,7 +48,7 @@ public class JobConfigurationAPIEntity extends JobBaseAPIEntity {
 
     public void setJobConfig(JobConfig jobConfig) {
         this.jobConfig = jobConfig;
-        _pcs.firePropertyChange("jobConfig", null, null);
+        pcs.firePropertyChange("jobConfig", null, null);
     }
 
     public String getConfigJobName() {
@@ -57,7 +57,7 @@ public class JobConfigurationAPIEntity extends JobBaseAPIEntity {
 
     public void setConfigJobName(String configJobName) {
         this.configJobName = configJobName;
-        _pcs.firePropertyChange("configJobName", null, null);
+        pcs.firePropertyChange("configJobName", null, null);
     }
 
     public String getAlertEmailList() {
@@ -66,6 +66,6 @@ public class JobConfigurationAPIEntity extends JobBaseAPIEntity {
 
     public void setAlertEmailList(String alertEmailList) {
         this.alertEmailList = alertEmailList;
-        _pcs.firePropertyChange("alertEmailList", null, null);
+        pcs.firePropertyChange("alertEmailList", null, null);
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/21187b55/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/mr/historyentity/JobEventAPIEntity.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/mr/historyentity/JobEventAPIEntity.java b/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/mr/historyentity/JobEventAPIEntity.java
index c6bb8e4..aa2c3af 100644
--- a/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/mr/historyentity/JobEventAPIEntity.java
+++ b/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/mr/historyentity/JobEventAPIEntity.java
@@ -39,6 +39,6 @@ public class JobEventAPIEntity extends JobBaseAPIEntity {
 
     public void setEventType(String eventType) {
         this.eventType = eventType;
-        _pcs.firePropertyChange("eventType", null, null);
+        pcs.firePropertyChange("eventType", null, null);
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/21187b55/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/mr/historyentity/JobExecutionAPIEntity.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/mr/historyentity/JobExecutionAPIEntity.java b/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/mr/historyentity/JobExecutionAPIEntity.java
index cdc5810..f7540d5 100644
--- a/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/mr/historyentity/JobExecutionAPIEntity.java
+++ b/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/mr/historyentity/JobExecutionAPIEntity.java
@@ -116,7 +116,7 @@ public class JobExecutionAPIEntity extends JobBaseAPIEntity {
 
     public void setCurrentState(String currentState) {
         this.currentState = currentState;
-        _pcs.firePropertyChange("currentState", null, null);
+        pcs.firePropertyChange("currentState", null, null);
     }
 
     public long getStartTime() {
@@ -125,7 +125,7 @@ public class JobExecutionAPIEntity extends JobBaseAPIEntity {
 
     public void setStartTime(long startTime) {
         this.startTime = startTime;
-        _pcs.firePropertyChange("startTime", null, null);
+        pcs.firePropertyChange("startTime", null, null);
     }
 
     public long getEndTime() {
@@ -134,7 +134,7 @@ public class JobExecutionAPIEntity extends JobBaseAPIEntity {
 
     public void setEndTime(long endTime) {
         this.endTime = endTime;
-        _pcs.firePropertyChange("endTime", null, null);
+        pcs.firePropertyChange("endTime", null, null);
     }
 
     public int getNumTotalMaps() {
@@ -143,7 +143,7 @@ public class JobExecutionAPIEntity extends JobBaseAPIEntity {
 
     public void setNumTotalMaps(int numTotalMaps) {
         this.numTotalMaps = numTotalMaps;
-        _pcs.firePropertyChange("numTotalMaps", null, null);
+        pcs.firePropertyChange("numTotalMaps", null, null);
     }
 
     public int getNumFailedMaps() {
@@ -152,7 +152,7 @@ public class JobExecutionAPIEntity extends JobBaseAPIEntity {
 
     public void setNumFailedMaps(int numFailedMaps) {
         this.numFailedMaps = numFailedMaps;
-        _pcs.firePropertyChange("numFailedMaps", null, null);
+        pcs.firePropertyChange("numFailedMaps", null, null);
     }
 
     public int getNumFinishedMaps() {
@@ -161,7 +161,7 @@ public class JobExecutionAPIEntity extends JobBaseAPIEntity {
 
     public void setNumFinishedMaps(int numFinishedMaps) {
         this.numFinishedMaps = numFinishedMaps;
-        _pcs.firePropertyChange("numFinishedMaps", null, null);
+        pcs.firePropertyChange("numFinishedMaps", null, null);
     }
 
     public int getNumTotalReduces() {
@@ -170,7 +170,7 @@ public class JobExecutionAPIEntity extends JobBaseAPIEntity {
 
     public void setNumTotalReduces(int numTotalReduces) {
         this.numTotalReduces = numTotalReduces;
-        _pcs.firePropertyChange("numTotalReduces", null, null);
+        pcs.firePropertyChange("numTotalReduces", null, null);
     }
 
     public int getNumFailedReduces() {
@@ -179,7 +179,7 @@ public class JobExecutionAPIEntity extends JobBaseAPIEntity {
 
     public void setNumFailedReduces(int numFailedReduces) {
         this.numFailedReduces = numFailedReduces;
-        _pcs.firePropertyChange("numFailedReduces", null, null);
+        pcs.firePropertyChange("numFailedReduces", null, null);
     }
 
     public int getNumFinishedReduces() {
@@ -188,7 +188,7 @@ public class JobExecutionAPIEntity extends JobBaseAPIEntity {
 
     public void setNumFinishedReduces(int numFinishedReduces) {
         this.numFinishedReduces = numFinishedReduces;
-        _pcs.firePropertyChange("numFinishedReduces", null, null);
+        pcs.firePropertyChange("numFinishedReduces", null, null);
     }
 
     public JobCounters getJobCounters() {
@@ -197,7 +197,7 @@ public class JobExecutionAPIEntity extends JobBaseAPIEntity {
 
     public void setJobCounters(JobCounters jobCounters) {
         this.jobCounters = jobCounters;
-        _pcs.firePropertyChange("jobCounters", null, null);
+        pcs.firePropertyChange("jobCounters", null, null);
     }
 
     public int getDataLocalMaps() {

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/21187b55/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/mr/historyentity/JobProcessTimeStampEntity.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/mr/historyentity/JobProcessTimeStampEntity.java b/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/mr/historyentity/JobProcessTimeStampEntity.java
index 6afe347..d887698 100644
--- a/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/mr/historyentity/JobProcessTimeStampEntity.java
+++ b/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/mr/historyentity/JobProcessTimeStampEntity.java
@@ -40,6 +40,6 @@ public class JobProcessTimeStampEntity extends TaggedLogAPIEntity {
 
     public void setCurrentTimeStamp(long currentTimeStamp) {
         this.currentTimeStamp = currentTimeStamp;
-        _pcs.firePropertyChange("currentTimeStamp", null, null);
+        pcs.firePropertyChange("currentTimeStamp", null, null);
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/21187b55/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/mr/historyentity/TaskAttemptCounterAPIEntity.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/mr/historyentity/TaskAttemptCounterAPIEntity.java b/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/mr/historyentity/TaskAttemptCounterAPIEntity.java
index e526f45..1bc9ca0 100644
--- a/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/mr/historyentity/TaskAttemptCounterAPIEntity.java
+++ b/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/mr/historyentity/TaskAttemptCounterAPIEntity.java
@@ -43,7 +43,7 @@ public class TaskAttemptCounterAPIEntity extends JobBaseAPIEntity {
 
     public void setKilledCount(int killedCount) {
         this.killedCount = killedCount;
-        _pcs.firePropertyChange("killedCount", null, null);
+        pcs.firePropertyChange("killedCount", null, null);
     }
 
     public int getFailedCount() {
@@ -52,7 +52,7 @@ public class TaskAttemptCounterAPIEntity extends JobBaseAPIEntity {
 
     public void setFailedCount(int failedCount) {
         this.failedCount = failedCount;
-        _pcs.firePropertyChange("failedCount", null, null);
+        pcs.firePropertyChange("failedCount", null, null);
     }
 
     public int getTotalCount() {
@@ -61,6 +61,6 @@ public class TaskAttemptCounterAPIEntity extends JobBaseAPIEntity {
 
     public void setTotalCount(int totalCount) {
         this.totalCount = totalCount;
-        _pcs.firePropertyChange("totalCount", null, null);
+        pcs.firePropertyChange("totalCount", null, null);
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/21187b55/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/mr/historyentity/TaskAttemptExecutionAPIEntity.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/mr/historyentity/TaskAttemptExecutionAPIEntity.java b/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/mr/historyentity/TaskAttemptExecutionAPIEntity.java
index fd96828..d1210b9 100644
--- a/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/mr/historyentity/TaskAttemptExecutionAPIEntity.java
+++ b/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/mr/historyentity/TaskAttemptExecutionAPIEntity.java
@@ -55,7 +55,7 @@ public class TaskAttemptExecutionAPIEntity extends JobBaseAPIEntity {
 
     public void setTaskStatus(String taskStatus) {
         this.taskStatus = taskStatus;
-        _pcs.firePropertyChange("taskStatus", null, null);
+        pcs.firePropertyChange("taskStatus", null, null);
     }
 
     public long getStartTime() {
@@ -64,7 +64,7 @@ public class TaskAttemptExecutionAPIEntity extends JobBaseAPIEntity {
 
     public void setStartTime(long startTime) {
         this.startTime = startTime;
-        _pcs.firePropertyChange("startTime", null, null);
+        pcs.firePropertyChange("startTime", null, null);
     }
 
     public long getEndTime() {
@@ -73,7 +73,7 @@ public class TaskAttemptExecutionAPIEntity extends JobBaseAPIEntity {
 
     public void setEndTime(long endTime) {
         this.endTime = endTime;
-        _pcs.firePropertyChange("endTime", null, null);
+        pcs.firePropertyChange("endTime", null, null);
     }
 
     public long getDuration() {
@@ -82,7 +82,7 @@ public class TaskAttemptExecutionAPIEntity extends JobBaseAPIEntity {
 
     public void setDuration(long duration) {
         this.duration = duration;
-        _pcs.firePropertyChange("duration", null, null);
+        pcs.firePropertyChange("duration", null, null);
     }
 
     public String getError() {
@@ -91,7 +91,7 @@ public class TaskAttemptExecutionAPIEntity extends JobBaseAPIEntity {
 
     public void setError(String error) {
         this.error = error;
-        _pcs.firePropertyChange("error", null, null);
+        pcs.firePropertyChange("error", null, null);
     }
 
     public JobCounters getJobCounters() {
@@ -100,7 +100,7 @@ public class TaskAttemptExecutionAPIEntity extends JobBaseAPIEntity {
 
     public void setJobCounters(JobCounters jobCounters) {
         this.jobCounters = jobCounters;
-        _pcs.firePropertyChange("jobCounters", null, null);
+        pcs.firePropertyChange("jobCounters", null, null);
     }
 
     public String getTaskAttemptID() {
@@ -109,6 +109,6 @@ public class TaskAttemptExecutionAPIEntity extends JobBaseAPIEntity {
 
     public void setTaskAttemptID(String taskAttemptID) {
         this.taskAttemptID = taskAttemptID;
-        _pcs.firePropertyChange("taskAttemptID", null, null);
+        pcs.firePropertyChange("taskAttemptID", null, null);
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/21187b55/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/mr/historyentity/TaskExecutionAPIEntity.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/mr/historyentity/TaskExecutionAPIEntity.java b/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/mr/historyentity/TaskExecutionAPIEntity.java
index bf559d4..d573da3 100644
--- a/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/mr/historyentity/TaskExecutionAPIEntity.java
+++ b/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/mr/historyentity/TaskExecutionAPIEntity.java
@@ -53,7 +53,7 @@ public class TaskExecutionAPIEntity extends JobBaseAPIEntity {
 
     public void setTaskStatus(String taskStatus) {
         this.taskStatus = taskStatus;
-        _pcs.firePropertyChange("taskStatus", null, null);
+        pcs.firePropertyChange("taskStatus", null, null);
     }
 
     public long getStartTime() {
@@ -62,7 +62,7 @@ public class TaskExecutionAPIEntity extends JobBaseAPIEntity {
 
     public void setStartTime(long startTime) {
         this.startTime = startTime;
-        _pcs.firePropertyChange("startTime", null, null);
+        pcs.firePropertyChange("startTime", null, null);
     }
 
     public long getEndTime() {
@@ -71,7 +71,7 @@ public class TaskExecutionAPIEntity extends JobBaseAPIEntity {
 
     public void setEndTime(long endTime) {
         this.endTime = endTime;
-        _pcs.firePropertyChange("endTime", null, null);
+        pcs.firePropertyChange("endTime", null, null);
     }
 
     public long getDuration() {
@@ -80,7 +80,7 @@ public class TaskExecutionAPIEntity extends JobBaseAPIEntity {
 
     public void setDuration(long duration) {
         this.duration = duration;
-        _pcs.firePropertyChange("duration", null, null);
+        pcs.firePropertyChange("duration", null, null);
     }
 
     public String getError() {
@@ -89,7 +89,7 @@ public class TaskExecutionAPIEntity extends JobBaseAPIEntity {
 
     public void setError(String error) {
         this.error = error;
-        _pcs.firePropertyChange("error", null, null);
+        pcs.firePropertyChange("error", null, null);
     }
 
     public JobCounters getJobCounters() {
@@ -98,6 +98,6 @@ public class TaskExecutionAPIEntity extends JobBaseAPIEntity {
 
     public void setJobCounters(JobCounters jobCounters) {
         this.jobCounters = jobCounters;
-        _pcs.firePropertyChange("jobCounters", null, null);
+        pcs.firePropertyChange("jobCounters", null, null);
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/21187b55/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/mr/historyentity/TaskFailureCountAPIEntity.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/mr/historyentity/TaskFailureCountAPIEntity.java b/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/mr/historyentity/TaskFailureCountAPIEntity.java
index 31f96da..fe41979 100644
--- a/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/mr/historyentity/TaskFailureCountAPIEntity.java
+++ b/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/mr/historyentity/TaskFailureCountAPIEntity.java
@@ -43,7 +43,7 @@ public class TaskFailureCountAPIEntity extends JobBaseAPIEntity {
 
     public void setTaskStatus(String taskStatus) {
         this.taskStatus = taskStatus;
-        _pcs.firePropertyChange("taskStatus", null, null);
+        pcs.firePropertyChange("taskStatus", null, null);
     }
 
     public String getError() {
@@ -52,7 +52,7 @@ public class TaskFailureCountAPIEntity extends JobBaseAPIEntity {
 
     public void setError(String error) {
         this.error = error;
-        _pcs.firePropertyChange("error", null, null);
+        pcs.firePropertyChange("error", null, null);
     }
 
     public int getFailureCount() {
@@ -61,6 +61,6 @@ public class TaskFailureCountAPIEntity extends JobBaseAPIEntity {
 
     public void setFailureCount(int failureCount) {
         this.failureCount = failureCount;
-        _pcs.firePropertyChange("failureCount", null, null);
+        pcs.firePropertyChange("failureCount", null, null);
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/21187b55/eagle-security/eagle-security-hdfs-web/src/main/java/org/apache/eagle/service/security/hdfs/rest/HDFSResourceWebResource.java
----------------------------------------------------------------------
diff --git a/eagle-security/eagle-security-hdfs-web/src/main/java/org/apache/eagle/service/security/hdfs/rest/HDFSResourceWebResource.java b/eagle-security/eagle-security-hdfs-web/src/main/java/org/apache/eagle/service/security/hdfs/rest/HDFSResourceWebResource.java
index 7e8a804..207389a 100644
--- a/eagle-security/eagle-security-hdfs-web/src/main/java/org/apache/eagle/service/security/hdfs/rest/HDFSResourceWebResource.java
+++ b/eagle-security/eagle-security-hdfs-web/src/main/java/org/apache/eagle/service/security/hdfs/rest/HDFSResourceWebResource.java
@@ -44,61 +44,59 @@ import org.slf4j.LoggerFactory;
 
 import org.apache.eagle.service.security.hdfs.HDFSFileSystem;
 
-
 /**
  * REST Web Service to browse files and Paths in HDFS
  */
 @Path(HDFSResourceConstants.HDFS_RESOURCE)
 public class HDFSResourceWebResource {
-	private static Logger LOG = LoggerFactory.getLogger(HDFSResourceWebResource.class);
-	final public static String HDFS_APPLICATION = "HdfsAuditLogApplication";
-	private ApplicationEntityService entityService;
-	private ISecurityMetadataDAO dao;
+    private static Logger LOG = LoggerFactory.getLogger(HDFSResourceWebResource.class);
+    final public static String HDFS_APPLICATION = "HdfsAuditLogApplication";
+    private ApplicationEntityService entityService;
+    private ISecurityMetadataDAO dao;
 
-	@Inject
-	public HDFSResourceWebResource(ApplicationEntityService entityService, Config eagleServerConfig){
-		this.entityService = entityService;
-		dao = MetadataDaoFactory.getMetadataDAO(eagleServerConfig);
-	}
+    @Inject
+    public HDFSResourceWebResource(ApplicationEntityService entityService, Config eagleServerConfig) {
+        this.entityService = entityService;
+        dao = MetadataDaoFactory.getMetadataDAO(eagleServerConfig);
+    }
 
-	@GET
-	@Consumes(MediaType.APPLICATION_JSON)
-	@Produces(MediaType.APPLICATION_JSON)
-	public HDFSResourceWebResponse  getHDFSResource( @QueryParam("site") String site , @QueryParam("path") String filePath )
-	{
-		LOG.info("Starting HDFS Resource Browsing.  Query Parameters ==> Site :"+site+"  Path : "+filePath );
-		HDFSResourceWebResponse response = new HDFSResourceWebResponse();
-		HDFSResourceWebRequestValidator validator = new HDFSResourceWebRequestValidator();
-		List<FileStatusEntity> result = new ArrayList<>();
-		List<FileStatus> fileStatuses = null;
-		try {
-			validator.validate(site, filePath); // First Step would be validating Request
-			Map<String, Object> config = getAppConfig(site, HDFS_APPLICATION);
-			Configuration conf = convert(config);
-			HDFSFileSystem fileSystem = new HDFSFileSystem(conf);
-			fileStatuses = fileSystem.browse(filePath);
-			// Join with File Sensitivity Info
-			HDFSResourceSensitivityDataJoiner joiner = new HDFSResourceSensitivityDataJoiner(dao);
-			result = joiner.joinFileSensitivity(site, fileStatuses);
-			LOG.info("Successfully browsed files in HDFS .");
-		} catch( Exception ex ) {
-			response.setException(EagleExceptionWrapper.wrap(ex));
-			LOG.error(" Exception When browsing Files for the HDFS Path  :"+filePath+"  " , ex);
-		}
-		response.setObj(result);
-		return response;
-	}
+    @GET
+    @Consumes(MediaType.APPLICATION_JSON)
+    @Produces(MediaType.APPLICATION_JSON)
+    public HDFSResourceWebResponse getHDFSResource(@QueryParam("site") String site, @QueryParam("path") String filePath) {
+        LOG.info("Starting HDFS Resource Browsing.  Query Parameters ==> Site :" + site + "  Path : " + filePath);
+        HDFSResourceWebResponse response = new HDFSResourceWebResponse();
+        HDFSResourceWebRequestValidator validator = new HDFSResourceWebRequestValidator();
+        List<FileStatusEntity> result = new ArrayList<>();
+        List<FileStatus> fileStatuses = null;
+        try {
+            validator.validate(site, filePath); // First Step would be validating Request
+            Map<String, Object> config = getAppConfig(site, HDFS_APPLICATION);
+            Configuration conf = convert(config);
+            HDFSFileSystem fileSystem = new HDFSFileSystem(conf);
+            fileStatuses = fileSystem.browse(filePath);
+            // Join with File Sensitivity Info
+            HDFSResourceSensitivityDataJoiner joiner = new HDFSResourceSensitivityDataJoiner(dao);
+            result = joiner.joinFileSensitivity(site, fileStatuses);
+            LOG.info("Successfully browsed files in HDFS .");
+        } catch (Exception ex) {
+            response.setException(EagleExceptionWrapper.wrap(ex));
+            LOG.error(" Exception When browsing Files for the HDFS Path  :" + filePath + "  ", ex);
+        }
+        response.setObj(result);
+        return response;
+    }
 
-	private Map<String, Object> getAppConfig(String site, String appType){
-		ApplicationEntity entity = entityService.getBySiteIdAndAppType(site, appType);
-		return entity.getConfiguration();
-	}
+    private Map<String, Object> getAppConfig(String site, String appType) {
+        ApplicationEntity entity = entityService.getBySiteIdAndAppType(site, appType);
+        return entity.getConfiguration();
+    }
 
-	private Configuration convert(Map<String, Object> originalConfig) throws Exception {
-		Configuration config = new Configuration();
-		for (Map.Entry<String, Object> entry : originalConfig.entrySet()) {
-			config.set(entry.getKey().toString(), entry.getValue().toString());
-		}
-		return config;
-	}
+    private Configuration convert(Map<String, Object> originalConfig) throws Exception {
+        Configuration config = new Configuration();
+        for (Map.Entry<String, Object> entry : originalConfig.entrySet()) {
+            config.set(entry.getKey().toString(), entry.getValue().toString());
+        }
+        return config;
+    }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/21187b55/eagle-server-assembly/pom.xml
----------------------------------------------------------------------
diff --git a/eagle-server-assembly/pom.xml b/eagle-server-assembly/pom.xml
index 5b5d524..5b9537b 100644
--- a/eagle-server-assembly/pom.xml
+++ b/eagle-server-assembly/pom.xml
@@ -25,7 +25,6 @@
         <artifactId>eagle-parent</artifactId>
         <version>0.5.0-incubating-SNAPSHOT</version>
     </parent>
-    <groupId>org.apache.eagle</groupId>
     <artifactId>eagle-server-assembly</artifactId>
     <name>eagle-server-assembly</name>
     <url>http://maven.apache.org</url>

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/21187b55/eagle-server-assembly/src/main/bin/eagle-env.sh
----------------------------------------------------------------------
diff --git a/eagle-server-assembly/src/main/bin/eagle-env.sh b/eagle-server-assembly/src/main/bin/eagle-env.sh
index 1aa556d..b2c04f8 100644
--- a/eagle-server-assembly/src/main/bin/eagle-env.sh
+++ b/eagle-server-assembly/src/main/bin/eagle-env.sh
@@ -1,38 +1,26 @@
-#!/bin/bash
-
-# Licensed to the Apache Software Foundation (ASF) under one or more
-# contributor license agreements.  See the NOTICE file distributed with
-# this work for additional information regarding copyright ownership.
-# The ASF licenses this file to You under the Apache License, Version 2.0
-# (the "License"); you may not use this file except in compliance with
-# the License.  You may obtain a copy of the License at
-#
-#    http://www.apache.org/licenses/LICENSE-2.0
-#
-# Unless required by applicable law or agreed to in writing, software
-# distributed under the License is distributed on an "AS IS" BASIS,
-# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-# See the License for the specific language governing permissions and
-# limitations under the License.
-
-# set EAGLE_HOME
-export EAGLE_HOME=$(dirname $0)/..
-
-# EAGLE_SERVICE_HOST, default is `hostname -f`
-export EAGLE_SERVICE_HOST=localhost
-
-# EAGLE_SERVICE_PORT, default is 9099
-export EAGLE_SERVICE_PORT=9099
-
-# EAGLE_SERVICE_USER
-export EAGLE_SERVICE_USER=admin
-
-# EAGLE_SERVICE_PASSWORD
-export EAGLE_SERVICE_PASSWD=secret
-
-export EAGLE_CLASSPATH=$EAGLE_HOME/conf
-
-# Add eagle shared library jars
-for file in `ls $EAGLE_HOME/lib`; do
-	EAGLE_CLASSPATH=$EAGLE_CLASSPATH:$EAGLE_HOME/lib/$file
+#!/bin/bash
+
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+# set EAGLE_HOME
+export EAGLE_HOME=$(dirname $0)/..
+
+export EAGLE_CLASSPATH=$EAGLE_HOME/conf
+
+# Add eagle shared library jars
+for file in `ls $EAGLE_HOME/lib`; do
+	EAGLE_CLASSPATH=$EAGLE_CLASSPATH:$EAGLE_HOME/lib/$file
 done
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/21187b55/eagle-server-assembly/src/main/bin/eagle-service.sh
----------------------------------------------------------------------
diff --git a/eagle-server-assembly/src/main/bin/eagle-service.sh b/eagle-server-assembly/src/main/bin/eagle-service.sh
deleted file mode 100644
index 533fa2c..0000000
--- a/eagle-server-assembly/src/main/bin/eagle-service.sh
+++ /dev/null
@@ -1,108 +0,0 @@
-#!/bin/bash
-
-# Licensed to the Apache Software Foundation (ASF) under one or more
-# contributor license agreements.  See the NOTICE file distributed with
-# this work for additional information regarding copyright ownership.
-# The ASF licenses this file to You under the Apache License, Version 2.0
-# (the "License"); you may not use this file except in compliance with
-# the License.  You may obtain a copy of the License at
-#
-#    http://www.apache.org/licenses/LICENSE-2.0
-#
-# Unless required by applicable law or agreed to in writing, software
-# distributed under the License is distributed on an "AS IS" BASIS,
-# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-# See the License for the specific language governing permissions and
-# limitations under the License.
-
-function print_help() {
-	echo "Usage: $0 {start | stop | restart | status}"
-	exit 1
-}
-
-if [ $# != 1 ]
-then
-	print_help
-fi
-
-DIR=$(dirname $0)
-
-source ${DIR}/eagle-env.sh
-
-export JAVA_OPTS="-Xmx3072m -XX:MaxPermSize=1024m"
-
-PIDFILE="${DIR}/eagle-service.pid"
-
-CONFIGURATION_YML="${DIR}/../conf/configuration.yml"
-
-DEBUG_OPTIONS="-agentlib:jdwp=transport=dt_socket,server=y,suspend=y,address=5005"
-
-PROGRAM="java -cp $EAGLE_CLASSPATH org.apache.eagle.server.ServerMain server ${CONFIGURATION_YML}"
-
-start() {
-    echo "Starting eagle service ..."
-	echo ${PROGRAM}
-	nohup ${PROGRAM} & echo $! > $PIDFILE
-	if [ $? != 0 ];then
-		echo "Error: failed starting"
-		exit 1
-	fi
-}
-
-stop() {
-    echo "Stopping eagle service ..."
-	if [[ ! -f $PIDFILE ]];then
-	    echo "Eagle service is not running"
-    	exit 1
-    fi
-
-    PID=`cat $PIDFILE`
-	kill $PID
-	if [ $? != 0 ];then
-		echo "Error: failed stopping"
-		exit 1
-	fi
-
-	rm ${PIDFILE}
-	echo "Stopping is completed"
-}
-
-case $1 in
-"start")
-    start;
-	;;
-"stop")
-    stop;
-	;;
-"restart")
-	echo "Restarting eagle service ..."
-    stop; sleep 1; start;
-	echo "Restarting is completed "
-	;;
-"status")
-	echo "Checking eagle service status ..."
-	if [[ -e ${PIDFILE} ]]; then
-	    PID=`cat $PIDFILE`
-	fi
-	if [[ -z ${PID} ]];then
-	    echo "Error: Eagle service is not running (missing PID)"
-	    exit 0
-	elif ps -p ${PID} > /dev/null; then
-	    echo "Eagle service is running with PID $PID"
-	    exit 0
-    else
-        echo "Eagle service is not running (tested PID: ${PID})"
-        exit 0
-    fi
-	;;
-*)
-	print_help
-	;;
-esac
-
-if [ $? != 0 ]; then
-	echo "Error: start failure"
-	exit 1
-fi
-
-exit 0
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/21187b55/eagle-server/src/main/java/org/apache/eagle/server/RESTExceptionMapper.java
----------------------------------------------------------------------
diff --git a/eagle-server/src/main/java/org/apache/eagle/server/RESTExceptionMapper.java b/eagle-server/src/main/java/org/apache/eagle/server/RESTExceptionMapper.java
index 64f9fe0..1799eff 100644
--- a/eagle-server/src/main/java/org/apache/eagle/server/RESTExceptionMapper.java
+++ b/eagle-server/src/main/java/org/apache/eagle/server/RESTExceptionMapper.java
@@ -16,14 +16,13 @@
  */
 package org.apache.eagle.server;
 
-import io.dropwizard.jersey.errors.LoggingExceptionMapper;
-import org.apache.commons.lang3.exception.ExceptionUtils;
 import org.apache.eagle.metadata.resource.RESTResponse;
+import io.dropwizard.jersey.errors.LoggingExceptionMapper;
 
+import java.util.concurrent.ThreadLocalRandom;
 import javax.ws.rs.WebApplicationException;
 import javax.ws.rs.core.MediaType;
 import javax.ws.rs.core.Response;
-import java.util.concurrent.ThreadLocalRandom;
 
 public class RESTExceptionMapper extends LoggingExceptionMapper<Throwable> {
     @Override
@@ -33,6 +32,6 @@ public class RESTExceptionMapper extends LoggingExceptionMapper<Throwable> {
         if (throwable instanceof WebApplicationException) {
             return ((WebApplicationException) throwable).getResponse();
         }
-       return Response.status(Response.Status.INTERNAL_SERVER_ERROR).entity(new RESTResponse<>(throwable)).type(MediaType.APPLICATION_JSON_TYPE).build();
+        return Response.status(Response.Status.INTERNAL_SERVER_ERROR).entity(new RESTResponse<>(throwable)).type(MediaType.APPLICATION_JSON_TYPE).build();
     }
 }
\ No newline at end of file


[13/52] [abbrv] incubator-eagle git commit: [minor] Change resourceFetch to resourcefetch

Posted by yo...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/b5fd782c/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/model/SparkStage.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/model/SparkStage.java b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/model/SparkStage.java
new file mode 100644
index 0000000..14edeee
--- /dev/null
+++ b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/model/SparkStage.java
@@ -0,0 +1,211 @@
+/*
+ *
+ *  * 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.eagle.jpm.util.resourcefetch.model;
+
+import org.codehaus.jackson.annotate.JsonIgnoreProperties;
+import org.codehaus.jackson.map.annotate.JsonSerialize;
+
+import java.util.Map;
+
+@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
+@JsonIgnoreProperties(ignoreUnknown = true)
+public class SparkStage {
+    private String status;
+    private int stageId;
+    private int attemptId;
+    private int numActiveTasks;
+    private int numCompleteTasks;
+    private int numFailedTasks;
+    private long executorRunTime;
+    private long inputBytes;
+    private long inputRecords;
+    private long outputBytes;
+    private long outputRecords;
+    private long shuffleReadBytes;
+    private long shuffleReadRecords;
+    private long shuffleWriteBytes;
+    private long shuffleWriteRecords;
+    private long memoryBytesSpilled;
+    private long diskBytesSpilled;
+    private String name;
+    private String schedulingPool;
+    //taskID
+    private Map<String, SparkTask> tasks;
+
+    public String getStatus() {
+        return status;
+    }
+
+    public void setStatus(String status) {
+        this.status = status;
+    }
+
+    public int getStageId() {
+        return stageId;
+    }
+
+    public void setStageId(int stageId) {
+        this.stageId = stageId;
+    }
+
+    public int getAttemptId() {
+        return attemptId;
+    }
+
+    public void setAttemptId(int attemptId) {
+        this.attemptId = attemptId;
+    }
+
+    public int getNumActiveTasks() {
+        return numActiveTasks;
+    }
+
+    public void setNumActiveTasks(int numActiveTasks) {
+        this.numActiveTasks = numActiveTasks;
+    }
+
+    public int getNumCompleteTasks() {
+        return numCompleteTasks;
+    }
+
+    public void setNumCompleteTasks(int numCompleteTasks) {
+        this.numCompleteTasks = numCompleteTasks;
+    }
+
+    public int getNumFailedTasks() {
+        return numFailedTasks;
+    }
+
+    public void setNumFailedTasks(int numFailedTasks) {
+        this.numFailedTasks = numFailedTasks;
+    }
+
+    public long getExecutorRunTime() {
+        return executorRunTime;
+    }
+
+    public void setExecutorRunTime(long executorRunTime) {
+        this.executorRunTime = executorRunTime;
+    }
+
+    public long getInputBytes() {
+        return inputBytes;
+    }
+
+    public void setInputBytes(long inputBytes) {
+        this.inputBytes = inputBytes;
+    }
+
+    public long getInputRecords() {
+        return inputRecords;
+    }
+
+    public void setInputRecords(long inputRecords) {
+        this.inputRecords = inputRecords;
+    }
+
+    public long getOutputBytes() {
+        return outputBytes;
+    }
+
+    public void setOutputBytes(long outputBytes) {
+        this.outputBytes = outputBytes;
+    }
+
+    public long getOutputRecords() {
+        return outputRecords;
+    }
+
+    public void setOutputRecords(long outputRecords) {
+        this.outputRecords = outputRecords;
+    }
+
+    public long getShuffleReadBytes() {
+        return shuffleReadBytes;
+    }
+
+    public void setShuffleReadBytes(long shuffleReadBytes) {
+        this.shuffleReadBytes = shuffleReadBytes;
+    }
+
+    public long getShuffleReadRecords() {
+        return shuffleReadRecords;
+    }
+
+    public void setShuffleReadRecords(long shuffleReadRecords) {
+        this.shuffleReadRecords = shuffleReadRecords;
+    }
+
+    public long getShuffleWriteBytes() {
+        return shuffleWriteBytes;
+    }
+
+    public void setShuffleWriteBytes(long shuffleWriteBytes) {
+        this.shuffleWriteBytes = shuffleWriteBytes;
+    }
+
+    public long getShuffleWriteRecords() {
+        return shuffleWriteRecords;
+    }
+
+    public void setShuffleWriteRecords(long shuffleWriteRecords) {
+        this.shuffleWriteRecords = shuffleWriteRecords;
+    }
+
+    public long getMemoryBytesSpilled() {
+        return memoryBytesSpilled;
+    }
+
+    public void setMemoryBytesSpilled(long memoryBytesSpilled) {
+        this.memoryBytesSpilled = memoryBytesSpilled;
+    }
+
+    public long getDiskBytesSpilled() {
+        return diskBytesSpilled;
+    }
+
+    public void setDiskBytesSpilled(long diskBytesSpilled) {
+        this.diskBytesSpilled = diskBytesSpilled;
+    }
+
+    public String getName() {
+        return name;
+    }
+
+    public void setName(String name) {
+        this.name = name;
+    }
+
+    public String getSchedulingPool() {
+        return schedulingPool;
+    }
+
+    public void setSchedulingPool(String schedulingPool) {
+        this.schedulingPool = schedulingPool;
+    }
+
+    public Map<String, SparkTask> getTasks() {
+        return tasks;
+    }
+
+    public void setTasks(Map<String, SparkTask> tasks) {
+        this.tasks = tasks;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/b5fd782c/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/model/SparkTask.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/model/SparkTask.java b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/model/SparkTask.java
new file mode 100644
index 0000000..a5dd08f
--- /dev/null
+++ b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/model/SparkTask.java
@@ -0,0 +1,111 @@
+/*
+ *
+ *  * 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.eagle.jpm.util.resourcefetch.model;
+
+
+import org.codehaus.jackson.annotate.JsonIgnoreProperties;
+import org.codehaus.jackson.map.annotate.JsonSerialize;
+
+@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
+@JsonIgnoreProperties(ignoreUnknown = true)
+public class SparkTask {
+    private int taskId;
+    private int index;
+    private int attempt;
+    private String launchTime;
+    private String executorId;
+    private String host;
+    private String taskLocality;
+    private boolean speculative;
+    private SparkTaskMetrics taskMetrics;
+
+    public int getTaskId() {
+        return taskId;
+    }
+
+    public void setTaskId(int taskId) {
+        this.taskId = taskId;
+    }
+
+    public int getIndex() {
+        return index;
+    }
+
+    public void setIndex(int index) {
+        this.index = index;
+    }
+
+    public int getAttempt() {
+        return attempt;
+    }
+
+    public void setAttempt(int attempt) {
+        this.attempt = attempt;
+    }
+
+    public String getLaunchTime() {
+        return launchTime;
+    }
+
+    public void setLaunchTime(String launchTime) {
+        this.launchTime = launchTime;
+    }
+
+    public String getExecutorId() {
+        return executorId;
+    }
+
+    public void setExecutorId(String executorId) {
+        this.executorId = executorId;
+    }
+
+    public String getHost() {
+        return host;
+    }
+
+    public void setHost(String host) {
+        this.host = host;
+    }
+
+    public String getTaskLocality() {
+        return taskLocality;
+    }
+
+    public void setTaskLocality(String taskLocality) {
+        this.taskLocality = taskLocality;
+    }
+
+    public boolean isSpeculative() {
+        return speculative;
+    }
+
+    public void setSpeculative(boolean speculative) {
+        this.speculative = speculative;
+    }
+
+    public SparkTaskMetrics getTaskMetrics() {
+        return taskMetrics;
+    }
+
+    public void setTaskMetrics(SparkTaskMetrics taskMetrics) {
+        this.taskMetrics = taskMetrics;
+    }
+}
+

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/b5fd782c/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/model/SparkTaskInputMetrics.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/model/SparkTaskInputMetrics.java b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/model/SparkTaskInputMetrics.java
new file mode 100644
index 0000000..d079694
--- /dev/null
+++ b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/model/SparkTaskInputMetrics.java
@@ -0,0 +1,46 @@
+/*
+ *
+ *  * 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.eagle.jpm.util.resourcefetch.model;
+
+import org.codehaus.jackson.annotate.JsonIgnoreProperties;
+import org.codehaus.jackson.map.annotate.JsonSerialize;
+
+@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
+@JsonIgnoreProperties(ignoreUnknown = true)
+public class SparkTaskInputMetrics {
+    private long bytesRead;
+    private long recordsRead;
+
+    public long getBytesRead() {
+        return bytesRead;
+    }
+
+    public void setBytesRead(long bytesRead) {
+        this.bytesRead = bytesRead;
+    }
+
+    public long getRecordsRead() {
+        return recordsRead;
+    }
+
+    public void setRecordsRead(long recordsRead) {
+        this.recordsRead = recordsRead;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/b5fd782c/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/model/SparkTaskMetrics.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/model/SparkTaskMetrics.java b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/model/SparkTaskMetrics.java
new file mode 100644
index 0000000..cdb913d
--- /dev/null
+++ b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/model/SparkTaskMetrics.java
@@ -0,0 +1,118 @@
+/*
+ *
+ *  * 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.eagle.jpm.util.resourcefetch.model;
+
+import org.codehaus.jackson.annotate.JsonIgnoreProperties;
+import org.codehaus.jackson.map.annotate.JsonSerialize;
+
+@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
+@JsonIgnoreProperties(ignoreUnknown = true)
+public class SparkTaskMetrics {
+    private long executorDeserializeTime;
+    private long executorRunTime;
+    private long resultSize;
+    private long jvmGcTime;
+    private long resultSerializationTime;
+    private long memoryBytesSpilled;
+    private long diskBytesSpilled;
+    private SparkTaskInputMetrics inputMetrics;
+    private SparkTaskShuffleWriteMetrics shuffleWriteMetrics;
+    private SparkTaskShuffleReadMetrics shuffleReadMetrics;
+
+    public long getExecutorDeserializeTime() {
+        return executorDeserializeTime;
+    }
+
+    public void setExecutorDeserializeTime(long executorDeserializeTime) {
+        this.executorDeserializeTime = executorDeserializeTime;
+    }
+
+    public long getExecutorRunTime() {
+        return executorRunTime;
+    }
+
+    public void setExecutorRunTime(long executorRunTime) {
+        this.executorRunTime = executorRunTime;
+    }
+
+    public long getResultSize() {
+        return resultSize;
+    }
+
+    public void setResultSize(long resultSize) {
+        this.resultSize = resultSize;
+    }
+
+    public long getJvmGcTime() {
+        return jvmGcTime;
+    }
+
+    public void setJvmGcTime(long jvmGcTime) {
+        this.jvmGcTime = jvmGcTime;
+    }
+
+    public long getResultSerializationTime() {
+        return resultSerializationTime;
+    }
+
+    public void setResultSerializationTime(long resultSerializationTime) {
+        this.resultSerializationTime = resultSerializationTime;
+    }
+
+    public long getMemoryBytesSpilled() {
+        return memoryBytesSpilled;
+    }
+
+    public void setMemoryBytesSpilled(long memoryBytesSpilled) {
+        this.memoryBytesSpilled = memoryBytesSpilled;
+    }
+
+    public long getDiskBytesSpilled() {
+        return diskBytesSpilled;
+    }
+
+    public void setDiskBytesSpilled(long diskBytesSpilled) {
+        this.diskBytesSpilled = diskBytesSpilled;
+    }
+
+    public SparkTaskInputMetrics getInputMetrics() {
+        return inputMetrics;
+    }
+
+    public void setInputMetrics(SparkTaskInputMetrics inputMetrics) {
+        this.inputMetrics = inputMetrics;
+    }
+
+    public SparkTaskShuffleWriteMetrics getShuffleWriteMetrics() {
+        return shuffleWriteMetrics;
+    }
+
+    public void setShuffleWriteMetrics(SparkTaskShuffleWriteMetrics shuffleWriteMetrics) {
+        this.shuffleWriteMetrics = shuffleWriteMetrics;
+    }
+
+    public SparkTaskShuffleReadMetrics getShuffleReadMetrics() {
+        return shuffleReadMetrics;
+    }
+
+    public void setShuffleReadMetrics(SparkTaskShuffleReadMetrics shuffleReadMetrics) {
+        this.shuffleReadMetrics = shuffleReadMetrics;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/b5fd782c/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/model/SparkTaskShuffleReadMetrics.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/model/SparkTaskShuffleReadMetrics.java b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/model/SparkTaskShuffleReadMetrics.java
new file mode 100644
index 0000000..3aa94bb
--- /dev/null
+++ b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/model/SparkTaskShuffleReadMetrics.java
@@ -0,0 +1,82 @@
+/*
+ *
+ *  * Licensed to the Apache Software Foundation (ASF) under one or more
+ *  * contributor license agreements.  See the NOTICE file distributed with
+ *  * this work for additional information regarding copyright ownership.
+ *  * The ASF licenses this file to You under the Apache License, Version 2.0
+ *  * (the "License"); you may not use this file except in compliance with
+ *  * the License.  You may obtain a copy of the License at
+ *  *
+ *  *    http://www.apache.org/licenses/LICENSE-2.0
+ *  *
+ *  * Unless required by applicable law or agreed to in writing, software
+ *  * distributed under the License is distributed on an "AS IS" BASIS,
+ *  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  * See the License for the specific language governing permissions and
+ *  * limitations under the License.
+ *
+ */
+
+package org.apache.eagle.jpm.util.resourcefetch.model;
+
+import org.codehaus.jackson.annotate.JsonIgnoreProperties;
+import org.codehaus.jackson.map.annotate.JsonSerialize;
+
+@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
+@JsonIgnoreProperties(ignoreUnknown = true)
+public class SparkTaskShuffleReadMetrics {
+    private long remoteBlocksFetched;
+    private long localBlocksFetched;
+    private long fetchWaitTime;
+    private long remoteBytesRead;
+    private long totalBlocksFetched;
+    private long recordsRead;
+
+    public long getRemoteBlocksFetched() {
+        return remoteBlocksFetched;
+    }
+
+    public void setRemoteBlocksFetched(long remoteBlocksFetched) {
+        this.remoteBlocksFetched = remoteBlocksFetched;
+    }
+
+    public long getLocalBlocksFetched() {
+        return localBlocksFetched;
+    }
+
+    public void setLocalBlocksFetched(long localBlocksFetched) {
+        this.localBlocksFetched = localBlocksFetched;
+    }
+
+    public long getFetchWaitTime() {
+        return fetchWaitTime;
+    }
+
+    public void setFetchWaitTime(long fetchWaitTime) {
+        this.fetchWaitTime = fetchWaitTime;
+    }
+
+    public long getRemoteBytesRead() {
+        return remoteBytesRead;
+    }
+
+    public void setRemoteBytesRead(long remoteBytesRead) {
+        this.remoteBytesRead = remoteBytesRead;
+    }
+
+    public long getTotalBlocksFetched() {
+        return totalBlocksFetched;
+    }
+
+    public void setTotalBlocksFetched(long totalBlocksFetched) {
+        this.totalBlocksFetched = totalBlocksFetched;
+    }
+
+    public long getRecordsRead() {
+        return recordsRead;
+    }
+
+    public void setRecordsRead(long recordsRead) {
+        this.recordsRead = recordsRead;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/b5fd782c/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/model/SparkTaskShuffleWriteMetrics.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/model/SparkTaskShuffleWriteMetrics.java b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/model/SparkTaskShuffleWriteMetrics.java
new file mode 100644
index 0000000..3d8f025
--- /dev/null
+++ b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/model/SparkTaskShuffleWriteMetrics.java
@@ -0,0 +1,55 @@
+/*
+ *
+ *  * 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.eagle.jpm.util.resourcefetch.model;
+
+import org.codehaus.jackson.annotate.JsonIgnoreProperties;
+import org.codehaus.jackson.map.annotate.JsonSerialize;
+
+@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
+@JsonIgnoreProperties(ignoreUnknown = true)
+public class SparkTaskShuffleWriteMetrics {
+    private long bytesWritten;
+    private long writeTime;
+    private long recordsWritten;
+
+    public long getBytesWritten() {
+        return bytesWritten;
+    }
+
+    public void setBytesWritten(long bytesWritten) {
+        this.bytesWritten = bytesWritten;
+    }
+
+    public long getWriteTime() {
+        return writeTime;
+    }
+
+    public void setWriteTime(long writeTime) {
+        this.writeTime = writeTime;
+    }
+
+    public long getRecordsWritten() {
+        return recordsWritten;
+    }
+
+    public void setRecordsWritten(long recordsWritten) {
+        this.recordsWritten = recordsWritten;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/b5fd782c/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/model/TaskCounterGroup.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/model/TaskCounterGroup.java b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/model/TaskCounterGroup.java
new file mode 100644
index 0000000..9803d11
--- /dev/null
+++ b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/model/TaskCounterGroup.java
@@ -0,0 +1,46 @@
+/*
+ * 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.eagle.jpm.util.resourcefetch.model;
+
+import org.codehaus.jackson.annotate.JsonIgnoreProperties;
+import org.codehaus.jackson.map.annotate.JsonSerialize;
+
+import java.util.List;
+
+@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
+@JsonIgnoreProperties(ignoreUnknown = true)
+public class TaskCounterGroup {
+    public String getCounterGroupName() {
+        return counterGroupName;
+    }
+
+    public void setCounterGroupName(String counterGroupName) {
+        this.counterGroupName = counterGroupName;
+    }
+
+    public List<TaskCounterItem> getCounter() {
+        return counter;
+    }
+
+    public void setCounter(List<TaskCounterItem> counter) {
+        this.counter = counter;
+    }
+
+    private String counterGroupName;
+    private List<TaskCounterItem> counter;
+}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/b5fd782c/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/model/TaskCounterItem.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/model/TaskCounterItem.java b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/model/TaskCounterItem.java
new file mode 100644
index 0000000..f8ea22c
--- /dev/null
+++ b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/model/TaskCounterItem.java
@@ -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.
+ */
+
+package org.apache.eagle.jpm.util.resourcefetch.model;
+
+import org.codehaus.jackson.annotate.JsonIgnoreProperties;
+import org.codehaus.jackson.map.annotate.JsonSerialize;
+
+@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
+@JsonIgnoreProperties(ignoreUnknown = true)
+public class TaskCounterItem {
+    public String getName() {
+        return name;
+    }
+
+    public void setName(String name) {
+        this.name = name;
+    }
+
+    public long getValue() {
+        return value;
+    }
+
+    public void setValue(long value) {
+        this.value = value;
+    }
+
+    private String name;
+    private long value;
+}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/b5fd782c/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/model/TaskCounters.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/model/TaskCounters.java b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/model/TaskCounters.java
new file mode 100644
index 0000000..ef99547
--- /dev/null
+++ b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/model/TaskCounters.java
@@ -0,0 +1,46 @@
+/*
+ * 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.eagle.jpm.util.resourcefetch.model;
+
+import org.codehaus.jackson.annotate.JsonIgnoreProperties;
+import org.codehaus.jackson.map.annotate.JsonSerialize;
+
+import java.util.List;
+
+@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
+@JsonIgnoreProperties(ignoreUnknown = true)
+public class TaskCounters {
+    public String getId() {
+        return id;
+    }
+
+    public void setId(String id) {
+        this.id = id;
+    }
+
+    public List<TaskCounterGroup> getTaskCounterGroup() {
+        return taskCounterGroup;
+    }
+
+    public void setTaskCounterGroup(List<TaskCounterGroup> taskCounterGroup) {
+        this.taskCounterGroup = taskCounterGroup;
+    }
+
+    private String id;
+    private List<TaskCounterGroup> taskCounterGroup;
+}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/b5fd782c/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/model/TaskCountersWrapper.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/model/TaskCountersWrapper.java b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/model/TaskCountersWrapper.java
new file mode 100644
index 0000000..80808a3
--- /dev/null
+++ b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/model/TaskCountersWrapper.java
@@ -0,0 +1,35 @@
+/*
+ * 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.eagle.jpm.util.resourcefetch.model;
+
+import org.codehaus.jackson.annotate.JsonIgnoreProperties;
+import org.codehaus.jackson.map.annotate.JsonSerialize;
+
+@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
+@JsonIgnoreProperties(ignoreUnknown = true)
+public class TaskCountersWrapper {
+    public TaskCounters getJobTaskCounters() {
+        return jobTaskCounters;
+    }
+
+    public void setJobTaskCounters(TaskCounters jobTaskCounters) {
+        this.jobTaskCounters = jobTaskCounters;
+    }
+
+    private TaskCounters jobTaskCounters;
+}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/b5fd782c/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/url/JobListServiceURLBuilderImpl.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/url/JobListServiceURLBuilderImpl.java b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/url/JobListServiceURLBuilderImpl.java
new file mode 100644
index 0000000..5513771
--- /dev/null
+++ b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/url/JobListServiceURLBuilderImpl.java
@@ -0,0 +1,52 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.eagle.jpm.util.resourcefetch.url;
+
+import org.apache.eagle.jpm.util.Constants;
+
+public class JobListServiceURLBuilderImpl implements ServiceURLBuilder {
+
+    public String build(String... parameters) {
+        /**
+         * {rmUrl}/ws/v1/cluster/apps?state=RUNNING.
+         * We need to remove tailing slashes to avoid "url//ws/v1"
+         * because it would not be found and would be redirected to
+         * history server ui.
+         */
+        String rmUrl = URLUtil.removeTrailingSlash(parameters[0]);
+
+        String restApi = null;
+        String jobState = parameters[1];
+
+        if (jobState.equals(Constants.JobState.RUNNING.name())) {
+            restApi = Constants.V2_APPS_RUNNING_URL;
+        } else if (jobState.equals(Constants.JobState.FINISHED.name())) {
+            restApi = Constants.V2_APPS_COMPLETED_URL;
+        } else if (jobState.equals(Constants.JobState.ALL.name())) {
+            restApi = Constants.V2_APPS_URL;
+        }
+        if (restApi == null) {
+            return null;
+        }
+        // "/ws/v1/cluster/apps?state=RUNNING"
+        StringBuilder sb = new StringBuilder();
+        sb.append(rmUrl).append("/").append(restApi);
+        sb.append("&").append(Constants.ANONYMOUS_PARAMETER);
+
+        return sb.toString();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/b5fd782c/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/url/ServiceURLBuilder.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/url/ServiceURLBuilder.java b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/url/ServiceURLBuilder.java
new file mode 100644
index 0000000..09fea2f
--- /dev/null
+++ b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/url/ServiceURLBuilder.java
@@ -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.
+ */
+package org.apache.eagle.jpm.util.resourcefetch.url;
+
+public interface ServiceURLBuilder {
+    String build(String... parameters);
+}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/b5fd782c/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/url/SparkCompleteJobServiceURLBuilderImpl.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/url/SparkCompleteJobServiceURLBuilderImpl.java b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/url/SparkCompleteJobServiceURLBuilderImpl.java
new file mode 100644
index 0000000..ca6e938
--- /dev/null
+++ b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/url/SparkCompleteJobServiceURLBuilderImpl.java
@@ -0,0 +1,33 @@
+/*
+ *
+ *  * 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.eagle.jpm.util.resourcefetch.url;
+
+import org.apache.eagle.jpm.util.Constants;
+
+public class SparkCompleteJobServiceURLBuilderImpl implements ServiceURLBuilder {
+
+    public String build(String... parameters) {
+        String url = URLUtil.removeTrailingSlash(parameters[0]);
+
+        return url + "/" + Constants.V2_APPS_URL
+                + "?applicationTypes=SPARK&state=FINISHED&finishedTimeBegin="
+                + parameters[1] + "&" + Constants.ANONYMOUS_PARAMETER;
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/b5fd782c/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/url/SparkJobServiceURLBuilderImpl.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/url/SparkJobServiceURLBuilderImpl.java b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/url/SparkJobServiceURLBuilderImpl.java
new file mode 100644
index 0000000..c5ec67a
--- /dev/null
+++ b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/url/SparkJobServiceURLBuilderImpl.java
@@ -0,0 +1,31 @@
+/*
+ *
+ *  * Licensed to the Apache Software Foundation (ASF) under one or more
+ *  * contributor license agreements.  See the NOTICE file distributed with
+ *  * this work for additional information regarding copyright ownership.
+ *  * The ASF licenses this file to You under the Apache License, Version 2.0
+ *  * (the "License"); you may not use this file except in compliance with
+ *  * the License.  You may obtain a copy of the License at
+ *  *
+ *  *    http://www.apache.org/licenses/LICENSE-2.0
+ *  *
+ *  * Unless required by applicable law or agreed to in writing, software
+ *  * distributed under the License is distributed on an "AS IS" BASIS,
+ *  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  * See the License for the specific language governing permissions and
+ *  * limitations under the License.
+ *
+ */
+
+package org.apache.eagle.jpm.util.resourcefetch.url;
+
+import org.apache.eagle.jpm.util.Constants;
+
+public class SparkJobServiceURLBuilderImpl implements ServiceURLBuilder {
+
+    public String build(String... parameters) {
+        String serverAddress = URLUtil.removeTrailingSlash(parameters[0]);
+
+        return serverAddress + Constants.SPARK_APPS_URL + parameters[1];
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/b5fd782c/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/url/URLUtil.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/url/URLUtil.java b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/url/URLUtil.java
new file mode 100644
index 0000000..11cde80
--- /dev/null
+++ b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/url/URLUtil.java
@@ -0,0 +1,33 @@
+/*
+ * 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.eagle.jpm.util.resourcefetch.url;
+
+/**
+ * URL utils.
+ */
+public class URLUtil {
+    public static String removeTrailingSlash(String url) {
+        int i = url.length() - 1;
+        // Skip all slashes from the end.
+        while (i >= 0 && url.charAt(i) == '/') {
+            i--;
+        }
+
+        return url.substring(0, i + 1);
+    }
+}



[08/52] [abbrv] incubator-eagle git commit: [EAGLE-496] fix code style of jpm

Posted by yo...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/0b852cbc/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/crawler/JHFCrawlerDriverImpl.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/crawler/JHFCrawlerDriverImpl.java b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/crawler/JHFCrawlerDriverImpl.java
index 52bd8ea..278599d 100644
--- a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/crawler/JHFCrawlerDriverImpl.java
+++ b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/crawler/JHFCrawlerDriverImpl.java
@@ -18,62 +18,63 @@
 
 package org.apache.eagle.jpm.mr.history.crawler;
 
-import org.apache.commons.lang3.tuple.Pair;
 import org.apache.eagle.jpm.mr.history.MRHistoryJobConfig;
-import org.apache.eagle.jpm.util.JobIdFilter;
 import org.apache.eagle.jpm.mr.history.zkres.JobHistoryZKStateLCM;
+import org.apache.eagle.jpm.util.JobIdFilter;
+import org.apache.commons.lang3.tuple.Pair;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-
 import java.util.*;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 
 /**
- * single thread crawling per driver
- * multiple drivers can achieve parallelism
+ * single thread crawling per driver.
+ * multiple drivers can achieve parallelism.
  *
  */
 public class JHFCrawlerDriverImpl implements JHFCrawlerDriver {
     private static final Logger LOG = LoggerFactory.getLogger(JHFCrawlerDriverImpl.class);
 
-    private final static int SLEEP_SECONDS_WHILE_QUEUE_IS_EMPTY = 10;
-    private final static String FORMAT_JOB_PROCESS_DATE = "%4d%02d%02d";
-    private final static Pattern PATTERN_JOB_PROCESS_DATE = Pattern.compile("([0-9]{4})([0-9]{2})([0-9]{2})");
+    private static final int SLEEP_SECONDS_WHILE_QUEUE_IS_EMPTY = 10;
+    private static final String FORMAT_JOB_PROCESS_DATE = "%4d%02d%02d";
+    private static final Pattern PATTERN_JOB_PROCESS_DATE = Pattern.compile("([0-9]{4})([0-9]{2})([0-9]{2})");
 
     private static final int INITIALIZED = 0x0;
     private static final int TODAY = 0x1;
     private static final int BEFORETODAY = 0x10;
-    private final int PROCESSED_JOB_KEEP_DAYS = 5;
+    private static final int PROCESSED_JOB_KEEP_DAYS = 5;
 
-    private int m_flag = INITIALIZED; // 0 not set, 1 TODAY, 2 BEFORETODAY
-    private Deque<Pair<Long, String> > m_processQueue = new LinkedList<>();
-    private Set<String> m_processedJobFileNames = new HashSet<>();
+    private int flag = INITIALIZED; // 0 not set, 1 TODAY, 2 BEFORETODAY
+    private Deque<Pair<Long, String>> processQueue = new LinkedList<>();
+    private Set<String> processedJobFileNames = new HashSet<>();
 
-    private final JobProcessDate m_proceeDate = new JobProcessDate();
-    private boolean m_dryRun;
-    private JHFInputStreamCallback m_reader;
-    protected boolean m_zeroBasedMonth = true;
+    private final JobProcessDate processDate = new JobProcessDate();
+    private boolean dryRun;
+    private JHFInputStreamCallback reader;
+    protected boolean zeroBasedMonth = true;
 
-    private JobHistoryZKStateLCM m_zkStatelcm;
-    private JobHistoryLCM m_jhfLCM;
-    private JobIdFilter m_jobFilter;
-    private int m_partitionId;
-    private TimeZone m_timeZone;
+    private JobHistoryZKStateLCM zkStateLcm;
+    private JobHistoryLCM jhfLCM;
+    private JobIdFilter jobFilter;
+    private int partitionId;
+    private TimeZone timeZone;
 
     public JHFCrawlerDriverImpl(MRHistoryJobConfig.JobHistoryEndpointConfig jobHistoryConfig,
                                 MRHistoryJobConfig.ControlConfig controlConfig, JHFInputStreamCallback reader,
                                 JobHistoryZKStateLCM zkStateLCM,
                                 JobHistoryLCM historyLCM, JobIdFilter jobFilter, int partitionId) throws Exception {
-        this.m_zeroBasedMonth = controlConfig.zeroBasedMonth;
-        this.m_dryRun = controlConfig.dryRun;
-        if (this.m_dryRun)  LOG.info("this is a dry run");
-        this.m_reader = reader;
-        m_jhfLCM = historyLCM;//new JobHistoryDAOImpl(jobHistoryConfig);
-        this.m_zkStatelcm = zkStateLCM;
-        this.m_partitionId = partitionId;
-        this.m_jobFilter = jobFilter;
-        m_timeZone = TimeZone.getTimeZone(controlConfig.timeZone);
+        this.zeroBasedMonth = controlConfig.zeroBasedMonth;
+        this.dryRun = controlConfig.dryRun;
+        if (this.dryRun)  {
+            LOG.info("this is a dry run");
+        }
+        this.reader = reader;
+        jhfLCM = historyLCM;//new JobHistoryDAOImpl(jobHistoryConfig);
+        this.zkStateLcm = zkStateLCM;
+        this.partitionId = partitionId;
+        this.jobFilter = jobFilter;
+        timeZone = TimeZone.getTimeZone(controlConfig.timeZone);
     }
 
     /**
@@ -90,70 +91,70 @@ public class JHFCrawlerDriverImpl implements JHFCrawlerDriver {
      */
     @Override
     public long crawl() throws Exception {
-        LOG.info("queue size is " + m_processQueue.size());
-        while (m_processQueue.isEmpty()) {
+        LOG.info("queue size is " + processQueue.size());
+        while (processQueue.isEmpty()) {
             // read lastProcessedDate only when it's initialized
-            if (m_flag == INITIALIZED) {
+            if (flag == INITIALIZED) {
                 readAndCacheLastProcessedDate();
             }
-            if (m_flag == BEFORETODAY) {
+            if (flag == BEFORETODAY) {
                 updateProcessDate();
                 clearProcessedJobFileNames();
             }
-            if (m_flag != TODAY) { // advance one day if initialized or BEFORE today
+            if (flag != TODAY) { // advance one day if initialized or BEFORE today
                 advanceOneDay();
             }
 
             if (isToday()) {
-                m_flag = TODAY;
+                flag = TODAY;
             } else {
-                m_flag = BEFORETODAY;
+                flag = BEFORETODAY;
             }
 
-            List<String> serialNumbers = m_jhfLCM.readSerialNumbers(this.m_proceeDate.year, getActualMonth(m_proceeDate.month), this.m_proceeDate.day);
-            List<Pair<Long, String> > allJobHistoryFiles = new LinkedList<>();
+            List<String> serialNumbers = jhfLCM.readSerialNumbers(this.processDate.year, getActualMonth(processDate.month), this.processDate.day);
+            List<Pair<Long, String>> allJobHistoryFiles = new LinkedList<>();
             for (String serialNumber : serialNumbers) {
-                List<Pair<Long, String> > jobHistoryFiles = m_jhfLCM.readFileNames(
-                        this.m_proceeDate.year,
-                        getActualMonth(m_proceeDate.month),
-                        this.m_proceeDate.day,
+                List<Pair<Long, String>> jobHistoryFiles = jhfLCM.readFileNames(
+                        this.processDate.year,
+                        getActualMonth(processDate.month),
+                        this.processDate.day,
                         Integer.parseInt(serialNumber));
                 LOG.info("total number of job history files " + jobHistoryFiles.size());
                 for (Pair<Long, String> jobHistoryFile : jobHistoryFiles) {
-                    if (m_jobFilter.accept(jobHistoryFile.getRight()) && !fileProcessed(jobHistoryFile.getRight())) {
+                    if (jobFilter.accept(jobHistoryFile.getRight()) && !fileProcessed(jobHistoryFile.getRight())) {
                         allJobHistoryFiles.add(jobHistoryFile);
                     }
                 }
                 jobHistoryFiles.clear();
-                LOG.info("after filtering, number of job history files " + m_processQueue.size());
+                LOG.info("after filtering, number of job history files " + processQueue.size());
             }
 
             Collections.sort(allJobHistoryFiles,
-                    new Comparator<Pair<Long, String>>() {
-                        @Override
-                        public int compare(Pair<Long, String> o1, Pair<Long, String> o2) {
-                            if (o1.getLeft() > o2.getLeft()) return 1;
-                            else if (o1.getLeft() == o2.getLeft()) return 0;
-                            else return -1;
-                        }
+                (o1, o2) -> {
+                    if (o1.getLeft() > o2.getLeft()) {
+                        return 1;
+                    } else if (o1.getLeft() == o2.getLeft()) {
+                        return 0;
+                    } else {
+                        return -1;
                     }
+                }
             );
             for (Pair<Long, String> jobHistoryFile : allJobHistoryFiles) {
-                m_processQueue.add(jobHistoryFile);
+                processQueue.add(jobHistoryFile);
             }
 
             allJobHistoryFiles.clear();
 
-            if (m_processQueue.isEmpty()) {
+            if (processQueue.isEmpty()) {
                 Thread.sleep(SLEEP_SECONDS_WHILE_QUEUE_IS_EMPTY * 1000);
             } else {
-                LOG.info("queue size after populating is now : " + m_processQueue.size());
+                LOG.info("queue size after populating is now : " + processQueue.size());
             }
         }
         // start to process job history file
-        Pair<Long, String> item = m_processQueue.pollFirst();
+        Pair<Long, String> item = processQueue.pollFirst();
         String jobHistoryFile = item.getRight();
-        Long modifiedTime = item.getLeft();
         if (jobHistoryFile == null) { // terminate this round of crawling when the queue is empty
             LOG.info("process queue is empty, ignore this round");
             return -1;
@@ -168,33 +169,34 @@ public class JHFCrawlerDriverImpl implements JHFCrawlerDriver {
             LOG.warn("illegal job history file name : " + jobHistoryFile);
             return -1;
         }
-        if (!m_dryRun) {
-            m_jhfLCM.readFileContent(
-                    m_proceeDate.year,
-                    getActualMonth(m_proceeDate.month),
-                    m_proceeDate.day,
+        if (!dryRun) {
+            jhfLCM.readFileContent(
+                    processDate.year,
+                    getActualMonth(processDate.month),
+                    processDate.day,
                     Integer.valueOf(serialNumber),
                     jobHistoryFile,
-                    m_reader);
+                reader);
         }
-        m_zkStatelcm.addProcessedJob(String.format(FORMAT_JOB_PROCESS_DATE,
-                this.m_proceeDate.year,
-                this.m_proceeDate.month + 1,
-                this.m_proceeDate.day),
+        zkStateLcm.addProcessedJob(String.format(FORMAT_JOB_PROCESS_DATE,
+                this.processDate.year,
+                this.processDate.month + 1,
+                this.processDate.day),
                 jobHistoryFile);
-        m_processedJobFileNames.add(jobHistoryFile);
+        processedJobFileNames.add(jobHistoryFile);
 
+        Long modifiedTime = item.getLeft();
         return modifiedTime;
     }
 
     private void updateProcessDate() throws Exception {
-        String line = String.format(FORMAT_JOB_PROCESS_DATE, this.m_proceeDate.year,
-                this.m_proceeDate.month + 1, this.m_proceeDate.day);
-        m_zkStatelcm.updateProcessedDate(m_partitionId, line);
+        String line = String.format(FORMAT_JOB_PROCESS_DATE, this.processDate.year,
+                this.processDate.month + 1, this.processDate.day);
+        zkStateLcm.updateProcessedDate(partitionId, line);
     }
 
-    private int getActualMonth(int month){
-        return m_zeroBasedMonth ? m_proceeDate.month : m_proceeDate.month + 1;
+    private int getActualMonth(int month) {
+        return zeroBasedMonth ? processDate.month : processDate.month + 1;
     }
 
     private static class JobProcessDate {
@@ -204,37 +206,37 @@ public class JHFCrawlerDriverImpl implements JHFCrawlerDriver {
     }
 
     private void clearProcessedJobFileNames() {
-        m_processedJobFileNames.clear();
+        processedJobFileNames.clear();
     }
 
     private void readAndCacheLastProcessedDate() throws Exception {
-        String lastProcessedDate = m_zkStatelcm.readProcessedDate(m_partitionId);
+        String lastProcessedDate = zkStateLcm.readProcessedDate(partitionId);
         Matcher m = PATTERN_JOB_PROCESS_DATE.matcher(lastProcessedDate);
         if (m.find() && m.groupCount() == 3) {
-            this.m_proceeDate.year = Integer.parseInt(m.group(1));
-            this.m_proceeDate.month = Integer.parseInt(m.group(2)) - 1; // zero based month
-            this.m_proceeDate.day = Integer.parseInt(m.group(3));
+            this.processDate.year = Integer.parseInt(m.group(1));
+            this.processDate.month = Integer.parseInt(m.group(2)) - 1; // zero based month
+            this.processDate.day = Integer.parseInt(m.group(3));
         } else {
             throw new IllegalStateException("job lastProcessedDate must have format YYYYMMDD " + lastProcessedDate);
         }
 
-        GregorianCalendar cal = new GregorianCalendar(m_timeZone);
-        cal.set(this.m_proceeDate.year, this.m_proceeDate.month, this.m_proceeDate.day, 0, 0, 0);
+        GregorianCalendar cal = new GregorianCalendar(timeZone);
+        cal.set(this.processDate.year, this.processDate.month, this.processDate.day, 0, 0, 0);
         cal.add(Calendar.DATE, 1);
-        List<String> list = m_zkStatelcm.readProcessedJobs(String.format(FORMAT_JOB_PROCESS_DATE, cal.get(Calendar.YEAR),
+        List<String> list = zkStateLcm.readProcessedJobs(String.format(FORMAT_JOB_PROCESS_DATE, cal.get(Calendar.YEAR),
                 cal.get(Calendar.MONTH) + 1, cal.get(Calendar.DAY_OF_MONTH)));
         if (list != null) {
-            this.m_processedJobFileNames = new HashSet<>(list);
+            this.processedJobFileNames = new HashSet<>(list);
         }
     }
 
     private void advanceOneDay() throws Exception {
-        GregorianCalendar cal = new GregorianCalendar(m_timeZone);
-        cal.set(this.m_proceeDate.year, this.m_proceeDate.month, this.m_proceeDate.day, 0, 0, 0);
+        GregorianCalendar cal = new GregorianCalendar(timeZone);
+        cal.set(this.processDate.year, this.processDate.month, this.processDate.day, 0, 0, 0);
         cal.add(Calendar.DATE, 1);
-        this.m_proceeDate.year = cal.get(Calendar.YEAR);
-        this.m_proceeDate.month = cal.get(Calendar.MONTH);
-        this.m_proceeDate.day = cal.get(Calendar.DAY_OF_MONTH);
+        this.processDate.year = cal.get(Calendar.YEAR);
+        this.processDate.month = cal.get(Calendar.MONTH);
+        this.processDate.day = cal.get(Calendar.DAY_OF_MONTH);
 
         try {
             clearProcessedJob(cal);
@@ -249,29 +251,30 @@ public class JHFCrawlerDriverImpl implements JHFCrawlerDriver {
         cal.add(Calendar.DATE, -1 - PROCESSED_JOB_KEEP_DAYS);
         String line = String.format(FORMAT_JOB_PROCESS_DATE, cal.get(Calendar.YEAR),
                 cal.get(Calendar.MONTH) + 1, cal.get(Calendar.DAY_OF_MONTH));
-        m_zkStatelcm.truncateProcessedJob(line);
+        zkStateLcm.truncateProcessedJob(line);
     }
 
     private boolean isToday() {
-        GregorianCalendar today = new GregorianCalendar(m_timeZone);
+        GregorianCalendar today = new GregorianCalendar(timeZone);
 
-        if (today.get(Calendar.YEAR) == this.m_proceeDate.year
-                && today.get(Calendar.MONTH) == this.m_proceeDate.month
-                && today.get(Calendar.DAY_OF_MONTH) == this.m_proceeDate.day)
+        if (today.get(Calendar.YEAR) == this.processDate.year
+                && today.get(Calendar.MONTH) == this.processDate.month
+                && today.get(Calendar.DAY_OF_MONTH) == this.processDate.day) {
             return true;
-
+        }
         return false;
     }
 
     /**
-     * check if this file was already processed
+     * check if this file was already processed.
      *
      * @param fileName
      * @return
      */
     private boolean fileProcessed(String fileName) {
-        if (m_processedJobFileNames.contains(fileName))
+        if (processedJobFileNames.contains(fileName)) {
             return true;
+        }
         return false;
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/0b852cbc/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/crawler/JHFInputStreamCallback.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/crawler/JHFInputStreamCallback.java b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/crawler/JHFInputStreamCallback.java
index 52a62a4..3e5ce1f 100644
--- a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/crawler/JHFInputStreamCallback.java
+++ b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/crawler/JHFInputStreamCallback.java
@@ -24,11 +24,11 @@ import java.io.InputStream;
 import java.io.Serializable;
 
 /**
- * callback when job history file input stream is ready
+ * callback when job history file input stream is ready.
  */
 public interface JHFInputStreamCallback extends Serializable {
     /**
-     * this is called when job file string and job configuration file is ready
+     * this is called when job file string and job configuration file is ready.
      * @param is
      * @param configuration
      * @throws Exception

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/0b852cbc/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/crawler/JobHistoryContentFilter.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/crawler/JobHistoryContentFilter.java b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/crawler/JobHistoryContentFilter.java
index 66dbce1..2c866f6 100644
--- a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/crawler/JobHistoryContentFilter.java
+++ b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/crawler/JobHistoryContentFilter.java
@@ -24,9 +24,14 @@ import java.util.regex.Pattern;
 
 public interface JobHistoryContentFilter extends Serializable {
     boolean acceptJobFile();
+
     boolean acceptJobConfFile();
+
     List<Pattern> getMustHaveJobConfKeyPatterns();
+
     List<Pattern> getJobConfKeyInclusionPatterns();
+
     List<Pattern> getJobConfKeyExclusionPatterns();
+    
     String getJobNameKey();
 }

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/0b852cbc/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/crawler/JobHistoryContentFilterBuilder.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/crawler/JobHistoryContentFilterBuilder.java b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/crawler/JobHistoryContentFilterBuilder.java
index 65b8dab..c43c366 100644
--- a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/crawler/JobHistoryContentFilterBuilder.java
+++ b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/crawler/JobHistoryContentFilterBuilder.java
@@ -27,56 +27,57 @@ import java.util.List;
 import java.util.regex.Pattern;
 
 public class JobHistoryContentFilterBuilder {
-    private final static Logger LOG = LoggerFactory.getLogger(JobHistoryContentFilterBuilder.class);
+    private static final Logger LOG = LoggerFactory.getLogger(JobHistoryContentFilterBuilder.class);
 
-    private boolean m_acceptJobFile;
-    private boolean m_acceptJobConfFile;
-    private List<Pattern> m_mustHaveJobConfKeyPatterns;
-    private List<Pattern> m_jobConfKeyInclusionPatterns;
-    private List<Pattern> m_jobConfKeyExclusionPatterns;
+    private boolean acceptJobFile;
+    private boolean acceptJobConfFile;
+    private List<Pattern> mustHaveJobConfKeyPatterns;
+    private List<Pattern> jobConfKeyInclusionPatterns;
+    private List<Pattern> jobConfKeyExclusionPatterns;
 
     private String jobNameKey;
 
-    public static JobHistoryContentFilterBuilder newBuilder(){
+    public static JobHistoryContentFilterBuilder newBuilder() {
         return new JobHistoryContentFilterBuilder();
     }
 
     public JobHistoryContentFilterBuilder acceptJobFile() {
-        this.m_acceptJobFile = true;
+        this.acceptJobFile = true;
         return this;
     }
 
     public JobHistoryContentFilterBuilder acceptJobConfFile() {
-        this.m_acceptJobConfFile = true;
+        this.acceptJobConfFile = true;
         return this;
     }
 
     public JobHistoryContentFilterBuilder mustHaveJobConfKeyPatterns(Pattern ...patterns) {
-        m_mustHaveJobConfKeyPatterns = Arrays.asList(patterns);
-        if (m_jobConfKeyInclusionPatterns != null) {
+        mustHaveJobConfKeyPatterns = Arrays.asList(patterns);
+        if (jobConfKeyInclusionPatterns != null) {
             List<Pattern> list = new ArrayList<Pattern>();
-            list.addAll(m_jobConfKeyInclusionPatterns);
+            list.addAll(jobConfKeyInclusionPatterns);
             list.addAll(Arrays.asList(patterns));
-            m_jobConfKeyInclusionPatterns = list;
+            jobConfKeyInclusionPatterns = list;
+        } else {
+            jobConfKeyInclusionPatterns = Arrays.asList(patterns);
         }
-        else
-            m_jobConfKeyInclusionPatterns = Arrays.asList(patterns);
         return this;
     }
 
     public JobHistoryContentFilterBuilder includeJobKeyPatterns(Pattern ... patterns) {
-        if (m_jobConfKeyInclusionPatterns != null) {
+        if (jobConfKeyInclusionPatterns != null) {
             List<Pattern> list = new ArrayList<Pattern>();
-            list.addAll(m_jobConfKeyInclusionPatterns);
+            list.addAll(jobConfKeyInclusionPatterns);
             list.addAll(Arrays.asList(patterns));
-            m_jobConfKeyInclusionPatterns = list;
-        } else
-            m_jobConfKeyInclusionPatterns = Arrays.asList(patterns);
+            jobConfKeyInclusionPatterns = list;
+        } else {
+            jobConfKeyInclusionPatterns = Arrays.asList(patterns);
+        }
         return this;
     }
 
     public JobHistoryContentFilterBuilder excludeJobKeyPatterns(Pattern ...patterns) {
-        m_jobConfKeyExclusionPatterns = Arrays.asList(patterns);
+        jobConfKeyExclusionPatterns = Arrays.asList(patterns);
         return this;
     }
 
@@ -87,11 +88,11 @@ public class JobHistoryContentFilterBuilder {
 
     public JobHistoryContentFilter build() {
         JobHistoryContentFilterImpl filter = new JobHistoryContentFilterImpl();
-        filter.setAcceptJobFile(m_acceptJobFile);
-        filter.setAcceptJobConfFile(m_acceptJobConfFile);
-        filter.setMustHaveJobConfKeyPatterns(m_mustHaveJobConfKeyPatterns);
-        filter.setJobConfKeyInclusionPatterns(m_jobConfKeyInclusionPatterns);
-        filter.setJobConfKeyExclusionPatterns(m_jobConfKeyExclusionPatterns);
+        filter.setAcceptJobFile(acceptJobFile);
+        filter.setAcceptJobConfFile(acceptJobConfFile);
+        filter.setMustHaveJobConfKeyPatterns(mustHaveJobConfKeyPatterns);
+        filter.setJobConfKeyInclusionPatterns(jobConfKeyInclusionPatterns);
+        filter.setJobConfKeyExclusionPatterns(jobConfKeyExclusionPatterns);
         filter.setJobNameKey(jobNameKey);
         LOG.info("job history content filter:" + filter);
         return filter;

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/0b852cbc/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/crawler/JobHistoryContentFilterImpl.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/crawler/JobHistoryContentFilterImpl.java b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/crawler/JobHistoryContentFilterImpl.java
index 5e7a856..5340372 100644
--- a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/crawler/JobHistoryContentFilterImpl.java
+++ b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/crawler/JobHistoryContentFilterImpl.java
@@ -22,37 +22,37 @@ import java.util.List;
 import java.util.regex.Pattern;
 
 public class JobHistoryContentFilterImpl implements JobHistoryContentFilter {
-    private boolean m_acceptJobFile;
-    private boolean m_acceptJobConfFile;
-    private List<Pattern> m_mustHaveJobConfKeyPatterns;
-    private List<Pattern> m_jobConfKeyInclusionPatterns;
-    private List<Pattern> m_jobConfKeyExclusionPatterns;
+    private boolean acceptJobFile;
+    private boolean acceptJobConfFile;
+    private List<Pattern> mustHaveJobConfKeyPatterns;
+    private List<Pattern> jobConfKeyInclusionPatterns;
+    private List<Pattern> jobConfKeyExclusionPatterns;
 
     private String jobNameKey;
 
     @Override
     public boolean acceptJobFile() {
-        return m_acceptJobFile;
+        return acceptJobFile;
     }
 
     @Override
     public boolean acceptJobConfFile() {
-        return m_acceptJobConfFile;
+        return acceptJobConfFile;
     }
 
     @Override
     public List<Pattern> getMustHaveJobConfKeyPatterns() {
-        return m_mustHaveJobConfKeyPatterns;
+        return mustHaveJobConfKeyPatterns;
     }
 
     @Override
     public List<Pattern> getJobConfKeyInclusionPatterns() {
-        return m_jobConfKeyInclusionPatterns;
+        return jobConfKeyInclusionPatterns;
     }
 
     @Override
     public List<Pattern> getJobConfKeyExclusionPatterns() {
-        return m_jobConfKeyExclusionPatterns;
+        return jobConfKeyExclusionPatterns;
     }
 
     @Override
@@ -65,40 +65,40 @@ public class JobHistoryContentFilterImpl implements JobHistoryContentFilter {
     }
 
     public void setAcceptJobFile(boolean acceptJobFile) {
-        this.m_acceptJobFile = acceptJobFile;
+        this.acceptJobFile = acceptJobFile;
     }
 
     public void setAcceptJobConfFile(boolean acceptJobConfFile) {
-        this.m_acceptJobConfFile = acceptJobConfFile;
+        this.acceptJobConfFile = acceptJobConfFile;
     }
 
     public void setJobConfKeyInclusionPatterns(
             List<Pattern> jobConfKeyInclusionPatterns) {
-        this.m_jobConfKeyInclusionPatterns = jobConfKeyInclusionPatterns;
+        this.jobConfKeyInclusionPatterns = jobConfKeyInclusionPatterns;
     }
 
     public void setJobConfKeyExclusionPatterns(
             List<Pattern> jobConfKeyExclusionPatterns) {
-        this.m_jobConfKeyExclusionPatterns = jobConfKeyExclusionPatterns;
+        this.jobConfKeyExclusionPatterns = jobConfKeyExclusionPatterns;
     }
 
     public void setMustHaveJobConfKeyPatterns(List<Pattern> mustHaveJobConfKeyPatterns) {
-        this.m_mustHaveJobConfKeyPatterns = mustHaveJobConfKeyPatterns;
+        this.mustHaveJobConfKeyPatterns = mustHaveJobConfKeyPatterns;
     }
 
     public String toString() {
         StringBuilder sb = new StringBuilder();
         sb.append("job history file:");
-        sb.append(m_acceptJobFile);
+        sb.append(acceptJobFile);
         sb.append(", job config file:");
-        sb.append(m_acceptJobConfFile);
-        if(m_acceptJobConfFile){
+        sb.append(acceptJobConfFile);
+        if (acceptJobConfFile) {
             sb.append(", must contain keys:");
-            sb.append(m_mustHaveJobConfKeyPatterns);
+            sb.append(mustHaveJobConfKeyPatterns);
             sb.append(", include keys:");
-            sb.append(m_jobConfKeyInclusionPatterns);
+            sb.append(jobConfKeyInclusionPatterns);
             sb.append(", exclude keys:");
-            sb.append(m_jobConfKeyExclusionPatterns);
+            sb.append(jobConfKeyExclusionPatterns);
         }
         return sb.toString();
     }

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/0b852cbc/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/crawler/JobHistoryDAOImpl.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/crawler/JobHistoryDAOImpl.java b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/crawler/JobHistoryDAOImpl.java
index cfd5994..0441f1f 100644
--- a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/crawler/JobHistoryDAOImpl.java
+++ b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/crawler/JobHistoryDAOImpl.java
@@ -18,6 +18,8 @@
 
 package org.apache.eagle.jpm.mr.history.crawler;
 
+import org.apache.eagle.jpm.mr.history.MRHistoryJobConfig.JobHistoryEndpointConfig;
+import org.apache.eagle.jpm.util.HDFSUtil;
 import org.apache.commons.lang3.tuple.Pair;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
@@ -32,36 +34,32 @@ import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Comparator;
 import java.util.List;
-import org.apache.eagle.jpm.util.HDFSUtil;
-import org.apache.eagle.jpm.mr.history.MRHistoryJobConfig.JobHistoryEndpointConfig;
 
 public class JobHistoryDAOImpl extends AbstractJobHistoryDAO {
     private static final Logger LOG = LoggerFactory.getLogger(JobHistoryDAOImpl.class);
 
-    private Configuration m_conf = new Configuration();
+    private Configuration conf = new Configuration();
 
-    private FileSystem m_hdfs;
+    private FileSystem hdfs;
 
     public JobHistoryDAOImpl(JobHistoryEndpointConfig endpointConfig) throws Exception {
         super(endpointConfig.basePath, endpointConfig.pathContainsJobTrackerName, endpointConfig.jobTrackerName);
-        this.m_conf.set("fs.defaultFS", endpointConfig.nnEndpoint);
-        this.m_conf.setBoolean("fs.hdfs.impl.disable.cache", true);
+        this.conf.set("fs.defaultFS", endpointConfig.nnEndpoint);
+        this.conf.setBoolean("fs.hdfs.impl.disable.cache", true);
         if (!endpointConfig.principal.equals("")) {
-            this.m_conf.set("hdfs.kerberos.principal", endpointConfig.principal);
-            this.m_conf.set("hdfs.keytab.file", endpointConfig.keyTab);
+            this.conf.set("hdfs.kerberos.principal", endpointConfig.principal);
+            this.conf.set("hdfs.keytab.file", endpointConfig.keyTab);
         }
         LOG.info("file system:" + endpointConfig.nnEndpoint);
-        m_hdfs = HDFSUtil.getFileSystem(m_conf);
+        hdfs = HDFSUtil.getFileSystem(conf);
     }
 
     @Override
     public void freshFileSystem() throws Exception {
         try {
-            m_hdfs.close();
-        } catch (Exception e) {
-
+            hdfs.close();
         } finally {
-            m_hdfs = HDFSUtil.getFileSystem(m_conf);
+            hdfs = HDFSUtil.getFileSystem(conf);
         }
     }
 
@@ -70,20 +68,17 @@ public class JobHistoryDAOImpl extends AbstractJobHistoryDAO {
         String latestJobTrackerName = null;
         try {
             Path hdfsFile = new Path(basePath);
-            FileStatus[] files = m_hdfs.listStatus(hdfsFile);
+            FileStatus[] files = hdfs.listStatus(hdfsFile);
 
             // Sort by modification time as order of desc
-            Arrays.sort(files, new Comparator<FileStatus>() {
-                @Override
-                public int compare(FileStatus o1, FileStatus o2) {
-                    long comp = parseJobTrackerNameTimestamp(o1.getPath().toString()) - parseJobTrackerNameTimestamp(o2.getPath().toString());
-                    if (comp > 0l) {
-                        return -1;
-                    } else if (comp < 0l) {
-                        return 1;
-                    }
-                    return 0;
+            Arrays.sort(files, (o1, o2) -> {
+                long comp = parseJobTrackerNameTimestamp(o1.getPath().toString()) - parseJobTrackerNameTimestamp(o2.getPath().toString());
+                if (comp > 0L) {
+                    return -1;
+                } else if (comp < 0L) {
+                    return 1;
                 }
+                return 0;
             });
 
             for (FileStatus fs : files) {
@@ -94,7 +89,7 @@ public class JobHistoryDAOImpl extends AbstractJobHistoryDAO {
                     break;
                 }
             }
-        } catch(Exception ex) {
+        } catch (Exception ex) {
             LOG.error("fail read job tracker name " + basePath, ex);
             throw ex;
         }
@@ -108,7 +103,7 @@ public class JobHistoryDAOImpl extends AbstractJobHistoryDAO {
         LOG.info("crawl serial numbers under one day : " + dailyPath);
         try {
             Path hdfsFile = new Path(dailyPath);
-            FileStatus[] files = m_hdfs.listStatus(hdfsFile);
+            FileStatus[] files = hdfs.listStatus(hdfsFile);
             for (FileStatus fs : files) {
                 if (fs.isDir()) {
                     serialNumbers.add(fs.getPath().getName());
@@ -125,7 +120,8 @@ public class JobHistoryDAOImpl extends AbstractJobHistoryDAO {
         }
         StringBuilder sb = new StringBuilder();
         for (String sn : serialNumbers) {
-            sb.append(sn);sb.append(",");
+            sb.append(sn);
+            sb.append(",");
         }
         LOG.info("crawled serialNumbers: " + sb);
         return serialNumbers;
@@ -133,18 +129,19 @@ public class JobHistoryDAOImpl extends AbstractJobHistoryDAO {
 
     @SuppressWarnings("deprecation")
     @Override
-    public List<Pair<Long, String> > readFileNames(int year, int month, int day, int serialNumber) throws Exception {
+    public List<Pair<Long, String>> readFileNames(int year, int month, int day, int serialNumber) throws Exception {
         LOG.info("crawl file names under one serial number : " + year + "/" + month + "/" + day + ":" + serialNumber);
-        List<Pair<Long, String> > jobFileNames = new ArrayList<>();
+        List<Pair<Long, String>> jobFileNames = new ArrayList<>();
         String serialPath = buildWholePathToSerialNumber(year, month, day, serialNumber);
         try {
             Path hdfsFile = new Path(serialPath);
             // filter those files which is job configuration file in xml format
-            FileStatus[] files = m_hdfs.listStatus(hdfsFile, new PathFilter(){
+            FileStatus[] files = hdfs.listStatus(hdfsFile, new PathFilter() {
                 @Override
-                public boolean accept(Path path){
-                    if (path.getName().endsWith(".xml"))
+                public boolean accept(Path path) {
+                    if (path.getName().endsWith(".xml")) {
                         return false;
+                    }
                     return true;
                 }
             });
@@ -156,7 +153,8 @@ public class JobHistoryDAOImpl extends AbstractJobHistoryDAO {
             if (LOG.isDebugEnabled()) {
                 StringBuilder sb = new StringBuilder();
                 for (Pair<Long, String> sn : jobFileNames) {
-                    sb.append(sn.getRight());sb.append(",");
+                    sb.append(sn.getRight());
+                    sb.append(",");
                 }
                 LOG.debug("crawled: " + sb);
             }
@@ -168,7 +166,7 @@ public class JobHistoryDAOImpl extends AbstractJobHistoryDAO {
     }
 
     /**
-     * it's the responsibility of caller to close input stream
+     * it's the responsibility of caller to close input stream.
      */
     @Override
     public InputStream getJHFFileContentAsStream(int year, int month, int day, int serialNumber, String jobHistoryFileName) throws Exception {
@@ -176,26 +174,28 @@ public class JobHistoryDAOImpl extends AbstractJobHistoryDAO {
         LOG.info("Read job history file: " + path);
         try {
             Path hdfsFile = new Path(path);
-            return m_hdfs.open(hdfsFile);
-        } catch(Exception ex) {
+            return hdfs.open(hdfsFile);
+        } catch (Exception ex) {
             LOG.error("fail getting hdfs file inputstream " + path, ex);
             throw ex;
         }
     }
 
     /**
-     * it's the responsibility of caller to close input stream
+     * it's the responsibility of caller to close input stream.
      */
     @Override
     public InputStream getJHFConfContentAsStream(int year, int month, int day, int serialNumber, String jobHistoryFileName) throws Exception {
         String path = buildWholePathToJobConfFile(year, month, day, serialNumber,jobHistoryFileName);
-        if (path  == null) return null;
+        if (path  == null) {
+            return null;
+        }
 
         LOG.info("Read job conf file: " + path);
         try {
             Path hdfsFile = new Path(path);
-            return m_hdfs.open(hdfsFile);
-        } catch(Exception ex) {
+            return hdfs.open(hdfsFile);
+        } catch (Exception ex) {
             LOG.error("fail getting job configuration input stream from " + path, ex);
             throw ex;
         }

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/0b852cbc/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/crawler/JobHistoryLCM.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/crawler/JobHistoryLCM.java b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/crawler/JobHistoryLCM.java
index de8d3f7..f428b11 100644
--- a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/crawler/JobHistoryLCM.java
+++ b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/crawler/JobHistoryLCM.java
@@ -25,12 +25,12 @@ import java.util.List;
 
 /**
  * Define various operations on job history file resource for lifecycle management
- *
+ * <p></p>
  * The job history file directory structure supported is as follows:
- * <basePath>/<jobTrackerName>/<year>/<month>/<day>/<serialNumber>/<jobHistoryFileName>
- *
- * In some hadoop version, <jobTrackerName> is not included
- *
+ * basePath/jobTrackerName/year/month/day/serialNumber/jobHistoryFileName
+ * <p></p>
+ * In some hadoop version, jobTrackerName is not included
+ * <p></p>
  * The operations involved in resource read
  * - list job tracker names under basePath (mostly basePath is configured in entry mapreduce.jobhistory.done-dir of mapred-site.xml)
  * - list serial numbers under one day
@@ -40,7 +40,9 @@ import java.util.List;
  */
 public interface JobHistoryLCM {
     String calculateJobTrackerName(String basePath) throws Exception;
+
     /**
+     * ...
      * @param year
      * @param month 0-based or 1-based month depending on hadoop cluster setting
      * @param day
@@ -48,7 +50,9 @@ public interface JobHistoryLCM {
      * @throws Exception
      */
     List<String> readSerialNumbers(int year, int month, int day) throws Exception;
+
     /**
+     * ...
      * @param year
      * @param month 0-based or 1-based month depending on hadoop cluster setting
      * @param day
@@ -56,8 +60,10 @@ public interface JobHistoryLCM {
      * @return
      * @throws Exception
      */
-    List<Pair<Long, String> > readFileNames(int year, int month, int day, int serialNumber) throws Exception;
+    List<Pair<Long, String>> readFileNames(int year, int month, int day, int serialNumber) throws Exception;
+
     /**
+     * ...
      * @param year
      * @param month 0-based or 1-based month depending on hadoop cluster setting
      * @param day
@@ -67,7 +73,9 @@ public interface JobHistoryLCM {
      * @throws Exception
      */
     void readFileContent(int year, int month, int day, int serialNumber, String jobHistoryFileName, JHFInputStreamCallback reader) throws Exception;
+
     /**
+     * ...
      * @param year
      * @param month 0-based or 1-based month depending on hadoop cluster setting
      * @param day
@@ -77,10 +85,8 @@ public interface JobHistoryLCM {
      * @throws Exception
      */
     InputStream getJHFFileContentAsStream(int year, int month, int day, int serialNumber, String jobHistoryFileName) throws Exception;
+
     InputStream getJHFConfContentAsStream(int year, int month, int day, int serialNumber, String jobConfFileName) throws Exception;
 
-    /**
-     *
-     */
     void freshFileSystem() throws Exception;
 }

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/0b852cbc/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/crawler/JobHistorySpoutCollectorInterceptor.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/crawler/JobHistorySpoutCollectorInterceptor.java b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/crawler/JobHistorySpoutCollectorInterceptor.java
index 211dd9d..a7dc9a8 100644
--- a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/crawler/JobHistorySpoutCollectorInterceptor.java
+++ b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/crawler/JobHistorySpoutCollectorInterceptor.java
@@ -18,18 +18,18 @@
 
 package org.apache.eagle.jpm.mr.history.crawler;
 
-import backtype.storm.spout.SpoutOutputCollector;
 import org.apache.eagle.dataproc.impl.storm.ValuesArray;
+import backtype.storm.spout.SpoutOutputCollector;
 
 public class JobHistorySpoutCollectorInterceptor implements EagleOutputCollector {
-    private SpoutOutputCollector m_collector;
+    private SpoutOutputCollector collector;
 
     public void setSpoutOutputCollector(SpoutOutputCollector collector) {
-        this.m_collector = collector;
+        this.collector = collector;
     }
 
     @Override
     public void collect(ValuesArray t) {
-        m_collector.emit(t);
+        collector.emit(t);
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/0b852cbc/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/HistoryJobEntityCreationListener.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/HistoryJobEntityCreationListener.java b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/HistoryJobEntityCreationListener.java
index 892c2ea..dec3f82 100644
--- a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/HistoryJobEntityCreationListener.java
+++ b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/HistoryJobEntityCreationListener.java
@@ -23,19 +23,17 @@ import org.apache.eagle.jpm.mr.historyentity.JobBaseAPIEntity;
 
 /**
  * generalizing this listener would decouple entity creation and entity handling, also will help unit testing.
- *
- * @author yonzhang
  */
 public interface HistoryJobEntityCreationListener {
     /**
-     * job entity created event
+     * job entity created event.
      *
      * @param entity
      */
     void jobEntityCreated(JobBaseAPIEntity entity) throws Exception;
 
     /**
-     * for streaming processing, flush would help commit the last several entities
+     * for streaming processing, flush would help commit the last several entities.
      */
     void flush() throws Exception;
 }

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/0b852cbc/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JHFEventReaderBase.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JHFEventReaderBase.java b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JHFEventReaderBase.java
index 82e305a..5d3d5b4 100644
--- a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JHFEventReaderBase.java
+++ b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JHFEventReaderBase.java
@@ -41,30 +41,30 @@ import java.util.regex.Pattern;
  */
 public abstract class JHFEventReaderBase extends JobEntityCreationPublisher implements Closeable {
     private static final Logger LOG = LoggerFactory.getLogger(JHFEventReaderBase.class);
-    protected Map<String, String> m_baseTags;
-    protected JobEventAPIEntity m_jobSubmitEventEntity;
-    protected JobEventAPIEntity m_jobLaunchEventEntity;
-    protected int m_numTotalMaps;
-    protected int m_numTotalReduces;
-    protected JobEventAPIEntity m_jobFinishEventEntity;
-    protected JobExecutionAPIEntity m_jobExecutionEntity;
-    protected Map<String, Long> m_taskStartTime;
+    protected Map<String, String> baseTags;
+    protected JobEventAPIEntity jobSubmitEventEntity;
+    protected JobEventAPIEntity jobLaunchEventEntity;
+    protected int numTotalMaps;
+    protected int numTotalReduces;
+    protected JobEventAPIEntity jobFinishEventEntity;
+    protected JobExecutionAPIEntity jobExecutionEntity;
+    protected Map<String, Long> taskStartTime;
     // taskAttemptID to task attempt startTime
-    protected Map<String, Long> m_taskAttemptStartTime;
+    protected Map<String, Long> taskAttemptStartTime;
 
     // taskID to host mapping, for task it's the host where the last attempt runs on
-    protected Map<String, String> m_taskRunningHosts;
+    protected Map<String, String> taskRunningHosts;
     // hostname to rack mapping
-    protected Map<String, String> m_host2RackMapping;
+    protected Map<String, String> host2RackMapping;
 
-    protected String m_jobId;
-    protected String m_jobName;
-    protected String m_jobType;
-    protected String m_jobDefId;
-    protected String m_user;
-    protected String m_queueName;
-    protected Long m_jobLauchTime;
-    protected JobHistoryContentFilter m_filter;
+    protected String jobId;
+    protected String jobName;
+    protected String jobType;
+    protected String jobDefId;
+    protected String user;
+    protected String queueName;
+    protected Long jobLaunchTime;
+    protected JobHistoryContentFilter filter;
 
     protected final List<HistoryJobEntityLifecycleListener> jobEntityLifecycleListeners = new ArrayList<>();
 
@@ -90,42 +90,42 @@ public abstract class JHFEventReaderBase extends JobEntityCreationPublisher impl
     }
 
     /**
-     * baseTags stores the basic tag name values which might be used for persisting various entities
+     * baseTags stores the basic tag name values which might be used for persisting various entities.
      * baseTags includes: cluster, datacenter and jobName
      * baseTags are used for all job/task related entities
      *
      * @param baseTags
      */
     public JHFEventReaderBase(Map<String, String> baseTags, Configuration configuration, JobHistoryContentFilter filter) {
-        this.m_filter = filter;
+        this.filter = filter;
 
-        this.m_baseTags = baseTags;
-        m_jobSubmitEventEntity = new JobEventAPIEntity();
-        m_jobSubmitEventEntity.setTags(new HashMap<>(baseTags));
+        this.baseTags = baseTags;
+        jobSubmitEventEntity = new JobEventAPIEntity();
+        jobSubmitEventEntity.setTags(new HashMap<>(baseTags));
 
-        m_jobLaunchEventEntity = new JobEventAPIEntity();
-        m_jobLaunchEventEntity.setTags(new HashMap<>(baseTags));
+        jobLaunchEventEntity = new JobEventAPIEntity();
+        jobLaunchEventEntity.setTags(new HashMap<>(baseTags));
 
-        m_jobFinishEventEntity = new JobEventAPIEntity();
-        m_jobFinishEventEntity.setTags(new HashMap<>(baseTags));
+        jobFinishEventEntity = new JobEventAPIEntity();
+        jobFinishEventEntity.setTags(new HashMap<>(baseTags));
 
-        m_jobExecutionEntity = new JobExecutionAPIEntity();
-        m_jobExecutionEntity.setTags(new HashMap<>(baseTags));
+        jobExecutionEntity = new JobExecutionAPIEntity();
+        jobExecutionEntity.setTags(new HashMap<>(baseTags));
 
-        m_taskRunningHosts = new HashMap<>();
+        taskRunningHosts = new HashMap<>();
 
-        m_host2RackMapping = new HashMap<>();
+        host2RackMapping = new HashMap<>();
 
-        m_taskStartTime = new HashMap<>();
-        m_taskAttemptStartTime = new HashMap<>();
+        taskStartTime = new HashMap<>();
+        taskAttemptStartTime = new HashMap<>();
 
         this.configuration = configuration;
 
-        if (this.configuration != null && this.m_jobType == null) {
+        if (this.configuration != null && this.jobType == null) {
             this.setJobType(fetchJobType(this.configuration).toString());
         }
-        this.sumMapTaskDuration = 0l;
-        this.sumReduceTaskDuration = 0l;
+        this.sumMapTaskDuration = 0L;
+        this.sumReduceTaskDuration = 0L;
     }
 
     public void register(HistoryJobEntityLifecycleListener lifecycleListener) {
@@ -135,8 +135,8 @@ public abstract class JHFEventReaderBase extends JobEntityCreationPublisher impl
     @Override
     public void close() throws IOException {
         // check if this job history file is complete
-        if (m_jobExecutionEntity.getEndTime() == 0L) {
-            throw new IOException(new JHFWriteNotCompletedException(m_jobId));
+        if (jobExecutionEntity.getEndTime() == 0L) {
+            throw new IOException(new JHFWriteNotCompletedException(jobId));
         }
         try {
             flush();
@@ -154,134 +154,135 @@ public abstract class JHFEventReaderBase extends JobEntityCreationPublisher impl
     }
 
     /**
+     * ...
      * @param id
      */
     private void setJobID(String id) {
-        this.m_jobId = id;
+        this.jobId = id;
     }
 
     private void setJobType(String jobType) {
-        this.m_jobType = jobType;
+        this.jobType = jobType;
     }
 
     protected void handleJob(EventType eventType, Map<Keys, String> values, Object totalCounters) throws Exception {
         String id = values.get(Keys.JOBID);
 
-        if (m_jobId == null) {
+        if (jobId == null) {
             setJobID(id);
-        } else if (!m_jobId.equals(id)) {
-            String msg = "Current job ID '" + id + "' does not match previously stored value '" + m_jobId + "'";
+        } else if (!jobId.equals(id)) {
+            String msg = "Current job ID '" + id + "' does not match previously stored value '" + jobId + "'";
             LOG.error(msg);
             throw new ImportException(msg);
         }
 
         if (values.get(Keys.SUBMIT_TIME) != null) {  // job submitted
-            m_jobSubmitEventEntity.setTimestamp(Long.valueOf(values.get(Keys.SUBMIT_TIME)));
-            m_user = values.get(Keys.USER);
-            m_queueName = values.get(Keys.JOB_QUEUE);
-            m_jobName = values.get(Keys.JOBNAME);
+            jobSubmitEventEntity.setTimestamp(Long.valueOf(values.get(Keys.SUBMIT_TIME)));
+            user = values.get(Keys.USER);
+            queueName = values.get(Keys.JOB_QUEUE);
+            jobName = values.get(Keys.JOBNAME);
 
             // If given job name then use it as norm job name, otherwise use eagle JobNameNormalization rule to generate.
             String jobDefId = null;
             if (configuration != null) {
-                jobDefId = configuration.get(m_filter.getJobNameKey());
+                jobDefId = configuration.get(filter.getJobNameKey());
             }
 
             if (jobDefId == null) {
-                m_jobDefId = JobNameNormalization.getInstance().normalize(m_jobName);
+                this.jobDefId = JobNameNormalization.getInstance().normalize(jobName);
             } else {
                 LOG.debug("Got JobDefId from job configuration for " + id + ": " + jobDefId);
-                m_jobDefId = jobDefId;
+                this.jobDefId = jobDefId;
             }
 
-            LOG.info("JobDefId of " + id + ": " + m_jobDefId);
+            LOG.info("JobDefId of " + id + ": " + this.jobDefId);
 
-            m_jobSubmitEventEntity.getTags().put(MRJobTagName.USER.toString(), m_user);
-            m_jobSubmitEventEntity.getTags().put(MRJobTagName.JOB_ID.toString(), m_jobId);
-            m_jobSubmitEventEntity.getTags().put(MRJobTagName.JOB_STATUS.toString(), EagleJobStatus.SUBMITTED.name());
-            m_jobSubmitEventEntity.getTags().put(MRJobTagName.JOB_NAME.toString(), m_jobName);
-            m_jobSubmitEventEntity.getTags().put(MRJobTagName.JOD_DEF_ID.toString(), m_jobDefId);
-            m_jobExecutionEntity.getTags().put(MRJobTagName.JOB_TYPE.toString(), this.m_jobType);
-            entityCreated(m_jobSubmitEventEntity);
+            jobSubmitEventEntity.getTags().put(MRJobTagName.USER.toString(), user);
+            jobSubmitEventEntity.getTags().put(MRJobTagName.JOB_ID.toString(), jobId);
+            jobSubmitEventEntity.getTags().put(MRJobTagName.JOB_STATUS.toString(), EagleJobStatus.SUBMITTED.name());
+            jobSubmitEventEntity.getTags().put(MRJobTagName.JOB_NAME.toString(), jobName);
+            jobSubmitEventEntity.getTags().put(MRJobTagName.JOD_DEF_ID.toString(), this.jobDefId);
+            jobExecutionEntity.getTags().put(MRJobTagName.JOB_TYPE.toString(), this.jobType);
+            entityCreated(jobSubmitEventEntity);
         } else if (values.get(Keys.LAUNCH_TIME) != null) {  // job launched
-            m_jobLaunchEventEntity.setTimestamp(Long.valueOf(values.get(Keys.LAUNCH_TIME)));
-            m_jobLauchTime = m_jobLaunchEventEntity.getTimestamp();
-            m_jobLaunchEventEntity.getTags().put(MRJobTagName.USER.toString(), m_user);
-            m_jobLaunchEventEntity.getTags().put(MRJobTagName.JOB_ID.toString(), m_jobId);
-            m_jobLaunchEventEntity.getTags().put(MRJobTagName.JOB_STATUS.toString(), EagleJobStatus.LAUNCHED.name());
-            m_jobLaunchEventEntity.getTags().put(MRJobTagName.JOB_NAME.toString(), m_jobName);
-            m_jobLaunchEventEntity.getTags().put(MRJobTagName.JOD_DEF_ID.toString(), m_jobDefId);
-            m_jobLaunchEventEntity.getTags().put(MRJobTagName.JOB_TYPE.toString(), this.m_jobType);
-            m_numTotalMaps = Integer.valueOf(values.get(Keys.TOTAL_MAPS));
-            m_numTotalReduces = Integer.valueOf(values.get(Keys.TOTAL_REDUCES));
-            entityCreated(m_jobLaunchEventEntity);
+            jobLaunchEventEntity.setTimestamp(Long.valueOf(values.get(Keys.LAUNCH_TIME)));
+            jobLaunchTime = jobLaunchEventEntity.getTimestamp();
+            jobLaunchEventEntity.getTags().put(MRJobTagName.USER.toString(), user);
+            jobLaunchEventEntity.getTags().put(MRJobTagName.JOB_ID.toString(), jobId);
+            jobLaunchEventEntity.getTags().put(MRJobTagName.JOB_STATUS.toString(), EagleJobStatus.LAUNCHED.name());
+            jobLaunchEventEntity.getTags().put(MRJobTagName.JOB_NAME.toString(), jobName);
+            jobLaunchEventEntity.getTags().put(MRJobTagName.JOD_DEF_ID.toString(), jobDefId);
+            jobLaunchEventEntity.getTags().put(MRJobTagName.JOB_TYPE.toString(), this.jobType);
+            numTotalMaps = Integer.valueOf(values.get(Keys.TOTAL_MAPS));
+            numTotalReduces = Integer.valueOf(values.get(Keys.TOTAL_REDUCES));
+            entityCreated(jobLaunchEventEntity);
         } else if (values.get(Keys.FINISH_TIME) != null) {  // job finished
-            m_jobFinishEventEntity.setTimestamp(Long.valueOf(values.get(Keys.FINISH_TIME)));
-            m_jobFinishEventEntity.getTags().put(MRJobTagName.USER.toString(), m_user);
-            m_jobFinishEventEntity.getTags().put(MRJobTagName.JOB_ID.toString(), m_jobId);
-            m_jobFinishEventEntity.getTags().put(MRJobTagName.JOB_STATUS.toString(), values.get(Keys.JOB_STATUS));
-            m_jobFinishEventEntity.getTags().put(MRJobTagName.JOB_NAME.toString(), m_jobName);
-            m_jobFinishEventEntity.getTags().put(MRJobTagName.JOD_DEF_ID.toString(), m_jobDefId);
-            m_jobFinishEventEntity.getTags().put(MRJobTagName.JOB_TYPE.toString(), this.m_jobType);
-            entityCreated(m_jobFinishEventEntity);
+            jobFinishEventEntity.setTimestamp(Long.valueOf(values.get(Keys.FINISH_TIME)));
+            jobFinishEventEntity.getTags().put(MRJobTagName.USER.toString(), user);
+            jobFinishEventEntity.getTags().put(MRJobTagName.JOB_ID.toString(), jobId);
+            jobFinishEventEntity.getTags().put(MRJobTagName.JOB_STATUS.toString(), values.get(Keys.JOB_STATUS));
+            jobFinishEventEntity.getTags().put(MRJobTagName.JOB_NAME.toString(), jobName);
+            jobFinishEventEntity.getTags().put(MRJobTagName.JOD_DEF_ID.toString(), jobDefId);
+            jobFinishEventEntity.getTags().put(MRJobTagName.JOB_TYPE.toString(), this.jobType);
+            entityCreated(jobFinishEventEntity);
 
             // populate jobExecutionEntity entity
-            m_jobExecutionEntity.getTags().put(MRJobTagName.USER.toString(), m_user);
-            m_jobExecutionEntity.getTags().put(MRJobTagName.JOB_ID.toString(), m_jobId);
-            m_jobExecutionEntity.getTags().put(MRJobTagName.JOB_NAME.toString(), m_jobName);
-            m_jobExecutionEntity.getTags().put(MRJobTagName.JOD_DEF_ID.toString(), m_jobDefId);
-            m_jobExecutionEntity.getTags().put(MRJobTagName.JOB_QUEUE.toString(), m_queueName);
-            m_jobExecutionEntity.getTags().put(MRJobTagName.JOB_TYPE.toString(), this.m_jobType);
-
-            m_jobExecutionEntity.setCurrentState(values.get(Keys.JOB_STATUS));
-            m_jobExecutionEntity.setStartTime(m_jobLaunchEventEntity.getTimestamp());
-            m_jobExecutionEntity.setEndTime(m_jobFinishEventEntity.getTimestamp());
-            m_jobExecutionEntity.setDurationTime(m_jobExecutionEntity.getEndTime() - m_jobExecutionEntity.getStartTime());
-            m_jobExecutionEntity.setTimestamp(m_jobLaunchEventEntity.getTimestamp());
-            m_jobExecutionEntity.setSubmissionTime(m_jobSubmitEventEntity.getTimestamp());
+            jobExecutionEntity.getTags().put(MRJobTagName.USER.toString(), user);
+            jobExecutionEntity.getTags().put(MRJobTagName.JOB_ID.toString(), jobId);
+            jobExecutionEntity.getTags().put(MRJobTagName.JOB_NAME.toString(), jobName);
+            jobExecutionEntity.getTags().put(MRJobTagName.JOD_DEF_ID.toString(), jobDefId);
+            jobExecutionEntity.getTags().put(MRJobTagName.JOB_QUEUE.toString(), queueName);
+            jobExecutionEntity.getTags().put(MRJobTagName.JOB_TYPE.toString(), this.jobType);
+
+            jobExecutionEntity.setCurrentState(values.get(Keys.JOB_STATUS));
+            jobExecutionEntity.setStartTime(jobLaunchEventEntity.getTimestamp());
+            jobExecutionEntity.setEndTime(jobFinishEventEntity.getTimestamp());
+            jobExecutionEntity.setDurationTime(jobExecutionEntity.getEndTime() - jobExecutionEntity.getStartTime());
+            jobExecutionEntity.setTimestamp(jobLaunchEventEntity.getTimestamp());
+            jobExecutionEntity.setSubmissionTime(jobSubmitEventEntity.getTimestamp());
             if (values.get(Keys.FAILED_MAPS) != null) {
                 // for Artemis
-                m_jobExecutionEntity.setNumFailedMaps(Integer.valueOf(values.get(Keys.FAILED_MAPS)));
+                jobExecutionEntity.setNumFailedMaps(Integer.valueOf(values.get(Keys.FAILED_MAPS)));
             }
             if (values.get(Keys.FAILED_REDUCES) != null) {
                 // for Artemis
-                m_jobExecutionEntity.setNumFailedReduces(Integer.valueOf(values.get(Keys.FAILED_REDUCES)));
+                jobExecutionEntity.setNumFailedReduces(Integer.valueOf(values.get(Keys.FAILED_REDUCES)));
             }
-            m_jobExecutionEntity.setNumFinishedMaps(Integer.valueOf(values.get(Keys.FINISHED_MAPS)));
-            m_jobExecutionEntity.setNumFinishedReduces(Integer.valueOf(values.get(Keys.FINISHED_REDUCES)));
-            m_jobExecutionEntity.setNumTotalMaps(m_numTotalMaps);
-            m_jobExecutionEntity.setNumTotalReduces(m_numTotalReduces);
+            jobExecutionEntity.setNumFinishedMaps(Integer.valueOf(values.get(Keys.FINISHED_MAPS)));
+            jobExecutionEntity.setNumFinishedReduces(Integer.valueOf(values.get(Keys.FINISHED_REDUCES)));
+            jobExecutionEntity.setNumTotalMaps(numTotalMaps);
+            jobExecutionEntity.setNumTotalReduces(numTotalReduces);
             if (values.get(Keys.COUNTERS) != null || totalCounters != null) {
                 JobCounters jobCounters = parseCounters(totalCounters);
-                m_jobExecutionEntity.setJobCounters(jobCounters);
+                jobExecutionEntity.setJobCounters(jobCounters);
                 if (jobCounters.getCounters().containsKey(Constants.JOB_COUNTER)) {
                     Map<String, Long> counters = jobCounters.getCounters().get(Constants.JOB_COUNTER);
                     if (counters.containsKey(Constants.JobCounter.DATA_LOCAL_MAPS.toString())) {
-                        m_jobExecutionEntity.setDataLocalMaps(counters.get(Constants.JobCounter.DATA_LOCAL_MAPS.toString()).intValue());
+                        jobExecutionEntity.setDataLocalMaps(counters.get(Constants.JobCounter.DATA_LOCAL_MAPS.toString()).intValue());
                     }
 
                     if (counters.containsKey(Constants.JobCounter.RACK_LOCAL_MAPS.toString())) {
-                        m_jobExecutionEntity.setRackLocalMaps(counters.get(Constants.JobCounter.RACK_LOCAL_MAPS.toString()).intValue());
+                        jobExecutionEntity.setRackLocalMaps(counters.get(Constants.JobCounter.RACK_LOCAL_MAPS.toString()).intValue());
                     }
 
                     if (counters.containsKey(Constants.JobCounter.TOTAL_LAUNCHED_MAPS.toString())) {
-                        m_jobExecutionEntity.setTotalLaunchedMaps(counters.get(Constants.JobCounter.TOTAL_LAUNCHED_MAPS.toString()).intValue());
+                        jobExecutionEntity.setTotalLaunchedMaps(counters.get(Constants.JobCounter.TOTAL_LAUNCHED_MAPS.toString()).intValue());
                     }
                 }
 
-                if (m_jobExecutionEntity.getTotalLaunchedMaps() > 0) {
-                    m_jobExecutionEntity.setDataLocalMapsPercentage(m_jobExecutionEntity.getDataLocalMaps() * 1.0 / m_jobExecutionEntity.getTotalLaunchedMaps());
-                    m_jobExecutionEntity.setRackLocalMapsPercentage(m_jobExecutionEntity.getRackLocalMaps() * 1.0 / m_jobExecutionEntity.getTotalLaunchedMaps());
+                if (jobExecutionEntity.getTotalLaunchedMaps() > 0) {
+                    jobExecutionEntity.setDataLocalMapsPercentage(jobExecutionEntity.getDataLocalMaps() * 1.0 / jobExecutionEntity.getTotalLaunchedMaps());
+                    jobExecutionEntity.setRackLocalMapsPercentage(jobExecutionEntity.getRackLocalMaps() * 1.0 / jobExecutionEntity.getTotalLaunchedMaps());
                 }
             }
-            m_jobExecutionEntity.setAvgMapTaskDuration(this.sumMapTaskDuration * 1.0 / m_numTotalMaps);
-            if (m_numTotalReduces == 0) {
-                m_jobExecutionEntity.setMaxReduceTaskDuration(0);
-                m_jobExecutionEntity.setAvgReduceTaskDuration(0);
+            jobExecutionEntity.setAvgMapTaskDuration(this.sumMapTaskDuration * 1.0 / numTotalMaps);
+            if (numTotalReduces == 0) {
+                jobExecutionEntity.setMaxReduceTaskDuration(0);
+                jobExecutionEntity.setAvgReduceTaskDuration(0);
             } else {
-                m_jobExecutionEntity.setAvgReduceTaskDuration(this.sumReduceTaskDuration * 1.0 / m_numTotalReduces);
+                jobExecutionEntity.setAvgReduceTaskDuration(this.sumReduceTaskDuration * 1.0 / numTotalReduces);
             }
-            entityCreated(m_jobExecutionEntity);
+            entityCreated(jobExecutionEntity);
         }
     }
 
@@ -290,8 +291,8 @@ public abstract class JHFEventReaderBase extends JobEntityCreationPublisher impl
             lifecycleListener.jobEntityCreated(entity);
         }
 
-        // job finished when passing JobExecutionAPIEntity: m_jobExecutionEntity
-        if (entity == this.m_jobExecutionEntity) {
+        // job finished when passing JobExecutionAPIEntity: jobExecutionEntity
+        if (entity == this.jobExecutionEntity) {
             for (HistoryJobEntityLifecycleListener lifecycleListener : this.jobEntityLifecycleListeners) {
                 lifecycleListener.jobFinish();
             }
@@ -317,49 +318,51 @@ public abstract class JHFEventReaderBase extends JobEntityCreationPublisher impl
         final String taskType = values.get(Keys.TASK_TYPE);
         final String taskID = values.get(Keys.TASKID);
 
-        Map<String, String> taskBaseTags = new HashMap<String, String>() {{
-            put(MRJobTagName.TASK_TYPE.toString(), taskType);
-            put(MRJobTagName.USER.toString(), m_user);
-            //put(MRJobTagName.JOB_NAME.toString(), _jobName);
-            put(MRJobTagName.JOD_DEF_ID.toString(), m_jobDefId);
-            put(MRJobTagName.JOB_TYPE.toString(), m_jobType);
-            put(MRJobTagName.JOB_ID.toString(), m_jobId);
-            put(MRJobTagName.TASK_ID.toString(), taskID);
-        }};
-        taskBaseTags.putAll(m_baseTags);
+        Map<String, String> taskBaseTags = new HashMap<String, String>() {
+            {
+                put(MRJobTagName.TASK_TYPE.toString(), taskType);
+                put(MRJobTagName.USER.toString(), user);
+                //put(MRJobTagName.JOB_NAME.toString(), _jobName);
+                put(MRJobTagName.JOD_DEF_ID.toString(), jobDefId);
+                put(MRJobTagName.JOB_TYPE.toString(), jobType);
+                put(MRJobTagName.JOB_ID.toString(), jobId);
+                put(MRJobTagName.TASK_ID.toString(), taskID);
+            }
+        };
+        taskBaseTags.putAll(baseTags);
         if (recType == RecordTypes.Task && startTime != null) { // task start, no host is assigned yet
-            m_taskStartTime.put(taskID, Long.valueOf(startTime));
+            taskStartTime.put(taskID, Long.valueOf(startTime));
         } else if (recType == RecordTypes.Task && finishTime != null) { // task finish
             // task execution entity setup
-            TaskExecutionAPIEntity entity = new TaskExecutionAPIEntity();
             Map<String, String> taskExecutionTags = new HashMap<>(taskBaseTags);
-            String hostname = m_taskRunningHosts.get(taskID);
+            String hostname = taskRunningHosts.get(taskID);
             hostname = (hostname == null) ? "" : hostname; // TODO if task fails, then no hostname
             taskExecutionTags.put(MRJobTagName.HOSTNAME.toString(), hostname);
-            taskExecutionTags.put(MRJobTagName.RACK.toString(), m_host2RackMapping.get(hostname));
+            taskExecutionTags.put(MRJobTagName.RACK.toString(), host2RackMapping.get(hostname));
+            TaskExecutionAPIEntity entity = new TaskExecutionAPIEntity();
             entity.setTags(taskExecutionTags);
-            entity.setStartTime(m_taskStartTime.get(taskID));
+            entity.setStartTime(taskStartTime.get(taskID));
             entity.setEndTime(Long.valueOf(finishTime));
             entity.setDuration(entity.getEndTime() - entity.getStartTime());
-            entity.setTimestamp(m_jobLauchTime);
+            entity.setTimestamp(jobLaunchTime);
             entity.setError(values.get(Keys.ERROR));
             entity.setTaskStatus(values.get(Keys.TASK_STATUS));
             if (values.get(Keys.COUNTERS) != null || counters != null) {
                 entity.setJobCounters(parseCounters(counters));
             }
-            long duration = entity.getEndTime() - m_jobSubmitEventEntity.getTimestamp();
-            if (taskType.equals(Constants.TaskType.MAP.toString()) && duration > m_jobExecutionEntity.getLastMapDuration()) {
-                m_jobExecutionEntity.setLastMapDuration(duration);
+            long duration = entity.getEndTime() - jobSubmitEventEntity.getTimestamp();
+            if (taskType.equals(Constants.TaskType.MAP.toString()) && duration > jobExecutionEntity.getLastMapDuration()) {
+                jobExecutionEntity.setLastMapDuration(duration);
             }
-            if (taskType.equals(Constants.TaskType.REDUCE.toString()) && duration > m_jobExecutionEntity.getLastReduceDuration()) {
-                m_jobExecutionEntity.setLastReduceDuration(duration);
+            if (taskType.equals(Constants.TaskType.REDUCE.toString()) && duration > jobExecutionEntity.getLastReduceDuration()) {
+                jobExecutionEntity.setLastReduceDuration(duration);
             }
 
-            if (taskType.equals(Constants.TaskType.MAP.toString()) && entity.getDuration() > m_jobExecutionEntity.getMaxMapTaskDuration()) {
-                m_jobExecutionEntity.setMaxMapTaskDuration(entity.getDuration());
+            if (taskType.equals(Constants.TaskType.MAP.toString()) && entity.getDuration() > jobExecutionEntity.getMaxMapTaskDuration()) {
+                jobExecutionEntity.setMaxMapTaskDuration(entity.getDuration());
             }
-            if (taskType.equals(Constants.TaskType.REDUCE.toString()) && entity.getDuration() > m_jobExecutionEntity.getMaxReduceTaskDuration()) {
-                m_jobExecutionEntity.setMaxReduceTaskDuration(entity.getDuration());
+            if (taskType.equals(Constants.TaskType.REDUCE.toString()) && entity.getDuration() > jobExecutionEntity.getMaxReduceTaskDuration()) {
+                jobExecutionEntity.setMaxReduceTaskDuration(entity.getDuration());
             }
 
             if (taskType.equals(Constants.TaskType.MAP.toString())) {
@@ -371,7 +374,7 @@ public abstract class JHFEventReaderBase extends JobEntityCreationPublisher impl
             entityCreated(entity);
             //_taskStartTime.remove(taskID); // clean this taskID
         } else if ((recType == RecordTypes.MapAttempt || recType == RecordTypes.ReduceAttempt) && startTime != null) { // task attempt start
-            m_taskAttemptStartTime.put(taskAttemptID, Long.valueOf(startTime));
+            taskAttemptStartTime.put(taskAttemptID, Long.valueOf(startTime));
         } else if ((recType == RecordTypes.MapAttempt || recType == RecordTypes.ReduceAttempt) && finishTime != null) {   // task attempt finish
             TaskAttemptExecutionAPIEntity entity = new TaskAttemptExecutionAPIEntity();
             Map<String, String> taskAttemptExecutionTags = new HashMap<>(taskBaseTags);
@@ -381,16 +384,16 @@ public abstract class JHFEventReaderBase extends JobEntityCreationPublisher impl
             taskAttemptExecutionTags.put(MRJobTagName.HOSTNAME.toString(), hostname);
             taskAttemptExecutionTags.put(MRJobTagName.RACK.toString(), rack);
             // put last attempt's hostname to task level
-            m_taskRunningHosts.put(taskID, hostname);
+            taskRunningHosts.put(taskID, hostname);
             // it is very likely that an attempt ID could be both succeeded and failed due to M/R system
             // in this case, we should ignore this attempt?
-            if (m_taskAttemptStartTime.get(taskAttemptID) == null) {
+            if (taskAttemptStartTime.get(taskAttemptID) == null) {
                 LOG.warn("task attemp has consistency issue " + taskAttemptID);
                 return;
             }
-            entity.setStartTime(m_taskAttemptStartTime.get(taskAttemptID));
+            entity.setStartTime(taskAttemptStartTime.get(taskAttemptID));
             entity.setEndTime(Long.valueOf(finishTime));
-            entity.setTimestamp(m_jobLauchTime);
+            entity.setTimestamp(jobLaunchTime);
             entity.setDuration(entity.getEndTime() - entity.getStartTime());
             entity.setTaskStatus(values.get(Keys.TASK_STATUS));
             entity.setError(values.get(Keys.ERROR));
@@ -400,7 +403,7 @@ public abstract class JHFEventReaderBase extends JobEntityCreationPublisher impl
             }
             entity.setTaskAttemptID(taskAttemptID);
             entityCreated(entity);
-            m_taskAttemptStartTime.remove(taskAttemptID);
+            taskAttemptStartTime.remove(taskAttemptID);
         } else {
             // silently ignore
             LOG.warn("It's an exceptional case ?");
@@ -410,7 +413,7 @@ public abstract class JHFEventReaderBase extends JobEntityCreationPublisher impl
     public void parseConfiguration() throws Exception {
         Map<String, String> prop = new TreeMap<>();
 
-        if (m_filter.acceptJobConfFile()) {
+        if (filter.acceptJobConfFile()) {
             Iterator<Map.Entry<String, String>> iter = configuration.iterator();
             while (iter.hasNext()) {
                 String key = iter.next().getKey();
@@ -423,28 +426,28 @@ public abstract class JHFEventReaderBase extends JobEntityCreationPublisher impl
         // check must-have keys are within prop
         if (matchMustHaveKeyPatterns(prop)) {
             JobConfigurationAPIEntity jobConfigurationEntity = new JobConfigurationAPIEntity();
-            jobConfigurationEntity.setTags(new HashMap<>(m_baseTags));
-            jobConfigurationEntity.getTags().put(MRJobTagName.USER.toString(), m_user);
-            jobConfigurationEntity.getTags().put(MRJobTagName.JOB_ID.toString(), m_jobId);
-            jobConfigurationEntity.getTags().put(MRJobTagName.JOB_NAME.toString(), m_jobName);
-            jobConfigurationEntity.getTags().put(MRJobTagName.JOD_DEF_ID.toString(), m_jobDefId);
-            jobConfigurationEntity.getTags().put(MRJobTagName.JOB_TYPE.toString(), m_jobType);
-            jobConfigurationEntity.setTimestamp(m_jobLaunchEventEntity.getTimestamp());
+            jobConfigurationEntity.setTags(new HashMap<>(baseTags));
+            jobConfigurationEntity.getTags().put(MRJobTagName.USER.toString(), user);
+            jobConfigurationEntity.getTags().put(MRJobTagName.JOB_ID.toString(), jobId);
+            jobConfigurationEntity.getTags().put(MRJobTagName.JOB_NAME.toString(), jobName);
+            jobConfigurationEntity.getTags().put(MRJobTagName.JOD_DEF_ID.toString(), jobDefId);
+            jobConfigurationEntity.getTags().put(MRJobTagName.JOB_TYPE.toString(), jobType);
+            jobConfigurationEntity.setTimestamp(jobLaunchEventEntity.getTimestamp());
 
             JobConfig jobConfig = new JobConfig();
             jobConfig.setConfig(prop);
             jobConfigurationEntity.setJobConfig(jobConfig);
-            jobConfigurationEntity.setConfigJobName(m_jobDefId);
+            jobConfigurationEntity.setConfigJobName(jobDefId);
             entityCreated(jobConfigurationEntity);
         }
     }
 
     private boolean matchMustHaveKeyPatterns(Map<String, String> prop) {
-        if (m_filter.getMustHaveJobConfKeyPatterns() == null) {
+        if (filter.getMustHaveJobConfKeyPatterns() == null) {
             return true;
         }
 
-        for (Pattern p : m_filter.getMustHaveJobConfKeyPatterns()) {
+        for (Pattern p : filter.getMustHaveJobConfKeyPatterns()) {
             boolean matched = false;
             for (String key : prop.keySet()) {
                 if (p.matcher(key).matches()) {
@@ -460,10 +463,10 @@ public abstract class JHFEventReaderBase extends JobEntityCreationPublisher impl
     }
 
     private boolean included(String key) {
-        if (m_filter.getJobConfKeyInclusionPatterns() == null) {
+        if (filter.getJobConfKeyInclusionPatterns() == null) {
             return true;
         }
-        for (Pattern p : m_filter.getJobConfKeyInclusionPatterns()) {
+        for (Pattern p : filter.getJobConfKeyInclusionPatterns()) {
             Matcher m = p.matcher(key);
             if (m.matches()) {
                 LOG.info("include key: " + p.toString());
@@ -474,10 +477,10 @@ public abstract class JHFEventReaderBase extends JobEntityCreationPublisher impl
     }
 
     private boolean excluded(String key) {
-        if (m_filter.getJobConfKeyExclusionPatterns() == null) {
+        if (filter.getJobConfKeyExclusionPatterns() == null) {
             return false;
         }
-        for (Pattern p : m_filter.getJobConfKeyExclusionPatterns()) {
+        for (Pattern p : filter.getJobConfKeyExclusionPatterns()) {
             Matcher m = p.matcher(key);
             if (m.matches()) {
                 return true;

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/0b852cbc/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JHFMRVer1EventReader.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JHFMRVer1EventReader.java b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JHFMRVer1EventReader.java
index 6932dad..e20836f 100644
--- a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JHFMRVer1EventReader.java
+++ b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JHFMRVer1EventReader.java
@@ -37,14 +37,12 @@ import java.util.Map;
 
 /**
  * Listener holds all informations related to one whole job history file, so it's stateful and does not support multithreading.
- *
- * @author yonzhang
  */
 public class JHFMRVer1EventReader extends JHFEventReaderBase implements JHFMRVer1PerLineListener {
     private static final Logger logger = LoggerFactory.getLogger(JHFMRVer1EventReader.class);
 
     /**
-     * baseTags stores the basic tag name values which might be used for persisting various entities
+     * baseTags stores the basic tag name values which might be used for persisting various entities.
      * baseTags includes: cluster, datacenter and jobName
      * baseTags are used for all job/task related entities
      *
@@ -93,12 +91,12 @@ public class JHFMRVer1EventReader extends JHFEventReaderBase implements JHFMRVer
             String[] tmp = decoratedHostname.split("/");
             hostname = tmp[tmp.length - 1];
             rack = tmp[tmp.length - 2];
-            m_host2RackMapping.put(hostname, rack);
+            host2RackMapping.put(hostname, rack);
         } else if (values.get(Keys.TASK_STATUS).equals(EagleTaskStatus.KILLED.name()) || values.get(Keys.TASK_STATUS).equals(EagleTaskStatus.FAILED.name())) {
             hostname = values.get(Keys.HOSTNAME);
             // make every effort to get RACK information
             hostname = (hostname == null) ? "" : hostname;
-            rack = m_host2RackMapping.get(hostname);
+            rack = host2RackMapping.get(hostname);
         }
 
         values.put(Keys.HOSTNAME, hostname);
@@ -149,6 +147,6 @@ public class JHFMRVer1EventReader extends JHFEventReaderBase implements JHFMRVer
     }
 
     public JobExecutionAPIEntity jobExecution() {
-        return m_jobExecutionEntity;
+        return jobExecutionEntity;
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/0b852cbc/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JHFMRVer1Parser.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JHFMRVer1Parser.java b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JHFMRVer1Parser.java
index ab59a41..2f63703 100644
--- a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JHFMRVer1Parser.java
+++ b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JHFMRVer1Parser.java
@@ -43,10 +43,10 @@ public class JHFMRVer1Parser implements JHFParserBase {
     static final Pattern pattern = Pattern.compile(KEY + "=" + "\"" + VALUE + "\"");
     static final String MAX_COUNTER_COUNT = "10000";
 
-    private JHFMRVer1EventReader m_reader;
+    private JHFMRVer1EventReader reader;
 
     public JHFMRVer1Parser(JHFMRVer1EventReader reader) {
-        this.m_reader = reader;
+        this.reader = reader;
     }
 
     /**
@@ -85,13 +85,14 @@ public class JHFMRVer1Parser implements JHFParserBase {
                     buf.append("\n");
                     continue;
                 }
-                parseLine(buf.toString(), m_reader, isEscaped);
+                parseLine(buf.toString(), this.reader, isEscaped);
                 buf = new StringBuffer();
-            } while ((line = reader.readLine()) != null);
+            }
+            while ((line = reader.readLine()) != null);
 
             // flush to tell listener that we have finished parsing
             logger.info("finish parsing job history file and close");
-            m_reader.close();
+            this.reader.close();
         } catch (Exception ex) {
             logger.error("can not parse correctly ", ex);
             throw ex;
@@ -121,10 +122,6 @@ public class JHFMRVer1Parser implements JHFParserBase {
             parseBuffer.put(Keys.valueOf(parts[0]), value);
         }
 
-//        if(conf!=null){
-//            parseBuffer.put(Keys.NORM_JOBNAME, conf.get(JPAConstants.JOB_CONF_NORM_JOBNAME_KEY));
-//        }
-
         try {
             l.handle(RecordTypes.valueOf(recType), parseBuffer);
         } catch (IllegalArgumentException ex) {

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/0b852cbc/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JHFMRVer2EventReader.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JHFMRVer2EventReader.java b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JHFMRVer2EventReader.java
index 4ca0449..f21fd41 100644
--- a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JHFMRVer2EventReader.java
+++ b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JHFMRVer2EventReader.java
@@ -33,14 +33,13 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 
-
 @InterfaceAudience.Private
 @InterfaceStability.Unstable
 public class JHFMRVer2EventReader extends JHFEventReaderBase {
     private static final Logger logger = LoggerFactory.getLogger(JHFMRVer2EventReader.class);
 
     /**
-     * Create a new Event Reader
+     * Create a new Event Reader.
      *
      * @throws IOException
      */
@@ -455,7 +454,7 @@ public class JHFMRVer2EventReader extends JHFEventReaderBase {
         String[] tmp = rackname.split("/");
         String rack = tmp[tmp.length - 1];
         values.put(Keys.RACK, rack);
-        m_host2RackMapping.put(values.get(Keys.HOSTNAME), rack);
+        host2RackMapping.put(values.get(Keys.HOSTNAME), rack);
     }
 
     private void handleMapAttemptFailed(Event wrapper) throws Exception {

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/0b852cbc/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JHFMRVer2Parser.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JHFMRVer2Parser.java b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JHFMRVer2Parser.java
index 2ccbf8d..f93f942 100644
--- a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JHFMRVer2Parser.java
+++ b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JHFMRVer2Parser.java
@@ -33,10 +33,10 @@ import java.io.InputStream;
 
 public class JHFMRVer2Parser implements JHFParserBase {
     private static final Logger logger = LoggerFactory.getLogger(JHFMRVer2Parser.class);
-    private JHFMRVer2EventReader _reader;
+    private JHFMRVer2EventReader reader;
 
     public JHFMRVer2Parser(JHFMRVer2EventReader reader) {
-        this._reader = reader;
+        this.reader = reader;
     }
 
     @SuppressWarnings( {"rawtypes", "deprecation"})
@@ -58,11 +58,11 @@ public class JHFMRVer2Parser implements JHFParserBase {
             Event wrapper;
             while ((wrapper = getNextEvent(datumReader, decoder)) != null) {
                 ++eventCtr;
-                _reader.handleEvent(wrapper);
+                reader.handleEvent(wrapper);
             }
-            _reader.parseConfiguration();
+            reader.parseConfiguration();
             // don't need put to finally as it's a kind of flushing data
-            _reader.close();
+            reader.close();
             logger.info("reader used " + (System.currentTimeMillis() - start) + "ms");
         } catch (Exception ioe) {
             logger.error("Caught exception parsing history file after " + eventCtr + " events", ioe);

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/0b852cbc/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JHFParserBase.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JHFParserBase.java b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JHFParserBase.java
index a2479ab..36a0ef7 100644
--- a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JHFParserBase.java
+++ b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JHFParserBase.java
@@ -20,12 +20,9 @@ package org.apache.eagle.jpm.mr.history.parser;
 
 import java.io.InputStream;
 
-/**
- * @author yonzhang
- */
 public interface JHFParserBase {
     /**
-     * this method will ensure to close the inputstream
+     * this method will ensure to close the inputStream.
      *
      * @param is
      * @throws Exception

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/0b852cbc/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JHFWriteNotCompletedException.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JHFWriteNotCompletedException.java b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JHFWriteNotCompletedException.java
index d5ed97a..64b9fe0 100644
--- a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JHFWriteNotCompletedException.java
+++ b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JHFWriteNotCompletedException.java
@@ -19,15 +19,11 @@
 package org.apache.eagle.jpm.mr.history.parser;
 
 /**
- * used to warn that one job history file has not yet completed writing to hdfs
+ * used to warn that one job history file has not yet completed writing to hdfs.
  * This happens when feeder catches up and the history file has not been written into hdfs completely
  *
- * @author yonzhang
  */
 public class JHFWriteNotCompletedException extends Exception {
-    /**
-     *
-     */
     private static final long serialVersionUID = -3060175780718218490L;
 
     public JHFWriteNotCompletedException(String msg) {


[18/52] [abbrv] incubator-eagle git commit: [EAGLE-501] add fields and fix bugs

Posted by yo...@apache.org.
[EAGLE-501] add fields and fix bugs

Author: wujinhu <wu...@126.com>

Closes #388 from wujinhu/EAGLE-501.


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

Branch: refs/heads/master
Commit: 7f37267160af48b5cb46b4bcbd85a3a82957b192
Parents: a846c40
Author: wujinhu <wu...@126.com>
Authored: Thu Aug 25 23:46:52 2016 +0800
Committer: Hao Chen <ha...@apache.org>
Committed: Thu Aug 25 23:46:52 2016 +0800

----------------------------------------------------------------------
 .../environment/impl/StormExecutionRuntime.java |  6 ++-
 .../mr/historyentity/JobExecutionAPIEntity.java | 44 ++++++++++++++++
 .../TaskAttemptExecutionAPIEntity.java          |  2 +-
 .../mr/runningentity/JobExecutionAPIEntity.java | 11 ++++
 eagle-jpm/eagle-jpm-mr-history/pom.xml          | 11 ++--
 .../jpm/mr/history/parser/EagleJobStatus.java   |  1 +
 .../mr/history/parser/JHFEventReaderBase.java   | 27 +++++++++-
 .../src/main/resources/application.conf         | 11 ++--
 .../jpm/mr/running/parser/MRJobParser.java      | 12 +++--
 .../mr/running/storm/MRRunningJobParseBolt.java |  1 +
 .../src/main/resources/application.conf         | 55 +++++++++++---------
 11 files changed, 141 insertions(+), 40 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/7f372671/eagle-core/eagle-app/eagle-app-base/src/main/java/org/apache/eagle/app/environment/impl/StormExecutionRuntime.java
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-app/eagle-app-base/src/main/java/org/apache/eagle/app/environment/impl/StormExecutionRuntime.java b/eagle-core/eagle-app/eagle-app-base/src/main/java/org/apache/eagle/app/environment/impl/StormExecutionRuntime.java
index 83d3592..1b989ac 100644
--- a/eagle-core/eagle-app/eagle-app-base/src/main/java/org/apache/eagle/app/environment/impl/StormExecutionRuntime.java
+++ b/eagle-core/eagle-app/eagle-app-base/src/main/java/org/apache/eagle/app/environment/impl/StormExecutionRuntime.java
@@ -60,6 +60,7 @@ public class StormExecutionRuntime implements ExecutionRuntime<StormEnvironment,
     private final static String STORM_NIMBUS_HOST_DEFAULT = "localhost";
     private final static Integer STORM_NIMBUS_THRIFT_DEFAULT = 6627;
     private final static String STORM_NIMBUS_THRIFT_CONF_PATH = "application.storm.nimbusThriftPort";
+    private static final String WORKERS = "workers";
 
     public backtype.storm.Config getStormConfig(){
         backtype.storm.Config conf = new backtype.storm.Config();
@@ -86,6 +87,9 @@ public class StormExecutionRuntime implements ExecutionRuntime<StormEnvironment,
         conf.put(backtype.storm.Config.NIMBUS_HOST, nimbusHost);
         conf.put(backtype.storm.Config.NIMBUS_THRIFT_PORT, nimbusThriftPort);
         conf.put(Config.STORM_THRIFT_TRANSPORT_PLUGIN, "backtype.storm.security.auth.SimpleTransportPlugin");
+        if (environment.config().hasPath(WORKERS)) {
+            conf.setNumWorkers(environment.config().getInt(WORKERS));
+        }
         return conf;
     }
 
@@ -96,7 +100,7 @@ public class StormExecutionRuntime implements ExecutionRuntime<StormEnvironment,
         StormTopology topology = executor.execute(config, environment);
         LOG.info("Starting {} ({})",topologyName,executor.getClass().getCanonicalName());
         Config conf = getStormConfig();
-        if(config.getString("mode") == ApplicationEntity.Mode.CLUSTER.name()){
+        if(config.getString("mode").equals(ApplicationEntity.Mode.CLUSTER.name())){
             String jarFile = config.hasPath("jarPath") ? config.getString("jarPath") : null;
             if(jarFile == null){
                 jarFile = DynamicJarPathFinder.findPath(executor.getClass());

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/7f372671/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/mr/historyentity/JobExecutionAPIEntity.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/mr/historyentity/JobExecutionAPIEntity.java b/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/mr/historyentity/JobExecutionAPIEntity.java
index 1f75f07..97e77b2 100644
--- a/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/mr/historyentity/JobExecutionAPIEntity.java
+++ b/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/mr/historyentity/JobExecutionAPIEntity.java
@@ -81,6 +81,14 @@ public class JobExecutionAPIEntity extends JobBaseAPIEntity {
     private double maxMapTaskDuration;
     @Column("x")
     private double maxReduceTaskDuration;
+    @Column("y")
+    private int totalMapAttempts;
+    @Column("z")
+    private int failedMapAttempts;
+    @Column("ab")
+    private int totalReduceAttempts;
+    @Column("ac")
+    private int failedReduceAttempts;
 
     public long getDurationTime() {
         return durationTime;
@@ -288,4 +296,40 @@ public class JobExecutionAPIEntity extends JobBaseAPIEntity {
         this.maxReduceTaskDuration = maxReduceTaskDuration;
         valueChanged("maxReduceTaskDuration");
     }
+
+    public int getTotalMapAttempts() {
+        return totalMapAttempts;
+    }
+
+    public void setTotalMapAttempts(int totalMapAttempts) {
+        this.totalMapAttempts = totalMapAttempts;
+        valueChanged("totalMapAttempts");
+    }
+
+    public int getFailedMapAttempts() {
+        return failedMapAttempts;
+    }
+
+    public void setFailedMapAttempts(int failedMapAttempts) {
+        this.failedMapAttempts = failedMapAttempts;
+        valueChanged("failedMapAttempts");
+    }
+
+    public int getTotalReduceAttempts() {
+        return totalReduceAttempts;
+    }
+
+    public void setTotalReduceAttempts(int totalReduceAttempts) {
+        this.totalReduceAttempts = totalReduceAttempts;
+        valueChanged("totalReduceAttempts");
+    }
+
+    public int getFailedReduceAttempts() {
+        return failedReduceAttempts;
+    }
+
+    public void setFailedReduceAttempts(int failedReduceAttempts) {
+        this.failedReduceAttempts = failedReduceAttempts;
+        valueChanged("failedReduceAttempts");
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/7f372671/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/mr/historyentity/TaskAttemptExecutionAPIEntity.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/mr/historyentity/TaskAttemptExecutionAPIEntity.java b/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/mr/historyentity/TaskAttemptExecutionAPIEntity.java
index 620ee1f..fd96828 100644
--- a/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/mr/historyentity/TaskAttemptExecutionAPIEntity.java
+++ b/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/mr/historyentity/TaskAttemptExecutionAPIEntity.java
@@ -31,7 +31,7 @@ import org.codehaus.jackson.map.annotate.JsonSerialize;
 @TimeSeries(true)
 @Partition({"site"})
 @Indexes({
-    @Index(name = "Index_1_jobId", columns = { "jobID" }, unique = false)
+    @Index(name = "Index_1_jobId", columns = { "jobId" }, unique = false)
     })
 public class TaskAttemptExecutionAPIEntity extends JobBaseAPIEntity {
     @Column("a")

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/7f372671/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/mr/runningentity/JobExecutionAPIEntity.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/mr/runningentity/JobExecutionAPIEntity.java b/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/mr/runningentity/JobExecutionAPIEntity.java
index 86b6554..dd81eb4 100644
--- a/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/mr/runningentity/JobExecutionAPIEntity.java
+++ b/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/mr/runningentity/JobExecutionAPIEntity.java
@@ -110,6 +110,8 @@ public class JobExecutionAPIEntity extends TaggedLogAPIEntity {
     private int totalLaunchedMaps;
     @Column("aj")
     private long submissionTime;
+    @Column("ak")
+    private String internalState;
 
     public JobConfig getJobConfig() {
         return jobConfig;
@@ -434,4 +436,13 @@ public class JobExecutionAPIEntity extends TaggedLogAPIEntity {
         this.submissionTime = submissionTime;
         valueChanged("submissionTime");
     }
+
+    public String getInternalState() {
+        return internalState;
+    }
+
+    public void setInternalState(String internalState) {
+        this.internalState = internalState;
+        valueChanged("internalState");
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/7f372671/eagle-jpm/eagle-jpm-mr-history/pom.xml
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-mr-history/pom.xml b/eagle-jpm/eagle-jpm-mr-history/pom.xml
index 1ffda6a..5732237 100644
--- a/eagle-jpm/eagle-jpm-mr-history/pom.xml
+++ b/eagle-jpm/eagle-jpm-mr-history/pom.xml
@@ -56,6 +56,11 @@
         </dependency>
         <dependency>
             <groupId>org.apache.eagle</groupId>
+            <artifactId>eagle-jpm-app</artifactId>
+            <version>${project.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.eagle</groupId>
             <artifactId>eagle-data-process</artifactId>
             <version>${project.version}</version>
             <exclusions>
@@ -104,13 +109,7 @@
             <artifactId>hadoop-mapreduce-client-core</artifactId>
             <version>${hadoop.version}</version>
         </dependency>
-        <dependency>
-            <groupId>org.apache.eagle</groupId>
-            <artifactId>eagle-app-base</artifactId>
-            <version>${project.version}</version>
-        </dependency>
     </dependencies>
-
     <build>
         <resources>
             <resource>

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/7f372671/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/EagleJobStatus.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/EagleJobStatus.java b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/EagleJobStatus.java
index 0a137be..fb218e3 100644
--- a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/EagleJobStatus.java
+++ b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/EagleJobStatus.java
@@ -24,5 +24,6 @@ public enum EagleJobStatus {
     PREP,
     RUNNING,
     SUCCESS,
+    KILLED,
     FAILED;
 }

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/7f372671/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JHFEventReaderBase.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JHFEventReaderBase.java b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JHFEventReaderBase.java
index 5d3d5b4..6916aad 100644
--- a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JHFEventReaderBase.java
+++ b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JHFEventReaderBase.java
@@ -111,6 +111,8 @@ public abstract class JHFEventReaderBase extends JobEntityCreationPublisher impl
 
         jobExecutionEntity = new JobExecutionAPIEntity();
         jobExecutionEntity.setTags(new HashMap<>(baseTags));
+        jobExecutionEntity.setNumFailedMaps(0);
+        jobExecutionEntity.setNumFailedReduces(0);
 
         taskRunningHosts = new HashMap<>();
 
@@ -350,7 +352,7 @@ public abstract class JHFEventReaderBase extends JobEntityCreationPublisher impl
             if (values.get(Keys.COUNTERS) != null || counters != null) {
                 entity.setJobCounters(parseCounters(counters));
             }
-            long duration = entity.getEndTime() - jobSubmitEventEntity.getTimestamp();
+            long duration = entity.getEndTime() - jobLaunchTime;
             if (taskType.equals(Constants.TaskType.MAP.toString()) && duration > jobExecutionEntity.getLastMapDuration()) {
                 jobExecutionEntity.setLastMapDuration(duration);
             }
@@ -367,8 +369,16 @@ public abstract class JHFEventReaderBase extends JobEntityCreationPublisher impl
 
             if (taskType.equals(Constants.TaskType.MAP.toString())) {
                 this.sumMapTaskDuration += entity.getDuration();
+                if (entity.getTaskStatus().equals(EagleTaskStatus.FAILED.name())
+                    || entity.getTaskStatus().equals(EagleTaskStatus.KILLED.name())) {
+                    jobExecutionEntity.setNumFailedMaps(1 + jobExecutionEntity.getNumFailedMaps());
+                }
             } else {
                 this.sumReduceTaskDuration += entity.getDuration();
+                if (entity.getTaskStatus().equals(EagleTaskStatus.FAILED.name())
+                    || entity.getTaskStatus().equals(EagleTaskStatus.KILLED.name())) {
+                    jobExecutionEntity.setNumFailedReduces(1 + jobExecutionEntity.getNumFailedReduces());
+                }
             }
 
             entityCreated(entity);
@@ -402,6 +412,21 @@ public abstract class JHFEventReaderBase extends JobEntityCreationPublisher impl
                 entity.setJobCounters(parseCounters(counters));
             }
             entity.setTaskAttemptID(taskAttemptID);
+
+            if (recType == RecordTypes.MapAttempt) {
+                jobExecutionEntity.setTotalMapAttempts(1 + jobExecutionEntity.getTotalMapAttempts());
+                if (entity.getTaskStatus().equals(EagleTaskStatus.FAILED.name())
+                    || entity.getTaskStatus().equals(EagleTaskStatus.KILLED.name())) {
+                    jobExecutionEntity.setFailedMapAttempts(1 + jobExecutionEntity.getFailedMapAttempts());
+                }
+            } else {
+                jobExecutionEntity.setTotalReduceAttempts(1 + jobExecutionEntity.getTotalReduceAttempts());
+                if (entity.getTaskStatus().equals(EagleTaskStatus.FAILED.name())
+                    || entity.getTaskStatus().equals(EagleTaskStatus.KILLED.name())) {
+                    jobExecutionEntity.setFailedReduceAttempts(1 + jobExecutionEntity.getFailedReduceAttempts());
+                }
+            }
+
             entityCreated(entity);
             taskAttemptStartTime.remove(taskAttemptID);
         } else {

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/7f372671/eagle-jpm/eagle-jpm-mr-history/src/main/resources/application.conf
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-mr-history/src/main/resources/application.conf b/eagle-jpm/eagle-jpm-mr-history/src/main/resources/application.conf
index 13e411f..db2c716 100644
--- a/eagle-jpm/eagle-jpm-mr-history/src/main/resources/application.conf
+++ b/eagle-jpm/eagle-jpm-mr-history/src/main/resources/application.conf
@@ -15,13 +15,15 @@
 
 {
   "envContextConfig" : {
+    "env" : "local",
+    "topologyName" : "mrHistoryJob",
+    "stormConfigFile" : "storm.yaml",
     "parallelismConfig" : {
       "mrHistoryJobExecutor" : 6
     },
     "tasks" : {
       "mrHistoryJobExecutor" : 6
-    },
-    "workers" : 3
+    }
   },
 
   "jobExtractorConfig" : {
@@ -59,8 +61,11 @@
       "password": "secret"
     }
   },
-  "appId":"mr_history",
+
+  "appId":"mrHistoryJob",
   "mode":"LOCAL",
+  "workers" : 3,
+  application.storm.nimbusHost=localhost
   "MRConfigureKeys" : {
     "jobNameKey" : "eagle.job.name",
     "jobConfigKey" : "mapreduce.map.output.compress,mapreduce.map.output.compress.codec,mapreduce.output.fileoutputformat.compress,mapreduce.output.fileoutputformat.compress.type,mapreduce.output.fileoutputformat.compress.codec,mapred.output.format.class, dataplatform.etl.info,mapreduce.map.memory.mb,mapreduce.reduce.memory.mb,mapreduce.map.java.opts,mapreduce.reduce.java.opts"

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/7f372671/eagle-jpm/eagle-jpm-mr-running/src/main/java/org/apache/eagle/jpm/mr/running/parser/MRJobParser.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-mr-running/src/main/java/org/apache/eagle/jpm/mr/running/parser/MRJobParser.java b/eagle-jpm/eagle-jpm-mr-running/src/main/java/org/apache/eagle/jpm/mr/running/parser/MRJobParser.java
index 9148c0c..9e156fa 100644
--- a/eagle-jpm/eagle-jpm-mr-running/src/main/java/org/apache/eagle/jpm/mr/running/parser/MRJobParser.java
+++ b/eagle-jpm/eagle-jpm-mr-running/src/main/java/org/apache/eagle/jpm/mr/running/parser/MRJobParser.java
@@ -110,6 +110,10 @@ public class MRJobParser implements Runnable {
         this.configKeys = configKeys;
     }
 
+    public void setAppInfo(AppInfo app) {
+        this.app = app;
+    }
+
     public ParserStatus status() {
         return this.parserStatus;
     }
@@ -120,7 +124,8 @@ public class MRJobParser implements Runnable {
 
     private void finishMRJob(String mrJobId) {
         JobExecutionAPIEntity jobExecutionAPIEntity = mrJobEntityMap.get(mrJobId);
-        jobExecutionAPIEntity.setCurrentState(Constants.AppState.FINISHED.toString());
+        jobExecutionAPIEntity.setInternalState(Constants.AppState.FINISHED.toString());
+        jobExecutionAPIEntity.setCurrentState(Constants.AppState.RUNNING.toString());
         mrJobConfigs.remove(mrJobId);
         if (mrJobConfigs.size() == 0) {
             this.parserStatus = ParserStatus.APP_FINISHED;
@@ -205,6 +210,7 @@ public class MRJobParser implements Runnable {
             jobExecutionAPIEntity.setStartTime(mrJob.getStartTime());
             jobExecutionAPIEntity.setDurationTime(mrJob.getElapsedTime());
             jobExecutionAPIEntity.setCurrentState(mrJob.getState());
+            jobExecutionAPIEntity.setInternalState(mrJob.getState());
             jobExecutionAPIEntity.setNumTotalMaps(mrJob.getMapsTotal());
             jobExecutionAPIEntity.setMapsCompleted(mrJob.getMapsCompleted());
             jobExecutionAPIEntity.setNumTotalReduces(mrJob.getReducesTotal());
@@ -562,8 +568,8 @@ public class MRJobParser implements Runnable {
                     mrJobEntityMap.keySet()
                         .stream()
                         .filter(
-                            jobId -> mrJobEntityMap.get(jobId).getCurrentState().equals(Constants.AppState.FINISHED.toString())
-                                || mrJobEntityMap.get(jobId).getCurrentState().equals(Constants.AppState.FAILED.toString()))
+                            jobId -> mrJobEntityMap.get(jobId).getInternalState().equals(Constants.AppState.FINISHED.toString())
+                                || mrJobEntityMap.get(jobId).getInternalState().equals(Constants.AppState.FAILED.toString()))
                         .forEach(
                             jobId -> this.runningJobManager.delete(app.getId(), jobId));
                 }

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/7f372671/eagle-jpm/eagle-jpm-mr-running/src/main/java/org/apache/eagle/jpm/mr/running/storm/MRRunningJobParseBolt.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-mr-running/src/main/java/org/apache/eagle/jpm/mr/running/storm/MRRunningJobParseBolt.java b/eagle-jpm/eagle-jpm-mr-running/src/main/java/org/apache/eagle/jpm/mr/running/storm/MRRunningJobParseBolt.java
index 3174eb1..e918597 100644
--- a/eagle-jpm/eagle-jpm-mr-running/src/main/java/org/apache/eagle/jpm/mr/running/storm/MRRunningJobParseBolt.java
+++ b/eagle-jpm/eagle-jpm-mr-running/src/main/java/org/apache/eagle/jpm/mr/running/storm/MRRunningJobParseBolt.java
@@ -88,6 +88,7 @@ public class MRRunningJobParseBolt extends BaseRichBolt {
             LOG.info("create application parser for {}", appInfo.getId());
         } else {
             applicationParser = runningMRParsers.get(appInfo.getId());
+            applicationParser.setAppInfo(appInfo);
         }
 
         Set<String> runningParserIds = new HashSet<>(runningMRParsers.keySet());

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/7f372671/eagle-jpm/eagle-jpm-mr-running/src/main/resources/application.conf
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-mr-running/src/main/resources/application.conf b/eagle-jpm/eagle-jpm-mr-running/src/main/resources/application.conf
index 4b6d4fe..0d1de78 100644
--- a/eagle-jpm/eagle-jpm-mr-running/src/main/resources/application.conf
+++ b/eagle-jpm/eagle-jpm-mr-running/src/main/resources/application.conf
@@ -16,24 +16,27 @@
 {
   "appId":"mrRunningJob",
   "mode":"LOCAL",
+  application.storm.nimbusHost=localhost,
+  "workers" : 8,
   "envContextConfig" : {
     "env" : "local",
+    "topologyName" : "mrRunningJob",
+    "stormConfigFile" : "storm.yaml",
     "parallelismConfig" : {
       "mrRunningJobFetchSpout" : 1,
-      "mrRunningJobParseBolt" : 10
+      "mrRunningJobParseBolt" : 16
     },
     "tasks" : {
       "mrRunningJobFetchSpout" : 1,
-      "mrRunningJobParseBolt" : 10
-    },
-    "workers" : 5
+      "mrRunningJobParseBolt" : 16
+    }
   },
 
   "jobExtractorConfig" : {
     "site" : "sandbox",
     "fetchRunningJobInterval" : 60,
-    "parseJobThreadPoolSize" : 5, #job concurrent
-    "topAndBottomTaskByElapsedTime" : 50
+    "parseJobThreadPoolSize" : 6,
+    "topAndBottomTaskByElapsedTime" : 10
   },
 
   "zookeeperConfig" : {
@@ -44,9 +47,11 @@
     "zkRetryTimes" : 3,
     "zkRetryInterval" : 20000
   },
+
   "dataSourceConfig" : {
     "rmUrls": "http://sandbox.hortonworks.com:50030"
   },
+
   "eagleProps" : {
     "mailHost" : "abc.com",
     "mailDebug" : "true",
@@ -63,23 +68,23 @@
   "MRConfigureKeys" : {
     "jobNameKey" : "eagle.job.name",
     "jobConfigKey" : [
-    "mapreduce.map.output.compress",
-    "mapreduce.map.output.compress.codec",
-    "mapreduce.output.fileoutputformat.compress",
-    "mapreduce.output.fileoutputformat.compress.type",
-    "mapreduce.output.fileoutputformat.compress.codec",
-    "mapred.output.format.class",
-    "eagle.job.runid",
-    "eagle.job.runidfieldname",
-    "eagle.job.name",
-    "eagle.job.normalizedfieldname",
-    "eagle.alert.email",
-    "eagle.job.alertemailaddress",
-    "dataplatform.etl.info",
-    "mapreduce.map.memory.mb",
-    "mapreduce.reduce.memory.mb",
-    "mapreduce.map.java.opts",
-    "mapreduce.reduce.java.opts"
-  ]
+      "mapreduce.map.output.compress",
+      "mapreduce.map.output.compress.codec",
+      "mapreduce.output.fileoutputformat.compress",
+      "mapreduce.output.fileoutputformat.compress.type",
+      "mapreduce.output.fileoutputformat.compress.codec",
+      "mapred.output.format.class",
+      "eagle.job.runid",
+      "eagle.job.runidfieldname",
+      "eagle.job.name",
+      "eagle.job.normalizedfieldname",
+      "eagle.alert.email",
+      "eagle.job.alertemailaddress",
+      "dataplatform.etl.info",
+      "mapreduce.map.memory.mb",
+      "mapreduce.reduce.memory.mb",
+      "mapreduce.map.java.opts",
+      "mapreduce.reduce.java.opts"
+    ]
   }
-}
+}
\ No newline at end of file


[14/52] [abbrv] incubator-eagle git commit: [minor] Change resourceFetch to resourcefetch

Posted by yo...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/b5fd782c/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/connection/URLConnectionUtils.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/connection/URLConnectionUtils.java b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/connection/URLConnectionUtils.java
new file mode 100644
index 0000000..2cd60ba
--- /dev/null
+++ b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/connection/URLConnectionUtils.java
@@ -0,0 +1,101 @@
+/*
+ * 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.eagle.jpm.util.resourcefetch.connection;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import java.io.IOException;
+import java.net.MalformedURLException;
+import java.net.URL;
+import java.net.URLConnection;
+import java.security.KeyManagementException;
+import java.security.NoSuchAlgorithmException;
+import java.security.cert.CertificateException;
+import javax.net.ssl.*;
+
+public final class URLConnectionUtils {
+    //TODO: change some public method to private
+    private static final Logger LOG = LoggerFactory.getLogger(URLConnectionUtils.class);
+
+    public static URLConnection getConnection(String url) throws Exception {
+        if (url.startsWith("https://")) {
+            return getHTTPSConnection(url);
+        } else if (url.startsWith("http://")) {
+            return getHTTPConnection(url);
+        }
+        throw new Exception("Invalid input argument url: " + url);
+    }
+
+    public static URLConnection getHTTPConnection(String urlString) throws Exception {
+        final URL url = new URL(urlString);
+        return url.openConnection();
+    }
+
+    public static URL getUrl(String urlString) throws Exception {
+        if (urlString.toLowerCase().contains("https")) {
+            return getHTTPSUrl(urlString);
+        } else if (urlString.toLowerCase().contains("http")) {
+            return getURL(urlString);
+        }
+        throw new Exception("Invalid input argument url: " + urlString);
+    }
+
+    public static URL getURL(String urlString) throws MalformedURLException {
+        return new URL(urlString);
+    }
+
+    public static URL getHTTPSUrl(String urlString) throws MalformedURLException, NoSuchAlgorithmException, KeyManagementException {
+        // Create a trust manager that does not validate certificate chains
+        final TrustManager[] trustAllCerts = new TrustManager[] {new TrustAllX509TrustManager()};
+        // Install the all-trusting trust manager   
+        final SSLContext sc = SSLContext.getInstance("SSL");
+        sc.init(null, trustAllCerts, new java.security.SecureRandom());
+        HttpsURLConnection.setDefaultSSLSocketFactory(sc.getSocketFactory());
+        // Create all-trusting host name verifier   
+        final HostnameVerifier allHostsValid = new HostnameVerifier() {
+            public boolean verify(String hostname, SSLSession session) {
+                return true;
+            }
+        };
+        HttpsURLConnection.setDefaultHostnameVerifier(allHostsValid);
+        return new URL(urlString);
+    }
+
+    public static URLConnection getHTTPSConnection(String urlString) throws IOException, KeyManagementException, NoSuchAlgorithmException {
+        final URL url = getHTTPSUrl(urlString);
+        return url.openConnection();
+    }
+
+    public static class TrustAllX509TrustManager implements X509TrustManager {
+        @Override
+        public void checkClientTrusted(
+            java.security.cert.X509Certificate[] chain, String authType)
+            throws CertificateException {
+        }
+
+        @Override
+        public void checkServerTrusted(
+            java.security.cert.X509Certificate[] chain, String authType)
+            throws CertificateException {
+        }
+
+        @Override
+        public java.security.cert.X509Certificate[] getAcceptedIssuers() {
+            return null;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/b5fd782c/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/ha/AbstractURLSelector.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/ha/AbstractURLSelector.java b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/ha/AbstractURLSelector.java
new file mode 100644
index 0000000..d25d05b
--- /dev/null
+++ b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/ha/AbstractURLSelector.java
@@ -0,0 +1,110 @@
+/*
+ * 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.eagle.jpm.util.resourcefetch.ha;
+
+import org.apache.eagle.jpm.util.Constants;
+import org.apache.eagle.jpm.util.resourcefetch.connection.InputStreamUtils;
+import org.apache.hadoop.util.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.Arrays;
+
+public abstract class AbstractURLSelector implements HAURLSelector {
+    private final String[] urls;
+    private volatile String selectedUrl;
+
+    private volatile boolean reselectInProgress;
+    private final Constants.CompressionType compressionType;
+
+    private static final long MAX_RETRY_TIME = 3;
+    private static final Logger LOG = LoggerFactory.getLogger(HAURLSelectorImpl.class);
+
+    public AbstractURLSelector(String[] urls, Constants.CompressionType compressionType) {
+        this.urls = urls;
+        this.compressionType = compressionType;
+    }
+
+    public boolean checkUrl(String urlString) {
+        InputStream is = null;
+        try {
+            is = InputStreamUtils.getInputStream(urlString, null, compressionType);
+        } catch (Exception ex) {
+            LOG.info("get input stream from url: " + urlString + " failed. ");
+            return false;
+        } finally {
+            if (is != null) {
+                try {
+                    is.close();
+                } catch (IOException e) {
+                    LOG.warn("{}", e);
+                }
+            }
+        }
+        return true;
+    }
+
+    @Override
+    public String getSelectedUrl() {
+        if (selectedUrl == null) {
+            selectedUrl = urls[0];
+        }
+        return selectedUrl;
+    }
+
+    @Override
+    public void reSelectUrl() throws IOException {
+        if (reselectInProgress) {
+            return;
+        }
+        synchronized (this) {
+            if (reselectInProgress) {
+                return;
+            }
+            reselectInProgress = true;
+            try {
+                LOG.info("Going to reselect url");
+                for (int i = 0; i < urls.length; i++) {
+                    String urlToCheck = urls[i];
+                    LOG.info("Going to try url :" + urlToCheck);
+                    for (int time = 0; time < MAX_RETRY_TIME; time++) {
+                        if (checkUrl(buildTestURL(urlToCheck))) {
+                            selectedUrl = urls[i];
+                            LOG.info("Successfully switch to new url : " + selectedUrl);
+                            return;
+                        }
+                        LOG.info("try url " + urlToCheck + "fail for " + (time + 1) + " times, sleep 5 seconds before try again. ");
+                        try {
+                            Thread.sleep(5 * 1000);
+                        } catch (InterruptedException ex) {
+                            LOG.warn("{}", ex);
+                        }
+                    }
+                }
+                throw new IOException("No alive url found: " + StringUtils.join(";", Arrays.asList(this.urls)));
+            } finally {
+                reselectInProgress = false;
+            }
+        }
+    }
+
+    protected abstract String buildTestURL(String urlToCheck);
+}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/b5fd782c/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/ha/HAURLSelector.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/ha/HAURLSelector.java b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/ha/HAURLSelector.java
new file mode 100644
index 0000000..fa9b52b
--- /dev/null
+++ b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/ha/HAURLSelector.java
@@ -0,0 +1,28 @@
+/*
+ * 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.eagle.jpm.util.resourcefetch.ha;
+
+import java.io.IOException;
+
+public interface HAURLSelector {
+
+    boolean checkUrl(String url);
+
+    void reSelectUrl() throws IOException;
+
+    String getSelectedUrl();
+}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/b5fd782c/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/ha/HAURLSelectorImpl.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/ha/HAURLSelectorImpl.java b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/ha/HAURLSelectorImpl.java
new file mode 100644
index 0000000..a083ef2
--- /dev/null
+++ b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/ha/HAURLSelectorImpl.java
@@ -0,0 +1,109 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.eagle.jpm.util.resourcefetch.ha;
+
+import org.apache.eagle.jpm.util.Constants;
+import org.apache.eagle.jpm.util.resourcefetch.connection.InputStreamUtils;
+import org.apache.eagle.jpm.util.resourcefetch.url.ServiceURLBuilder;
+import org.apache.hadoop.util.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.Arrays;
+
+public class HAURLSelectorImpl implements HAURLSelector {
+
+    private final String[] urls;
+    private volatile String selectedUrl;
+    private final ServiceURLBuilder builder;
+
+    private volatile boolean reselectInProgress;
+    private final Constants.CompressionType compressionType;
+    private static final long MAX_RETRY_TIME = 2;
+    private static final Logger LOG = LoggerFactory.getLogger(HAURLSelectorImpl.class);
+
+    public HAURLSelectorImpl(String[] urls, ServiceURLBuilder builder, Constants.CompressionType compressionType) {
+        this.urls = urls;
+        this.compressionType = compressionType;
+        this.builder = builder;
+    }
+
+    public boolean checkUrl(String urlString) {
+        InputStream is = null;
+        try {
+            is = InputStreamUtils.getInputStream(urlString, null, compressionType);
+        } catch (Exception ex) {
+            LOG.info("get inputstream from url: " + urlString + " failed. ");
+            return false;
+        } finally {
+            if (is != null) {
+                try {
+                    is.close();
+                } catch (IOException e) {
+                    LOG.warn("{}", e);
+                }
+            }
+        }
+        return true;
+    }
+
+    @Override
+    public String getSelectedUrl() {
+        if (selectedUrl == null) {
+            selectedUrl = urls[0];
+        }
+        return selectedUrl;
+    }
+
+    @Override
+    public void reSelectUrl() throws IOException {
+        if (reselectInProgress) {
+            return;
+        }
+        synchronized (this) {
+            if (reselectInProgress) {
+                return;
+            }
+            reselectInProgress = true;
+            try {
+                LOG.info("Going to reselect url");
+                for (int i = 0; i < urls.length; i++) {
+                    String urlToCheck = urls[i];
+                    LOG.info("Going to try url :" + urlToCheck);
+                    for (int time = 0; time < MAX_RETRY_TIME; time++) {
+                        if (checkUrl(builder.build(urlToCheck, Constants.JobState.RUNNING.name()))) {
+                            selectedUrl = urls[i];
+                            LOG.info("Successfully switch to new url : " + selectedUrl);
+                            return;
+                        }
+                        LOG.info("try url " + urlToCheck + "fail for " + (time + 1) + " times, sleep 5 seconds before try again. ");
+                        try {
+                            Thread.sleep(1000);
+                        } catch (InterruptedException ex) {
+                            LOG.warn("{}", ex);
+                        }
+                    }
+                }
+                throw new IOException("No alive url found: " + StringUtils.join(";", Arrays.asList(this.urls)));
+            } finally {
+                reselectInProgress = false;
+            }
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/b5fd782c/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/model/AppInfo.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/model/AppInfo.java b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/model/AppInfo.java
new file mode 100644
index 0000000..f769217
--- /dev/null
+++ b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/model/AppInfo.java
@@ -0,0 +1,210 @@
+/*
+ * 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.eagle.jpm.util.resourcefetch.model;
+
+import org.codehaus.jackson.annotate.JsonIgnoreProperties;
+import org.codehaus.jackson.map.annotate.JsonSerialize;
+
+import java.io.Serializable;
+
+@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
+@JsonIgnoreProperties(ignoreUnknown = true)
+public class AppInfo implements Serializable {
+    private static final long serialVersionUID = 1L;
+
+    private String id;
+    private String user;
+    private String name;
+    private String queue;
+    private String state;
+    private String finalStatus;
+    private double progress;
+    private String trackingUI;
+    private String trackingUrl;
+    private String diagnostics;
+    private String clusterId;
+    private String applicationType;
+    private long startedTime;
+    private long finishedTime;
+    private long elapsedTime;
+    private String amContainerLogs;
+    private String amHostHttpAddress;
+    private long allocatedMB;
+    private int allocatedVCores;
+    private int runningContainers;
+
+    public String getId() {
+        return id;
+    }
+
+    public void setId(String id) {
+        this.id = id;
+    }
+
+    public String getUser() {
+        return user;
+    }
+
+    public void setUser(String user) {
+        this.user = user;
+    }
+
+    public String getName() {
+        return name;
+    }
+
+    public void setName(String name) {
+        this.name = name;
+    }
+
+    public String getQueue() {
+        return queue;
+    }
+
+    public void setQueue(String queue) {
+        this.queue = queue;
+    }
+
+    public String getState() {
+        return state;
+    }
+
+    public void setState(String state) {
+        this.state = state;
+    }
+
+    public String getFinalStatus() {
+        return finalStatus;
+    }
+
+    public void setFinalStatus(String finalStatus) {
+        this.finalStatus = finalStatus;
+    }
+
+    public double getProgress() {
+        return progress;
+    }
+
+    public void setProgress(double progress) {
+        this.progress = progress;
+    }
+
+    public String getTrackingUI() {
+        return trackingUI;
+    }
+
+    public void setTrackingUI(String trackingUI) {
+        this.trackingUI = trackingUI;
+    }
+
+    public String getTrackingUrl() {
+        return trackingUrl;
+    }
+
+    public void setTrackingUrl(String trackingUrl) {
+        this.trackingUrl = trackingUrl;
+    }
+
+    public String getDiagnostics() {
+        return diagnostics;
+    }
+
+    public void setDiagnostics(String diagnostics) {
+        this.diagnostics = diagnostics;
+    }
+
+    public String getClusterId() {
+        return clusterId;
+    }
+
+    public void setClusterId(String clusterId) {
+        this.clusterId = clusterId;
+    }
+
+    public String getApplicationType() {
+        return applicationType;
+    }
+
+    public void setApplicationType(String applicationType) {
+        this.applicationType = applicationType;
+    }
+
+    public long getStartedTime() {
+        return startedTime;
+    }
+
+    public void setStartedTime(long startedTime) {
+        this.startedTime = startedTime;
+    }
+
+    public long getFinishedTime() {
+        return finishedTime;
+    }
+
+    public void setFinishedTime(long finishedTime) {
+        this.finishedTime = finishedTime;
+    }
+
+    public long getElapsedTime() {
+        return elapsedTime;
+    }
+
+    public void setElapsedTime(long elapsedTime) {
+        this.elapsedTime = elapsedTime;
+    }
+
+    public String getAmContainerLogs() {
+        return amContainerLogs;
+    }
+
+    public void setAmContainerLogs(String amContainerLogs) {
+        this.amContainerLogs = amContainerLogs;
+    }
+
+    public String getAmHostHttpAddress() {
+        return amHostHttpAddress;
+    }
+
+    public void setAmHostHttpAddress(String amHostHttpAddress) {
+        this.amHostHttpAddress = amHostHttpAddress;
+    }
+
+    public long getAllocatedMB() {
+        return allocatedMB;
+    }
+
+    public void setAllocatedMB(long allocatedMB) {
+        this.allocatedMB = allocatedMB;
+    }
+
+    public int getAllocatedVCores() {
+        return allocatedVCores;
+    }
+
+    public void setAllocatedVCores(int allocatedVCores) {
+        this.allocatedVCores = allocatedVCores;
+    }
+
+    public int getRunningContainers() {
+        return runningContainers;
+    }
+
+    public void setRunningContainers(int runningContainers) {
+        this.runningContainers = runningContainers;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/b5fd782c/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/model/Applications.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/model/Applications.java b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/model/Applications.java
new file mode 100644
index 0000000..22ee2b9
--- /dev/null
+++ b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/model/Applications.java
@@ -0,0 +1,38 @@
+/*
+ * 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.eagle.jpm.util.resourcefetch.model;
+
+import org.codehaus.jackson.annotate.JsonIgnoreProperties;
+import org.codehaus.jackson.map.annotate.JsonSerialize;
+
+import java.util.List;
+
+@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
+@JsonIgnoreProperties(ignoreUnknown = true)
+public class Applications {
+
+    private List<AppInfo> app;
+
+    public List<AppInfo> getApp() {
+        return app;
+    }
+
+    public void setApp(List<AppInfo> app) {
+        this.app = app;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/b5fd782c/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/model/AppsWrapper.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/model/AppsWrapper.java b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/model/AppsWrapper.java
new file mode 100644
index 0000000..63e37eb
--- /dev/null
+++ b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/model/AppsWrapper.java
@@ -0,0 +1,37 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.eagle.jpm.util.resourcefetch.model;
+
+import org.codehaus.jackson.annotate.JsonIgnoreProperties;
+import org.codehaus.jackson.map.annotate.JsonSerialize;
+
+@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
+@JsonIgnoreProperties(ignoreUnknown = true)
+public class AppsWrapper {
+    private static final long serialVersionUID = 1L;
+
+    private Applications apps;
+
+    public Applications getApps() {
+        return apps;
+    }
+
+    public void setApps(Applications apps) {
+        this.apps = apps;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/b5fd782c/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/model/ClusterInfo.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/model/ClusterInfo.java b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/model/ClusterInfo.java
new file mode 100644
index 0000000..6e16b7f
--- /dev/null
+++ b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/model/ClusterInfo.java
@@ -0,0 +1,119 @@
+/*
+ * 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.eagle.jpm.util.resourcefetch.model;
+
+import org.codehaus.jackson.annotate.JsonIgnoreProperties;
+import org.codehaus.jackson.map.annotate.JsonSerialize;
+
+import java.io.Serializable;
+
+@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
+@JsonIgnoreProperties(ignoreUnknown = true)
+public class ClusterInfo implements Serializable {
+    private static final long serialVersionUID = 1L;
+
+    private long id;
+    private long startedOn;
+    private String state;
+    private String haState;
+    private String resourceManagerVersion;
+    private String resourceManagerBuildVersion;
+    private String resourceManagerVersionBuiltOn;
+    private String hadoopVersion;
+    private String hadoopBuildVersion;
+    private String hadoopVersionBuiltOn;
+
+    public long getId() {
+        return id;
+    }
+
+    public void setId(long id) {
+        this.id = id;
+    }
+
+    public long getStartedOn() {
+        return startedOn;
+    }
+
+    public void setStartedOn(long startedOn) {
+        this.startedOn = startedOn;
+    }
+
+    public String getState() {
+        return state;
+    }
+
+    public void setState(String state) {
+        this.state = state;
+    }
+
+    public String getHaState() {
+        return haState;
+    }
+
+    public void setHaState(String haState) {
+        this.haState = haState;
+    }
+
+    public String getResourceManagerVersion() {
+        return resourceManagerVersion;
+    }
+
+    public void setResourceManagerVersion(String resourceManagerVersion) {
+        this.resourceManagerVersion = resourceManagerVersion;
+    }
+
+    public String getResourceManagerBuildVersion() {
+        return resourceManagerBuildVersion;
+    }
+
+    public void setResourceManagerBuildVersion(String resourceManagerBuildVersion) {
+        this.resourceManagerBuildVersion = resourceManagerBuildVersion;
+    }
+
+    public String getResourceManagerVersionBuiltOn() {
+        return resourceManagerVersionBuiltOn;
+    }
+
+    public void setResourceManagerVersionBuiltOn(String resourceManagerVersionBuiltOn) {
+        this.resourceManagerVersionBuiltOn = resourceManagerVersionBuiltOn;
+    }
+
+    public String getHadoopVersion() {
+        return hadoopVersion;
+    }
+
+    public void setHadoopVersion(String hadoopVersion) {
+        this.hadoopVersion = hadoopVersion;
+    }
+
+    public String getHadoopBuildVersion() {
+        return hadoopBuildVersion;
+    }
+
+    public void setHadoopBuildVersion(String hadoopBuildVersion) {
+        this.hadoopBuildVersion = hadoopBuildVersion;
+    }
+
+    public String getHadoopVersionBuiltOn() {
+        return hadoopVersionBuiltOn;
+    }
+
+    public void setHadoopVersionBuiltOn(String hadoopVersionBuiltOn) {
+        this.hadoopVersionBuiltOn = hadoopVersionBuiltOn;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/b5fd782c/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/model/ClusterInfoWrapper.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/model/ClusterInfoWrapper.java b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/model/ClusterInfoWrapper.java
new file mode 100644
index 0000000..4359e66
--- /dev/null
+++ b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/model/ClusterInfoWrapper.java
@@ -0,0 +1,35 @@
+/*
+ * 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.eagle.jpm.util.resourcefetch.model;
+
+import org.codehaus.jackson.annotate.JsonIgnoreProperties;
+import org.codehaus.jackson.map.annotate.JsonSerialize;
+
+@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
+@JsonIgnoreProperties(ignoreUnknown = true)
+public class ClusterInfoWrapper {
+    private ClusterInfo clusterInfo;
+
+    public ClusterInfo getClusterInfo() {
+        return clusterInfo;
+    }
+
+    public void setClusterInfo(ClusterInfo clusterInfo) {
+        this.clusterInfo = clusterInfo;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/b5fd782c/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/model/JobCounterGroup.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/model/JobCounterGroup.java b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/model/JobCounterGroup.java
new file mode 100644
index 0000000..189767b
--- /dev/null
+++ b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/model/JobCounterGroup.java
@@ -0,0 +1,46 @@
+/*
+ * 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.eagle.jpm.util.resourcefetch.model;
+
+import org.codehaus.jackson.annotate.JsonIgnoreProperties;
+import org.codehaus.jackson.map.annotate.JsonSerialize;
+
+import java.util.List;
+
+@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
+@JsonIgnoreProperties(ignoreUnknown = true)
+public class JobCounterGroup {
+    private String counterGroupName;
+    private List<JobCounterItem> counter;
+
+    public String getCounterGroupName() {
+        return counterGroupName;
+    }
+
+    public void setCounterGroupName(String counterGroupName) {
+        this.counterGroupName = counterGroupName;
+    }
+
+    public List<JobCounterItem> getCounter() {
+        return counter;
+    }
+
+    public void setCounter(List<JobCounterItem> counter) {
+        this.counter = counter;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/b5fd782c/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/model/JobCounterItem.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/model/JobCounterItem.java b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/model/JobCounterItem.java
new file mode 100644
index 0000000..001c5ff
--- /dev/null
+++ b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/model/JobCounterItem.java
@@ -0,0 +1,62 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.eagle.jpm.util.resourcefetch.model;
+
+import org.codehaus.jackson.annotate.JsonIgnoreProperties;
+import org.codehaus.jackson.map.annotate.JsonSerialize;
+
+@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
+@JsonIgnoreProperties(ignoreUnknown = true)
+public class JobCounterItem {
+    private String name;
+    private long totalCounterValue;
+    private long mapCounterValue;
+    private long reduceCounterValue;
+
+    public String getName() {
+        return name;
+    }
+
+    public void setName(String name) {
+        this.name = name;
+    }
+
+    public long getTotalCounterValue() {
+        return totalCounterValue;
+    }
+
+    public void setTotalCounterValue(long totalCounterValue) {
+        this.totalCounterValue = totalCounterValue;
+    }
+
+    public long getMapCounterValue() {
+        return mapCounterValue;
+    }
+
+    public void setMapCounterValue(long mapCounterValue) {
+        this.mapCounterValue = mapCounterValue;
+    }
+
+    public long getReduceCounterValue() {
+        return reduceCounterValue;
+    }
+
+    public void setReduceCounterValue(long reduceCounterValue) {
+        this.reduceCounterValue = reduceCounterValue;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/b5fd782c/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/model/JobCounters.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/model/JobCounters.java b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/model/JobCounters.java
new file mode 100644
index 0000000..cc0709f
--- /dev/null
+++ b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/model/JobCounters.java
@@ -0,0 +1,46 @@
+/*
+ * 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.eagle.jpm.util.resourcefetch.model;
+
+import org.codehaus.jackson.annotate.JsonIgnoreProperties;
+import org.codehaus.jackson.map.annotate.JsonSerialize;
+
+import java.util.List;
+
+@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
+@JsonIgnoreProperties(ignoreUnknown = true)
+public class JobCounters {
+    private String id;
+    private List<JobCounterGroup> counterGroup;
+
+    public String getId() {
+        return id;
+    }
+
+    public void setId(String id) {
+        this.id = id;
+    }
+
+    public List<JobCounterGroup> getCounterGroup() {
+        return counterGroup;
+    }
+
+    public void setCounterGroup(List<JobCounterGroup> counterGroup) {
+        this.counterGroup = counterGroup;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/b5fd782c/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/model/JobCountersWrapper.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/model/JobCountersWrapper.java b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/model/JobCountersWrapper.java
new file mode 100644
index 0000000..f6bb29c
--- /dev/null
+++ b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/model/JobCountersWrapper.java
@@ -0,0 +1,35 @@
+/*
+ * 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.eagle.jpm.util.resourcefetch.model;
+
+import org.codehaus.jackson.annotate.JsonIgnoreProperties;
+import org.codehaus.jackson.map.annotate.JsonSerialize;
+
+@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
+@JsonIgnoreProperties(ignoreUnknown = true)
+public class JobCountersWrapper {
+    public JobCounters getJobCounters() {
+        return jobCounters;
+    }
+
+    public void setJobCounters(JobCounters jobCounters) {
+        this.jobCounters = jobCounters;
+    }
+
+    private JobCounters jobCounters;
+}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/b5fd782c/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/model/MRJob.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/model/MRJob.java b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/model/MRJob.java
new file mode 100644
index 0000000..f12a5d7
--- /dev/null
+++ b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/model/MRJob.java
@@ -0,0 +1,289 @@
+/*
+ *
+ *  * 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.eagle.jpm.util.resourcefetch.model;
+
+import org.codehaus.jackson.annotate.JsonIgnoreProperties;
+import org.codehaus.jackson.map.annotate.JsonSerialize;
+
+@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
+@JsonIgnoreProperties(ignoreUnknown = true)
+public class MRJob {
+    private long startTime;
+    private long finishTime;
+    private long elapsedTime;
+    private String id;
+    private String name;
+    private String user;
+    private String state;
+    private int mapsTotal;
+    private int mapsCompleted;
+    private int reducesTotal;
+    private int reducesCompleted;
+    private double mapProgress;
+    private double reduceProgress;
+    private int mapsPending;
+    private int mapsRunning;
+    private int reducesPending;
+    private int reducesRunning;
+    private boolean uberized;
+    private String diagnostics;
+    private int newReduceAttempts;
+    private int runningReduceAttempts;
+    private int failedReduceAttempts;
+    private int killedReduceAttempts;
+    private int successfulReduceAttempts;
+    private int newMapAttempts;
+    private int runningMapAttempts;
+    private int failedMapAttempts;
+    private int killedMapAttempts;
+    private int successfulMapAttempts;
+
+    public long getStartTime() {
+        return startTime;
+    }
+
+    public void setStartTime(long startTime) {
+        this.startTime = startTime;
+    }
+
+    public long getFinishTime() {
+        return finishTime;
+    }
+
+    public void setFinishTime(long finishTime) {
+        this.finishTime = finishTime;
+    }
+
+    public long getElapsedTime() {
+        return elapsedTime;
+    }
+
+    public void setElapsedTime(long elapsedTime) {
+        this.elapsedTime = elapsedTime;
+    }
+
+    public String getId() {
+        return id;
+    }
+
+    public void setId(String id) {
+        this.id = id;
+    }
+
+    public String getName() {
+        return name;
+    }
+
+    public void setName(String name) {
+        this.name = name;
+    }
+
+    public String getUser() {
+        return user;
+    }
+
+    public void setUser(String user) {
+        this.user = user;
+    }
+
+    public String getState() {
+        return state;
+    }
+
+    public void setState(String state) {
+        this.state = state;
+    }
+
+    public int getMapsTotal() {
+        return mapsTotal;
+    }
+
+    public void setMapsTotal(int mapsTotal) {
+        this.mapsTotal = mapsTotal;
+    }
+
+    public int getMapsCompleted() {
+        return mapsCompleted;
+    }
+
+    public void setMapsCompleted(int mapsCompleted) {
+        this.mapsCompleted = mapsCompleted;
+    }
+
+    public int getReducesTotal() {
+        return reducesTotal;
+    }
+
+    public void setReducesTotal(int reducesTotal) {
+        this.reducesTotal = reducesTotal;
+    }
+
+    public int getReducesCompleted() {
+        return reducesCompleted;
+    }
+
+    public void setReducesCompleted(int reducesCompleted) {
+        this.reducesCompleted = reducesCompleted;
+    }
+
+    public double getMapProgress() {
+        return mapProgress;
+    }
+
+    public void setMapProgress(double mapProgress) {
+        this.mapProgress = mapProgress;
+    }
+
+    public double getReduceProgress() {
+        return reduceProgress;
+    }
+
+    public void setReduceProgress(double reduceProgress) {
+        this.reduceProgress = reduceProgress;
+    }
+
+    public int getMapsPending() {
+        return mapsPending;
+    }
+
+    public void setMapsPending(int mapsPending) {
+        this.mapsPending = mapsPending;
+    }
+
+    public int getMapsRunning() {
+        return mapsRunning;
+    }
+
+    public void setMapsRunning(int mapsRunning) {
+        this.mapsRunning = mapsRunning;
+    }
+
+    public int getReducesPending() {
+        return reducesPending;
+    }
+
+    public void setReducesPending(int reducesPending) {
+        this.reducesPending = reducesPending;
+    }
+
+    public int getReducesRunning() {
+        return reducesRunning;
+    }
+
+    public void setReducesRunning(int reducesRunning) {
+        this.reducesRunning = reducesRunning;
+    }
+
+    public boolean isUberized() {
+        return uberized;
+    }
+
+    public void setUberized(boolean uberized) {
+        this.uberized = uberized;
+    }
+
+    public String getDiagnostics() {
+        return diagnostics;
+    }
+
+    public void setDiagnostics(String diagnostics) {
+        this.diagnostics = diagnostics;
+    }
+
+    public int getNewReduceAttempts() {
+        return newReduceAttempts;
+    }
+
+    public void setNewReduceAttempts(int newReduceAttempts) {
+        this.newReduceAttempts = newReduceAttempts;
+    }
+
+    public int getRunningReduceAttempts() {
+        return runningReduceAttempts;
+    }
+
+    public void setRunningReduceAttempts(int runningReduceAttempts) {
+        this.runningReduceAttempts = runningReduceAttempts;
+    }
+
+    public int getFailedReduceAttempts() {
+        return failedReduceAttempts;
+    }
+
+    public void setFailedReduceAttempts(int failedReduceAttempts) {
+        this.failedReduceAttempts = failedReduceAttempts;
+    }
+
+    public int getKilledReduceAttempts() {
+        return killedReduceAttempts;
+    }
+
+    public void setKilledReduceAttempts(int killedReduceAttempts) {
+        this.killedReduceAttempts = killedReduceAttempts;
+    }
+
+    public int getSuccessfulReduceAttempts() {
+        return successfulReduceAttempts;
+    }
+
+    public void setSuccessfulReduceAttempts(int successfulReduceAttempts) {
+        this.successfulReduceAttempts = successfulReduceAttempts;
+    }
+
+    public int getNewMapAttempts() {
+        return newMapAttempts;
+    }
+
+    public void setNewMapAttempts(int newMapAttempts) {
+        this.newMapAttempts = newMapAttempts;
+    }
+
+    public int getRunningMapAttempts() {
+        return runningMapAttempts;
+    }
+
+    public void setRunningMapAttempts(int runningMapAttempts) {
+        this.runningMapAttempts = runningMapAttempts;
+    }
+
+    public int getFailedMapAttempts() {
+        return failedMapAttempts;
+    }
+
+    public void setFailedMapAttempts(int failedMapAttempts) {
+        this.failedMapAttempts = failedMapAttempts;
+    }
+
+    public int getKilledMapAttempts() {
+        return killedMapAttempts;
+    }
+
+    public void setKilledMapAttempts(int killedMapAttempts) {
+        this.killedMapAttempts = killedMapAttempts;
+    }
+
+    public int getSuccessfulMapAttempts() {
+        return successfulMapAttempts;
+    }
+
+    public void setSuccessfulMapAttempts(int successfulMapAttempts) {
+        this.successfulMapAttempts = successfulMapAttempts;
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/b5fd782c/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/model/MRJobsWrapper.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/model/MRJobsWrapper.java b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/model/MRJobsWrapper.java
new file mode 100644
index 0000000..acd316f
--- /dev/null
+++ b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/model/MRJobsWrapper.java
@@ -0,0 +1,37 @@
+/*
+ *
+ *  * Licensed to the Apache Software Foundation (ASF) under one or more
+ *  * contributor license agreements.  See the NOTICE file distributed with
+ *  * this work for additional information regarding copyright ownership.
+ *  * The ASF licenses this file to You under the Apache License, Version 2.0
+ *  * (the "License"); you may not use this file except in compliance with
+ *  * the License.  You may obtain a copy of the License at
+ *  *
+ *  *    http://www.apache.org/licenses/LICENSE-2.0
+ *  *
+ *  * Unless required by applicable law or agreed to in writing, software
+ *  * distributed under the License is distributed on an "AS IS" BASIS,
+ *  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  * See the License for the specific language governing permissions and
+ *  * limitations under the License.
+ *
+ */
+
+package org.apache.eagle.jpm.util.resourcefetch.model;
+
+import org.codehaus.jackson.annotate.JsonIgnoreProperties;
+import org.codehaus.jackson.map.annotate.JsonSerialize;
+
+@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
+@JsonIgnoreProperties(ignoreUnknown = true)
+public class MRJobsWrapper {
+    public MrJobs getJobs() {
+        return jobs;
+    }
+
+    public void setJobs(MrJobs jobs) {
+        this.jobs = jobs;
+    }
+
+    private MrJobs jobs;
+}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/b5fd782c/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/model/MRTask.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/model/MRTask.java b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/model/MRTask.java
new file mode 100644
index 0000000..4a549de
--- /dev/null
+++ b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/model/MRTask.java
@@ -0,0 +1,109 @@
+/*
+ *
+ *  * Licensed to the Apache Software Foundation (ASF) under one or more
+ *  * contributor license agreements.  See the NOTICE file distributed with
+ *  * this work for additional information regarding copyright ownership.
+ *  * The ASF licenses this file to You under the Apache License, Version 2.0
+ *  * (the "License"); you may not use this file except in compliance with
+ *  * the License.  You may obtain a copy of the License at
+ *  *
+ *  *    http://www.apache.org/licenses/LICENSE-2.0
+ *  *
+ *  * Unless required by applicable law or agreed to in writing, software
+ *  * distributed under the License is distributed on an "AS IS" BASIS,
+ *  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  * See the License for the specific language governing permissions and
+ *  * limitations under the License.
+ *
+ */
+
+package org.apache.eagle.jpm.util.resourcefetch.model;
+
+import org.codehaus.jackson.annotate.JsonIgnoreProperties;
+import org.codehaus.jackson.map.annotate.JsonSerialize;
+
+@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
+@JsonIgnoreProperties(ignoreUnknown = true)
+public class MRTask {
+    private long startTime;
+    private long finishTime;
+    private long elapsedTime;
+    private double progress;
+    private String id;
+    private String state;
+    private String type;
+    private String successfulAttempt;
+    private String status;
+
+    public long getStartTime() {
+        return startTime;
+    }
+
+    public void setStartTime(long startTime) {
+        this.startTime = startTime;
+    }
+
+    public long getFinishTime() {
+        return finishTime;
+    }
+
+    public void setFinishTime(long finishTime) {
+        this.finishTime = finishTime;
+    }
+
+    public long getElapsedTime() {
+        return elapsedTime;
+    }
+
+    public void setElapsedTime(long elapsedTime) {
+        this.elapsedTime = elapsedTime;
+    }
+
+    public double getProgress() {
+        return progress;
+    }
+
+    public void setProgress(double progress) {
+        this.progress = progress;
+    }
+
+    public String getId() {
+        return id;
+    }
+
+    public void setId(String id) {
+        this.id = id;
+    }
+
+    public String getState() {
+        return state;
+    }
+
+    public void setState(String state) {
+        this.state = state;
+    }
+
+    public String getType() {
+        return type;
+    }
+
+    public void setType(String type) {
+        this.type = type;
+    }
+
+    public String getSuccessfulAttempt() {
+        return successfulAttempt;
+    }
+
+    public void setSuccessfulAttempt(String successfulAttempt) {
+        this.successfulAttempt = successfulAttempt;
+    }
+
+    public String getStatus() {
+        return status;
+    }
+
+    public void setStatus(String status) {
+        this.status = status;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/b5fd782c/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/model/MRTaskAttempt.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/model/MRTaskAttempt.java b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/model/MRTaskAttempt.java
new file mode 100644
index 0000000..218d021
--- /dev/null
+++ b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/model/MRTaskAttempt.java
@@ -0,0 +1,136 @@
+/*
+ *
+ *  * 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.eagle.jpm.util.resourcefetch.model;
+
+import org.codehaus.jackson.annotate.JsonIgnoreProperties;
+import org.codehaus.jackson.map.annotate.JsonSerialize;
+
+@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
+@JsonIgnoreProperties(ignoreUnknown = true)
+public class MRTaskAttempt {
+    private long startTime;
+    private long finishTime;
+    private long elapsedTime;
+    private double progress;
+    private String id;
+    private String rack;
+    private String state;
+    private String status;
+    private String nodeHttpAddress;
+    private String diagnostics;
+    private String type;
+    private String assignedContainerId;
+
+    public long getStartTime() {
+        return startTime;
+    }
+
+    public void setStartTime(long startTime) {
+        this.startTime = startTime;
+    }
+
+    public long getFinishTime() {
+        return finishTime;
+    }
+
+    public void setFinishTime(long finishTime) {
+        this.finishTime = finishTime;
+    }
+
+    public long getElapsedTime() {
+        return elapsedTime;
+    }
+
+    public void setElapsedTime(long elapsedTime) {
+        this.elapsedTime = elapsedTime;
+    }
+
+    public double getProgress() {
+        return progress;
+    }
+
+    public void setProgress(double progress) {
+        this.progress = progress;
+    }
+
+    public String getId() {
+        return id;
+    }
+
+    public void setId(String id) {
+        this.id = id;
+    }
+
+    public String getRack() {
+        return rack;
+    }
+
+    public void setRack(String rack) {
+        this.rack = rack;
+    }
+
+    public String getState() {
+        return state;
+    }
+
+    public void setState(String state) {
+        this.state = state;
+    }
+
+    public String getStatus() {
+        return status;
+    }
+
+    public void setStatus(String status) {
+        this.status = status;
+    }
+
+    public String getNodeHttpAddress() {
+        return nodeHttpAddress;
+    }
+
+    public void setNodeHttpAddress(String nodeHttpAddress) {
+        this.nodeHttpAddress = nodeHttpAddress;
+    }
+
+    public String getDiagnostics() {
+        return diagnostics;
+    }
+
+    public void setDiagnostics(String diagnostics) {
+        this.diagnostics = diagnostics;
+    }
+
+    public String getType() {
+        return type;
+    }
+
+    public void setType(String type) {
+        this.type = type;
+    }
+
+    public String getAssignedContainerId() {
+        return assignedContainerId;
+    }
+
+    public void setAssignedContainerId(String assignedContainerId) {
+        this.assignedContainerId = assignedContainerId;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/b5fd782c/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/model/MRTaskAttemptWrapper.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/model/MRTaskAttemptWrapper.java b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/model/MRTaskAttemptWrapper.java
new file mode 100644
index 0000000..005b73a
--- /dev/null
+++ b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/model/MRTaskAttemptWrapper.java
@@ -0,0 +1,37 @@
+/*
+ *
+ *  * Licensed to the Apache Software Foundation (ASF) under one or more
+ *  * contributor license agreements.  See the NOTICE file distributed with
+ *  * this work for additional information regarding copyright ownership.
+ *  * The ASF licenses this file to You under the Apache License, Version 2.0
+ *  * (the "License"); you may not use this file except in compliance with
+ *  * the License.  You may obtain a copy of the License at
+ *  *
+ *  *    http://www.apache.org/licenses/LICENSE-2.0
+ *  *
+ *  * Unless required by applicable law or agreed to in writing, software
+ *  * distributed under the License is distributed on an "AS IS" BASIS,
+ *  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  * See the License for the specific language governing permissions and
+ *  * limitations under the License.
+ *
+ */
+
+package org.apache.eagle.jpm.util.resourcefetch.model;
+
+import org.codehaus.jackson.annotate.JsonIgnoreProperties;
+import org.codehaus.jackson.map.annotate.JsonSerialize;
+
+@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
+@JsonIgnoreProperties(ignoreUnknown = true)
+public class MRTaskAttemptWrapper {
+    public MRTaskAttempts getTaskAttempts() {
+        return taskAttempts;
+    }
+
+    public void setTaskAttempts(MRTaskAttempts taskAttempts) {
+        this.taskAttempts = taskAttempts;
+    }
+
+    private MRTaskAttempts taskAttempts;
+}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/b5fd782c/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/model/MRTaskAttempts.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/model/MRTaskAttempts.java b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/model/MRTaskAttempts.java
new file mode 100644
index 0000000..03af0d7
--- /dev/null
+++ b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/model/MRTaskAttempts.java
@@ -0,0 +1,39 @@
+/*
+ *
+ *  * Licensed to the Apache Software Foundation (ASF) under one or more
+ *  * contributor license agreements.  See the NOTICE file distributed with
+ *  * this work for additional information regarding copyright ownership.
+ *  * The ASF licenses this file to You under the Apache License, Version 2.0
+ *  * (the "License"); you may not use this file except in compliance with
+ *  * the License.  You may obtain a copy of the License at
+ *  *
+ *  *    http://www.apache.org/licenses/LICENSE-2.0
+ *  *
+ *  * Unless required by applicable law or agreed to in writing, software
+ *  * distributed under the License is distributed on an "AS IS" BASIS,
+ *  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  * See the License for the specific language governing permissions and
+ *  * limitations under the License.
+ *
+ */
+
+package org.apache.eagle.jpm.util.resourcefetch.model;
+
+import org.codehaus.jackson.annotate.JsonIgnoreProperties;
+import org.codehaus.jackson.map.annotate.JsonSerialize;
+
+import java.util.List;
+
+@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
+@JsonIgnoreProperties(ignoreUnknown = true)
+public class MRTaskAttempts {
+    public List<MRTaskAttempt> getTaskAttempt() {
+        return taskAttempt;
+    }
+
+    public void setTaskAttempt(List<MRTaskAttempt> taskAttempt) {
+        this.taskAttempt = taskAttempt;
+    }
+
+    private List<MRTaskAttempt> taskAttempt;
+}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/b5fd782c/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/model/MRTasks.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/model/MRTasks.java b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/model/MRTasks.java
new file mode 100644
index 0000000..b23b23d
--- /dev/null
+++ b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/model/MRTasks.java
@@ -0,0 +1,40 @@
+/*
+ *
+ *  * Licensed to the Apache Software Foundation (ASF) under one or more
+ *  * contributor license agreements.  See the NOTICE file distributed with
+ *  * this work for additional information regarding copyright ownership.
+ *  * The ASF licenses this file to You under the Apache License, Version 2.0
+ *  * (the "License"); you may not use this file except in compliance with
+ *  * the License.  You may obtain a copy of the License at
+ *  *
+ *  *    http://www.apache.org/licenses/LICENSE-2.0
+ *  *
+ *  * Unless required by applicable law or agreed to in writing, software
+ *  * distributed under the License is distributed on an "AS IS" BASIS,
+ *  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  * See the License for the specific language governing permissions and
+ *  * limitations under the License.
+ *
+ */
+
+package org.apache.eagle.jpm.util.resourcefetch.model;
+
+import org.codehaus.jackson.annotate.JsonIgnoreProperties;
+import org.codehaus.jackson.map.annotate.JsonSerialize;
+
+import java.util.List;
+
+@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
+@JsonIgnoreProperties(ignoreUnknown = true)
+public class MRTasks {
+    public List<MRTask> getTask() {
+        return task;
+    }
+
+    public void setTask(List<MRTask> task) {
+        this.task = task;
+    }
+
+    private List<MRTask> task;
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/b5fd782c/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/model/MRTasksWrapper.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/model/MRTasksWrapper.java b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/model/MRTasksWrapper.java
new file mode 100644
index 0000000..f92234a
--- /dev/null
+++ b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/model/MRTasksWrapper.java
@@ -0,0 +1,37 @@
+/*
+ *
+ *  * Licensed to the Apache Software Foundation (ASF) under one or more
+ *  * contributor license agreements.  See the NOTICE file distributed with
+ *  * this work for additional information regarding copyright ownership.
+ *  * The ASF licenses this file to You under the Apache License, Version 2.0
+ *  * (the "License"); you may not use this file except in compliance with
+ *  * the License.  You may obtain a copy of the License at
+ *  *
+ *  *    http://www.apache.org/licenses/LICENSE-2.0
+ *  *
+ *  * Unless required by applicable law or agreed to in writing, software
+ *  * distributed under the License is distributed on an "AS IS" BASIS,
+ *  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  * See the License for the specific language governing permissions and
+ *  * limitations under the License.
+ *
+ */
+
+package org.apache.eagle.jpm.util.resourcefetch.model;
+
+import org.codehaus.jackson.annotate.JsonIgnoreProperties;
+import org.codehaus.jackson.map.annotate.JsonSerialize;
+
+@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
+@JsonIgnoreProperties(ignoreUnknown = true)
+public class MRTasksWrapper {
+    public MRTasks getTasks() {
+        return tasks;
+    }
+
+    public void setTasks(MRTasks tasks) {
+        this.tasks = tasks;
+    }
+
+    private MRTasks tasks;
+}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/b5fd782c/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/model/MrJobs.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/model/MrJobs.java b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/model/MrJobs.java
new file mode 100644
index 0000000..1d4871c
--- /dev/null
+++ b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/model/MrJobs.java
@@ -0,0 +1,39 @@
+/*
+ *
+ *  * Licensed to the Apache Software Foundation (ASF) under one or more
+ *  * contributor license agreements.  See the NOTICE file distributed with
+ *  * this work for additional information regarding copyright ownership.
+ *  * The ASF licenses this file to You under the Apache License, Version 2.0
+ *  * (the "License"); you may not use this file except in compliance with
+ *  * the License.  You may obtain a copy of the License at
+ *  *
+ *  *    http://www.apache.org/licenses/LICENSE-2.0
+ *  *
+ *  * Unless required by applicable law or agreed to in writing, software
+ *  * distributed under the License is distributed on an "AS IS" BASIS,
+ *  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  * See the License for the specific language governing permissions and
+ *  * limitations under the License.
+ *
+ */
+
+package org.apache.eagle.jpm.util.resourcefetch.model;
+
+import org.codehaus.jackson.annotate.JsonIgnoreProperties;
+import org.codehaus.jackson.map.annotate.JsonSerialize;
+
+import java.util.List;
+
+@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
+@JsonIgnoreProperties(ignoreUnknown = true)
+public class MrJobs {
+    public List<MRJob> getJob() {
+        return job;
+    }
+
+    public void setJobs(List<MRJob> job) {
+        this.job = job;
+    }
+
+    private List<MRJob> job;
+}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/b5fd782c/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/model/SparkApplication.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/model/SparkApplication.java b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/model/SparkApplication.java
new file mode 100644
index 0000000..299934e
--- /dev/null
+++ b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/model/SparkApplication.java
@@ -0,0 +1,57 @@
+/*
+ *
+ *  * 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.eagle.jpm.util.resourcefetch.model;
+
+import org.codehaus.jackson.annotate.JsonIgnoreProperties;
+import org.codehaus.jackson.map.annotate.JsonSerialize;
+
+import java.util.List;
+
+@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
+@JsonIgnoreProperties(ignoreUnknown = true)
+public class SparkApplication {
+    String id;
+    String name;
+    List<SparkApplicationAttempt> attempts;
+
+    public String getId() {
+        return id;
+    }
+
+    public void setId(String id) {
+        this.id = id;
+    }
+
+    public String getName() {
+        return name;
+    }
+
+    public void setName(String name) {
+        this.name = name;
+    }
+
+    public List<SparkApplicationAttempt> getAttempts() {
+        return attempts;
+    }
+
+    public void setAttempts(List<SparkApplicationAttempt> attempts) {
+        this.attempts = attempts;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/b5fd782c/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/model/SparkApplicationAttempt.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/model/SparkApplicationAttempt.java b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/model/SparkApplicationAttempt.java
new file mode 100644
index 0000000..b437cc2
--- /dev/null
+++ b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/model/SparkApplicationAttempt.java
@@ -0,0 +1,73 @@
+/*
+ *
+ *  * 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.eagle.jpm.util.resourcefetch.model;
+
+import org.codehaus.jackson.annotate.JsonIgnoreProperties;
+import org.codehaus.jackson.map.annotate.JsonSerialize;
+
+@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
+@JsonIgnoreProperties(ignoreUnknown = true)
+public class SparkApplicationAttempt {
+    String attemptId;
+    String sparkUser;
+    String startTime;
+    String endTime;
+    boolean completed;
+
+    public String getAttemptId() {
+        return attemptId;
+    }
+
+    public void setAttemptId(String attemptId) {
+        this.attemptId = attemptId;
+    }
+
+    public String getSparkUser() {
+        return sparkUser;
+    }
+
+    public void setSparkUser(String sparkUser) {
+        this.sparkUser = sparkUser;
+    }
+
+    public String getStartTime() {
+        return startTime;
+    }
+
+    public void setStartTime(String startTime) {
+        this.startTime = startTime;
+    }
+
+    public String getEndTime() {
+        return endTime;
+    }
+
+    public void setEndTime(String endTime) {
+        this.endTime = endTime;
+    }
+
+    public boolean isCompleted() {
+        return completed;
+    }
+
+    public void setCompleted(boolean completed) {
+        this.completed = completed;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/b5fd782c/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/model/SparkApplicationWrapper.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/model/SparkApplicationWrapper.java b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/model/SparkApplicationWrapper.java
new file mode 100644
index 0000000..0b04f31
--- /dev/null
+++ b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/model/SparkApplicationWrapper.java
@@ -0,0 +1,38 @@
+/*
+ *
+ *  * 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.eagle.jpm.util.resourcefetch.model;
+
+import org.codehaus.jackson.annotate.JsonIgnoreProperties;
+import org.codehaus.jackson.map.annotate.JsonSerialize;
+
+@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
+@JsonIgnoreProperties(ignoreUnknown = true)
+public class SparkApplicationWrapper {
+
+    SparkApplication app;
+
+    public SparkApplication getApp() {
+        return app;
+    }
+
+    public void setApp(SparkApplication app) {
+        this.app = app;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/b5fd782c/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/model/SparkExecutor.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/model/SparkExecutor.java b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/model/SparkExecutor.java
new file mode 100644
index 0000000..c2a2115
--- /dev/null
+++ b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/model/SparkExecutor.java
@@ -0,0 +1,155 @@
+/*
+ *
+ *  * 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.eagle.jpm.util.resourcefetch.model;
+
+import org.codehaus.jackson.annotate.JsonIgnoreProperties;
+import org.codehaus.jackson.map.annotate.JsonSerialize;
+
+@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
+@JsonIgnoreProperties(ignoreUnknown = true)
+public class SparkExecutor {
+    private String id;
+    private String hostPort;
+    private int rddBlocks;
+    private long memoryUsed;
+    private long diskUsed;
+    private int activeTasks;
+    private int failedTasks;
+    private int completedTasks;
+    private int totalTasks;
+    private long totalDuration;
+    private long totalInputBytes;
+    private long totalShuffleRead;
+    private long totalShuffleWrite;
+    private long maxMemory;
+
+    public String getId() {
+        return id;
+    }
+
+    public void setId(String id) {
+        this.id = id;
+    }
+
+    public String getHostPort() {
+        return hostPort;
+    }
+
+    public void setHostPort(String hostPort) {
+        this.hostPort = hostPort;
+    }
+
+    public int getRddBlocks() {
+        return rddBlocks;
+    }
+
+    public void setRddBlocks(int rddBlocks) {
+        this.rddBlocks = rddBlocks;
+    }
+
+    public long getMemoryUsed() {
+        return memoryUsed;
+    }
+
+    public void setMemoryUsed(long memoryUsed) {
+        this.memoryUsed = memoryUsed;
+    }
+
+    public long getDiskUsed() {
+        return diskUsed;
+    }
+
+    public void setDiskUsed(long diskUsed) {
+        this.diskUsed = diskUsed;
+    }
+
+    public int getActiveTasks() {
+        return activeTasks;
+    }
+
+    public void setActiveTasks(int activeTasks) {
+        this.activeTasks = activeTasks;
+    }
+
+    public int getFailedTasks() {
+        return failedTasks;
+    }
+
+    public void setFailedTasks(int failedTasks) {
+        this.failedTasks = failedTasks;
+    }
+
+    public int getCompletedTasks() {
+        return completedTasks;
+    }
+
+    public void setCompletedTasks(int completedTasks) {
+        this.completedTasks = completedTasks;
+    }
+
+    public int getTotalTasks() {
+        return totalTasks;
+    }
+
+    public void setTotalTasks(int totalTasks) {
+        this.totalTasks = totalTasks;
+    }
+
+    public long getTotalDuration() {
+        return totalDuration;
+    }
+
+    public void setTotalDuration(long totalDuration) {
+        this.totalDuration = totalDuration;
+    }
+
+    public long getTotalInputBytes() {
+        return totalInputBytes;
+    }
+
+    public void setTotalInputBytes(long totalInputBytes) {
+        this.totalInputBytes = totalInputBytes;
+    }
+
+    public long getTotalShuffleRead() {
+        return totalShuffleRead;
+    }
+
+    public void setTotalShuffleRead(long totalShuffleRead) {
+        this.totalShuffleRead = totalShuffleRead;
+    }
+
+    public long getTotalShuffleWrite() {
+        return totalShuffleWrite;
+    }
+
+    public void setTotalShuffleWrite(long totalShuffleWrite) {
+        this.totalShuffleWrite = totalShuffleWrite;
+    }
+
+    public long getMaxMemory() {
+        return maxMemory;
+    }
+
+    public void setMaxMemory(long maxMemory) {
+        this.maxMemory = maxMemory;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/b5fd782c/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/model/SparkJob.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/model/SparkJob.java b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/model/SparkJob.java
new file mode 100644
index 0000000..280cf67
--- /dev/null
+++ b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/model/SparkJob.java
@@ -0,0 +1,165 @@
+/*
+ *
+ *  * 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.eagle.jpm.util.resourcefetch.model;
+
+import org.codehaus.jackson.annotate.JsonIgnoreProperties;
+import org.codehaus.jackson.map.annotate.JsonSerialize;
+
+import java.util.List;
+
+@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
+@JsonIgnoreProperties(ignoreUnknown = true)
+public class SparkJob {
+    private int jobId;
+    private String name;
+    private String submissionTime;
+    private String completionTime;
+    private List<Integer> stageIds;
+    private String status;
+    private int numTasks;
+    private int numActiveTasks;
+    private int numCompletedTasks;
+    private int numSkippedTasks;
+    private int numFailedTasks;
+    private int numActiveStages;
+    private int numCompletedStages;
+    private int numSkippedStages;
+    private int numFailedStages;
+
+    public int getJobId() {
+        return jobId;
+    }
+
+    public void setJobId(int jobId) {
+        this.jobId = jobId;
+    }
+
+    public String getName() {
+        return name;
+    }
+
+    public void setName(String name) {
+        this.name = name;
+    }
+
+    public String getSubmissionTime() {
+        return submissionTime;
+    }
+
+    public void setSubmissionTime(String submissionTime) {
+        this.submissionTime = submissionTime;
+    }
+
+    public String getCompletionTime() {
+        return completionTime;
+    }
+
+    public void setCompletionTime(String completionTime) {
+        this.completionTime = completionTime;
+    }
+
+    public List<Integer> getStageIds() {
+        return stageIds;
+    }
+
+    public void setStageIds(List<Integer> stageIds) {
+        this.stageIds = stageIds;
+    }
+
+    public String getStatus() {
+        return status;
+    }
+
+    public void setStatus(String status) {
+        this.status = status;
+    }
+
+    public int getNumTasks() {
+        return numTasks;
+    }
+
+    public void setNumTasks(int numTasks) {
+        this.numTasks = numTasks;
+    }
+
+    public int getNumActiveTasks() {
+        return numActiveTasks;
+    }
+
+    public void setNumActiveTasks(int numActiveTasks) {
+        this.numActiveTasks = numActiveTasks;
+    }
+
+    public int getNumCompletedTasks() {
+        return numCompletedTasks;
+    }
+
+    public void setNumCompletedTasks(int numCompletedTasks) {
+        this.numCompletedTasks = numCompletedTasks;
+    }
+
+    public int getNumSkippedTasks() {
+        return numSkippedTasks;
+    }
+
+    public void setNumSkippedTasks(int numSkippedTasks) {
+        this.numSkippedTasks = numSkippedTasks;
+    }
+
+    public int getNumFailedTasks() {
+        return numFailedTasks;
+    }
+
+    public void setNumFailedTasks(int numFailedTasks) {
+        this.numFailedTasks = numFailedTasks;
+    }
+
+    public int getNumActiveStages() {
+        return numActiveStages;
+    }
+
+    public void setNumActiveStages(int numActiveStages) {
+        this.numActiveStages = numActiveStages;
+    }
+
+    public int getNumCompletedStages() {
+        return numCompletedStages;
+    }
+
+    public void setNumCompletedStages(int numCompletedStages) {
+        this.numCompletedStages = numCompletedStages;
+    }
+
+    public int getNumSkippedStages() {
+        return numSkippedStages;
+    }
+
+    public void setNumSkippedStages(int numSkippedStages) {
+        this.numSkippedStages = numSkippedStages;
+    }
+
+    public int getNumFailedStages() {
+        return numFailedStages;
+    }
+
+    public void setNumFailedStages(int numFailedStages) {
+        this.numFailedStages = numFailedStages;
+    }
+}


[04/52] [abbrv] incubator-eagle git commit: [EAGLE-484][EAGLE-497] Add override equals method and fix code style of Oozie

Posted by yo...@apache.org.
[EAGLE-484][EAGLE-497] Add override equals method and fix code style of Oozie

EAGLE-497 Fix code style of Oozie
 - Fix code style of Oozie

  https://issues.apache.org/jira/browse/EAGLE-497

Author: r7raul1984 <ta...@yhd.com>

Closes #382 from r7raul1984/EAGLE-497.


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

Branch: refs/heads/master
Commit: 6a55b59f0a53f6e832318691ac1dee5a6d0b90e7
Parents: 0bde482
Author: r7raul1984 <ta...@yhd.com>
Authored: Wed Aug 24 20:20:31 2016 +0800
Committer: Hao Chen <ha...@apache.org>
Committed: Wed Aug 24 20:20:31 2016 +0800

----------------------------------------------------------------------
 .../coordination/model/PolicyWorkerQueue.java   | 22 ++++++++++++++++
 .../coordination/model/StreamRouterSpec.java    | 16 ++++++++++++
 .../alert/coordination/model/WorkSlot.java      | 23 +++++++++++++++++
 .../security/entity/OozieResourceEntity.java    | 12 ++++-----
 .../OozieResourceSensitivityAPIEntity.java      |  2 +-
 .../oozie/parse/OozieAuditLogApplication.java   | 25 +++++++++---------
 .../parse/OozieAuditLogKafkaDeserializer.java   |  9 +++----
 .../oozie/parse/OozieAuditLogParser.java        | 25 +++++++++---------
 .../oozie/parse/OozieAuditLogParserBolt.java    |  6 ++---
 .../OozieResourceSensitivityDataJoinBolt.java   | 17 +++++-------
 .../OozieSensitivityDataEnrichLCM.java          |  7 +++--
 .../OozieResourceSensitivityDataJoiner.java     |  2 +-
 .../BadOozieMetadataAccessConfigException.java  |  6 ++---
 .../oozie/dao/OozieMetadataAccessConfig.java    | 26 +++++++++----------
 .../dao/OozieMetadataAccessConfigDAOImpl.java   |  7 ++---
 .../oozie/dao/OozieMetadataDAOImpl.java         |  3 ++-
 .../oozie/dao/OozieSensitivityMetadataDAO.java  |  3 ++-
 .../dao/OozieSensitivityMetadataDAOImpl.java    | 27 ++++++++++----------
 .../TestOozieResourceSensitivityDataJoiner.java |  8 +-----
 19 files changed, 153 insertions(+), 93 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/6a55b59f/eagle-core/eagle-alert-parent/eagle-alert/alert-common/src/main/java/org/apache/eagle/alert/coordination/model/PolicyWorkerQueue.java
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-alert-parent/eagle-alert/alert-common/src/main/java/org/apache/eagle/alert/coordination/model/PolicyWorkerQueue.java b/eagle-core/eagle-alert-parent/eagle-alert/alert-common/src/main/java/org/apache/eagle/alert/coordination/model/PolicyWorkerQueue.java
index 43780f3..e32d13c 100644
--- a/eagle-core/eagle-alert-parent/eagle-alert/alert-common/src/main/java/org/apache/eagle/alert/coordination/model/PolicyWorkerQueue.java
+++ b/eagle-core/eagle-alert-parent/eagle-alert/alert-common/src/main/java/org/apache/eagle/alert/coordination/model/PolicyWorkerQueue.java
@@ -18,8 +18,11 @@ package org.apache.eagle.alert.coordination.model;
 
 import java.util.ArrayList;
 import java.util.List;
+import java.util.Objects;
 
+import org.apache.commons.collections.CollectionUtils;
 import org.apache.commons.lang3.StringUtils;
+import org.apache.commons.lang3.builder.HashCodeBuilder;
 import org.apache.eagle.alert.engine.coordinator.StreamPartition;
 
 /**
@@ -56,6 +59,7 @@ public class PolicyWorkerQueue {
         this.partition = partition;
     }
 
+
     public StreamPartition getPartition() {
         return partition;
     }
@@ -72,6 +76,24 @@ public class PolicyWorkerQueue {
         this.workers = workers;
     }
 
+    @Override
+    public boolean equals(Object other) {
+        if (this == other) {
+            return true;
+        }
+        if (!(other instanceof PolicyWorkerQueue)) {
+            return false;
+        }
+        PolicyWorkerQueue that = (PolicyWorkerQueue) other;
+        return Objects.equals(partition, that.partition) &&
+                CollectionUtils.isEqualCollection(workers, that.workers);
+    }
+
+    @Override
+    public int hashCode() {
+        return new HashCodeBuilder().append(partition).append(workers).build();
+    }
+
     public String toString() {
         return "[" + StringUtils.join(workers, ",") + "]";
     }

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/6a55b59f/eagle-core/eagle-alert-parent/eagle-alert/alert-common/src/main/java/org/apache/eagle/alert/coordination/model/StreamRouterSpec.java
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-alert-parent/eagle-alert/alert-common/src/main/java/org/apache/eagle/alert/coordination/model/StreamRouterSpec.java b/eagle-core/eagle-alert-parent/eagle-alert/alert-common/src/main/java/org/apache/eagle/alert/coordination/model/StreamRouterSpec.java
index 773ae56..195c8d9 100644
--- a/eagle-core/eagle-alert-parent/eagle-alert/alert-common/src/main/java/org/apache/eagle/alert/coordination/model/StreamRouterSpec.java
+++ b/eagle-core/eagle-alert-parent/eagle-alert/alert-common/src/main/java/org/apache/eagle/alert/coordination/model/StreamRouterSpec.java
@@ -18,7 +18,9 @@ package org.apache.eagle.alert.coordination.model;
 
 import java.util.ArrayList;
 import java.util.List;
+import java.util.Objects;
 
+import org.apache.commons.collections.CollectionUtils;
 import org.apache.commons.lang3.builder.HashCodeBuilder;
 import org.apache.eagle.alert.engine.coordinator.StreamPartition;
 
@@ -72,4 +74,18 @@ public class StreamRouterSpec {
         return String.format("StreamRouterSpec[streamId=%s,partition=%s, queue=[%s]]", this.getStreamId(),
                 this.getPartition(), this.getTargetQueue());
     }
+
+    @Override
+    public boolean equals(Object other) {
+        if (this == other) {
+            return true;
+        }
+        if (!(other instanceof StreamRouterSpec)) {
+            return false;
+        }
+        StreamRouterSpec that = (StreamRouterSpec) other;
+        return Objects.equals(streamId, that.streamId) &&
+                Objects.equals(partition, that.partition) &&
+                CollectionUtils.isEqualCollection(targetQueue, that.targetQueue);
+    }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/6a55b59f/eagle-core/eagle-alert-parent/eagle-alert/alert-common/src/main/java/org/apache/eagle/alert/coordination/model/WorkSlot.java
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-alert-parent/eagle-alert/alert-common/src/main/java/org/apache/eagle/alert/coordination/model/WorkSlot.java b/eagle-core/eagle-alert-parent/eagle-alert/alert-common/src/main/java/org/apache/eagle/alert/coordination/model/WorkSlot.java
index 96016f4..3f6f36d 100644
--- a/eagle-core/eagle-alert-parent/eagle-alert/alert-common/src/main/java/org/apache/eagle/alert/coordination/model/WorkSlot.java
+++ b/eagle-core/eagle-alert-parent/eagle-alert/alert-common/src/main/java/org/apache/eagle/alert/coordination/model/WorkSlot.java
@@ -17,6 +17,10 @@
 package org.apache.eagle.alert.coordination.model;
 
 
+import org.apache.commons.lang3.builder.HashCodeBuilder;
+
+import java.util.Objects;
+
 /**
  * A slot is simply a bolt.
  */
@@ -44,11 +48,30 @@ public class WorkSlot {
 
     }
 
+
     public WorkSlot(String topo, String boltId) {
         this.topologyName = topo;
         this.boltId = boltId;
     }
 
+    @Override
+    public boolean equals(Object other) {
+        if (this == other) {
+            return true;
+        }
+        if (!(other instanceof WorkSlot)) {
+            return false;
+        }
+        WorkSlot workSlot = (WorkSlot) other;
+        return Objects.equals(topologyName, workSlot.topologyName) &&
+                Objects.equals(boltId, workSlot.boltId);
+    }
+
+    @Override
+    public int hashCode() {
+        return new HashCodeBuilder().append(topologyName).append(boltId).build();
+    }
+
     public String toString() {
         return "(" + topologyName + ":" + boltId + ")";
     }

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/6a55b59f/eagle-security/eagle-security-common/src/main/java/org/apache/eagle/security/entity/OozieResourceEntity.java
----------------------------------------------------------------------
diff --git a/eagle-security/eagle-security-common/src/main/java/org/apache/eagle/security/entity/OozieResourceEntity.java b/eagle-security/eagle-security-common/src/main/java/org/apache/eagle/security/entity/OozieResourceEntity.java
index 0506dd2..4dc2249 100644
--- a/eagle-security/eagle-security-common/src/main/java/org/apache/eagle/security/entity/OozieResourceEntity.java
+++ b/eagle-security/eagle-security-common/src/main/java/org/apache/eagle/security/entity/OozieResourceEntity.java
@@ -32,7 +32,7 @@ public class OozieResourceEntity implements Serializable {
     }
 
     public OozieResourceEntity(String jobId, String name) {
-        this(jobId,name,null);
+        this(jobId, name, null);
     }
 
     public String getJobId() {
@@ -72,11 +72,11 @@ public class OozieResourceEntity implements Serializable {
 
     @Override
     public String toString() {
-        return "OozieResourceEntity{" +
-                "jobId='" + jobId + '\'' +
-                ", name='" + name + '\'' +
-                ", sensitiveType='" + sensitiveType + '\'' +
-                '}';
+        return "OozieResourceEntity{"
+                + "jobId='" + jobId + '\''
+                + ", name='" + name + '\''
+                + ", sensitiveType='" + sensitiveType + '\''
+                + '}';
     }
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/6a55b59f/eagle-security/eagle-security-common/src/main/java/org/apache/eagle/security/entity/OozieResourceSensitivityAPIEntity.java
----------------------------------------------------------------------
diff --git a/eagle-security/eagle-security-common/src/main/java/org/apache/eagle/security/entity/OozieResourceSensitivityAPIEntity.java b/eagle-security/eagle-security-common/src/main/java/org/apache/eagle/security/entity/OozieResourceSensitivityAPIEntity.java
index 82953f5..767a4fb 100644
--- a/eagle-security/eagle-security-common/src/main/java/org/apache/eagle/security/entity/OozieResourceSensitivityAPIEntity.java
+++ b/eagle-security/eagle-security-common/src/main/java/org/apache/eagle/security/entity/OozieResourceSensitivityAPIEntity.java
@@ -30,7 +30,7 @@ import org.codehaus.jackson.map.annotate.JsonSerialize;
 public class OozieResourceSensitivityAPIEntity extends TaggedLogAPIEntity {
     private static final long serialVersionUID = 2L;
     /**
-     * sensitivityType can be multi-value attribute, and values can be separated by "|"
+     * sensitivityType can be multi-value attribute, and values can be separated by "|".
      */
     @Column("a")
     private String sensitivityType;

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/6a55b59f/eagle-security/eagle-security-oozie-auditlog/src/main/java/org/apache/eagle/security/oozie/parse/OozieAuditLogApplication.java
----------------------------------------------------------------------
diff --git a/eagle-security/eagle-security-oozie-auditlog/src/main/java/org/apache/eagle/security/oozie/parse/OozieAuditLogApplication.java b/eagle-security/eagle-security-oozie-auditlog/src/main/java/org/apache/eagle/security/oozie/parse/OozieAuditLogApplication.java
index 4204a17..23fef45 100644
--- a/eagle-security/eagle-security-oozie-auditlog/src/main/java/org/apache/eagle/security/oozie/parse/OozieAuditLogApplication.java
+++ b/eagle-security/eagle-security-oozie-auditlog/src/main/java/org/apache/eagle/security/oozie/parse/OozieAuditLogApplication.java
@@ -17,6 +17,12 @@
  */
 package org.apache.eagle.security.oozie.parse;
 
+import org.apache.eagle.app.StormApplication;
+import org.apache.eagle.app.environment.impl.StormEnvironment;
+import org.apache.eagle.app.sink.StormStreamSink;
+import org.apache.eagle.dataproc.impl.storm.kafka.KafkaSpoutProvider;
+import org.apache.eagle.security.oozie.parse.sensitivity.OozieResourceSensitivityDataJoinBolt;
+
 import backtype.storm.generated.StormTopology;
 import backtype.storm.topology.BoltDeclarer;
 import backtype.storm.topology.IRichSpout;
@@ -24,20 +30,15 @@ import backtype.storm.topology.TopologyBuilder;
 import backtype.storm.tuple.Fields;
 import com.typesafe.config.Config;
 import com.typesafe.config.ConfigFactory;
-import org.apache.eagle.app.StormApplication;
-import org.apache.eagle.app.environment.impl.StormEnvironment;
-import org.apache.eagle.app.sink.StormStreamSink;
-import org.apache.eagle.security.oozie.parse.sensitivity.OozieResourceSensitivityDataJoinBolt;
-import org.apache.eagle.dataproc.impl.storm.kafka.KafkaSpoutProvider;
 
 /**
  * Since 8/12/16.
  */
 public class OozieAuditLogApplication extends StormApplication {
-    public final static String SPOUT_TASK_NUM = "topology.numOfSpoutTasks";
-    public final static String PARSER_TASK_NUM = "topology.numOfParserTasks";
-    public final static String JOIN_TASK_NUM = "topology.numOfJoinTasks";
-    public final static String SINK_TASK_NUM = "topology.numOfSinkTasks";
+    public static final String SPOUT_TASK_NUM = "topology.numOfSpoutTasks";
+    public static final String PARSER_TASK_NUM = "topology.numOfParserTasks";
+    public static final String JOIN_TASK_NUM = "topology.numOfJoinTasks";
+    public static final String SINK_TASK_NUM = "topology.numOfSinkTasks";
 
     @Override
     public StormTopology execute(Config config, StormEnvironment environment) {
@@ -48,7 +49,6 @@ public class OozieAuditLogApplication extends StormApplication {
         int numOfSpoutTasks = config.getInt(SPOUT_TASK_NUM);
         int numOfParserTask = config.getInt(PARSER_TASK_NUM);
         int numOfJoinTasks = config.getInt(JOIN_TASK_NUM);
-        int numOfSinkTasks = config.getInt(SINK_TASK_NUM);
 
         builder.setSpout("ingest", spout, numOfSpoutTasks);
 
@@ -61,13 +61,14 @@ public class OozieAuditLogApplication extends StormApplication {
         BoltDeclarer boltDeclarer = builder.setBolt("joinBolt", joinBolt, numOfJoinTasks);
         boltDeclarer.fieldsGrouping("parserBolt", new Fields("f1"));
 
-        StormStreamSink sinkBolt = environment.getStreamSink("oozie_audit_log_stream",config);
+        StormStreamSink sinkBolt = environment.getStreamSink("oozie_audit_log_stream", config);
+        int numOfSinkTasks = config.getInt(SINK_TASK_NUM);
         BoltDeclarer kafkaBoltDeclarer = builder.setBolt("kafkaSink", sinkBolt, numOfSinkTasks);
         kafkaBoltDeclarer.fieldsGrouping("joinBolt", new Fields("user"));
         return builder.createTopology();
     }
 
-    public static void main(String[] args){
+    public static void main(String[] args) {
         Config config = ConfigFactory.load();
         OozieAuditLogApplication app = new OozieAuditLogApplication();
         app.run(config);

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/6a55b59f/eagle-security/eagle-security-oozie-auditlog/src/main/java/org/apache/eagle/security/oozie/parse/OozieAuditLogKafkaDeserializer.java
----------------------------------------------------------------------
diff --git a/eagle-security/eagle-security-oozie-auditlog/src/main/java/org/apache/eagle/security/oozie/parse/OozieAuditLogKafkaDeserializer.java b/eagle-security/eagle-security-oozie-auditlog/src/main/java/org/apache/eagle/security/oozie/parse/OozieAuditLogKafkaDeserializer.java
index 06d6a3e..b7413e4 100644
--- a/eagle-security/eagle-security-oozie-auditlog/src/main/java/org/apache/eagle/security/oozie/parse/OozieAuditLogKafkaDeserializer.java
+++ b/eagle-security/eagle-security-oozie-auditlog/src/main/java/org/apache/eagle/security/oozie/parse/OozieAuditLogKafkaDeserializer.java
@@ -17,7 +17,6 @@
  */
 package org.apache.eagle.security.oozie.parse;
 
-
 import org.apache.eagle.dataproc.impl.storm.kafka.SpoutKafkaMessageDeserializer;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -31,7 +30,7 @@ public class OozieAuditLogKafkaDeserializer implements SpoutKafkaMessageDeserial
     private static Logger LOG = LoggerFactory.getLogger(OozieAuditLogKafkaDeserializer.class);
     private Properties props;
 
-    public OozieAuditLogKafkaDeserializer(Properties props){
+    public OozieAuditLogKafkaDeserializer(Properties props) {
         this.props = props;
     }
 
@@ -41,12 +40,12 @@ public class OozieAuditLogKafkaDeserializer implements SpoutKafkaMessageDeserial
 
         OozieAuditLogParser parser = new OozieAuditLogParser();
         OozieAuditLogObject entity = null;
-        try{
+        try {
             entity = parser.parse(logLine);
-        }catch(Exception ex){
+        } catch (Exception ex) {
             LOG.error("Failing oozie parse audit log message", ex);
         }
-        if(entity == null){
+        if (entity == null) {
             LOG.warn("Event ignored as it can't be correctly parsed, the log is ", logLine);
             return null;
         }

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/6a55b59f/eagle-security/eagle-security-oozie-auditlog/src/main/java/org/apache/eagle/security/oozie/parse/OozieAuditLogParser.java
----------------------------------------------------------------------
diff --git a/eagle-security/eagle-security-oozie-auditlog/src/main/java/org/apache/eagle/security/oozie/parse/OozieAuditLogParser.java b/eagle-security/eagle-security-oozie-auditlog/src/main/java/org/apache/eagle/security/oozie/parse/OozieAuditLogParser.java
index 56228f8..bb88895 100644
--- a/eagle-security/eagle-security-oozie-auditlog/src/main/java/org/apache/eagle/security/oozie/parse/OozieAuditLogParser.java
+++ b/eagle-security/eagle-security-oozie-auditlog/src/main/java/org/apache/eagle/security/oozie/parse/OozieAuditLogParser.java
@@ -17,9 +17,10 @@
  */
 package org.apache.eagle.security.oozie.parse;
 
-import org.apache.commons.lang.StringUtils;
 import org.apache.eagle.common.DateTimeUtil;
 
+import org.apache.commons.lang.StringUtils;
+
 import java.text.ParseException;
 import java.util.ArrayList;
 import java.util.List;
@@ -37,17 +38,17 @@ public class OozieAuditLogParser {
     private static final String OOZIEAUDIT_FLAG = "(\\w+:\\d+)";
     private static final String PREFIX_REGEX = TIMESTAMP_REGEX + WHITE_SPACE_REGEX + LOG_LEVEL_REGEX
             + WHITE_SPACE_REGEX;
-    private final static String IP = "IP";
-    private final static String USER = "USER";
-    private final static String GROUP = "GROUP";
-    private final static String APP = "APP";
-    private final static String JOBID = "JOBID";
-    private final static String OPERATION = "OPERATION";
-    private final static String PARAMETER = "PARAMETER";
-    private final static String STATUS = "STATUS";
-    private final static String HTTPCODE = "HTTPCODE";
-    private final static String ERRORCODE = "ERRORCODE";
-    private final static String ERRORMESSAGE = "ERRORMESSAGE";
+    private static final String IP = "IP";
+    private static final String USER = "USER";
+    private static final String GROUP = "GROUP";
+    private static final String APP = "APP";
+    private static final String JOBID = "JOBID";
+    private static final String OPERATION = "OPERATION";
+    private static final String PARAMETER = "PARAMETER";
+    private static final String STATUS = "STATUS";
+    private static final String HTTPCODE = "HTTPCODE";
+    private static final String ERRORCODE = "ERRORCODE";
+    private static final String ERRORMESSAGE = "ERRORMESSAGE";
     private static final Pattern LOG_PATTERN = constructPattern();
 
     public OozieAuditLogObject parse(String logLine) throws Exception {

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/6a55b59f/eagle-security/eagle-security-oozie-auditlog/src/main/java/org/apache/eagle/security/oozie/parse/OozieAuditLogParserBolt.java
----------------------------------------------------------------------
diff --git a/eagle-security/eagle-security-oozie-auditlog/src/main/java/org/apache/eagle/security/oozie/parse/OozieAuditLogParserBolt.java b/eagle-security/eagle-security-oozie-auditlog/src/main/java/org/apache/eagle/security/oozie/parse/OozieAuditLogParserBolt.java
index 13f5e8c..ed59ec1 100644
--- a/eagle-security/eagle-security-oozie-auditlog/src/main/java/org/apache/eagle/security/oozie/parse/OozieAuditLogParserBolt.java
+++ b/eagle-security/eagle-security-oozie-auditlog/src/main/java/org/apache/eagle/security/oozie/parse/OozieAuditLogParserBolt.java
@@ -36,7 +36,7 @@ import java.util.TreeMap;
 /**
  * Since 8/12/16.
  */
-public class OozieAuditLogParserBolt extends BaseRichBolt{
+public class OozieAuditLogParserBolt extends BaseRichBolt {
     private static Logger LOG = LoggerFactory.getLogger(OozieAuditLogParserBolt.class);
     private OutputCollector collector;
 
@@ -72,9 +72,9 @@ public class OozieAuditLogParserBolt extends BaseRichBolt{
             map.put("errorcode", entity.errorcode);
             map.put("errormessage", entity.errormessage);
             collector.emit(Arrays.asList(map));
-        }catch(Exception ex){
+        } catch (Exception ex) {
             LOG.error("error in parsing oozie audit log", ex);
-        }finally {
+        } finally {
             collector.ack(input);
         }
     }

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/6a55b59f/eagle-security/eagle-security-oozie-auditlog/src/main/java/org/apache/eagle/security/oozie/parse/sensitivity/OozieResourceSensitivityDataJoinBolt.java
----------------------------------------------------------------------
diff --git a/eagle-security/eagle-security-oozie-auditlog/src/main/java/org/apache/eagle/security/oozie/parse/sensitivity/OozieResourceSensitivityDataJoinBolt.java b/eagle-security/eagle-security-oozie-auditlog/src/main/java/org/apache/eagle/security/oozie/parse/sensitivity/OozieResourceSensitivityDataJoinBolt.java
index 91f9e6f..156e326 100644
--- a/eagle-security/eagle-security-oozie-auditlog/src/main/java/org/apache/eagle/security/oozie/parse/sensitivity/OozieResourceSensitivityDataJoinBolt.java
+++ b/eagle-security/eagle-security-oozie-auditlog/src/main/java/org/apache/eagle/security/oozie/parse/sensitivity/OozieResourceSensitivityDataJoinBolt.java
@@ -17,17 +17,14 @@
  */
 package org.apache.eagle.security.oozie.parse.sensitivity;
 
+import org.apache.eagle.security.enrich.AbstractDataEnrichBolt;
+import org.apache.eagle.security.entity.OozieResourceSensitivityAPIEntity;
+
 import backtype.storm.task.OutputCollector;
-import backtype.storm.task.TopologyContext;
 import backtype.storm.topology.OutputFieldsDeclarer;
-import backtype.storm.topology.base.BaseRichBolt;
 import backtype.storm.tuple.Fields;
 import backtype.storm.tuple.Tuple;
 import com.typesafe.config.Config;
-import org.apache.eagle.security.enrich.AbstractDataEnrichBolt;
-import org.apache.eagle.security.entity.OozieResourceSensitivityAPIEntity;
-import org.apache.eagle.security.enrich.ExternalDataCache;
-import org.apache.eagle.security.enrich.ExternalDataJoiner;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -37,11 +34,11 @@ import java.util.TreeMap;
 import java.util.regex.Pattern;
 
 public class OozieResourceSensitivityDataJoinBolt extends AbstractDataEnrichBolt<OozieResourceSensitivityAPIEntity, String> {
-    private final static Logger LOG = LoggerFactory.getLogger(OozieResourceSensitivityDataJoinBolt.class);
+    private static final Logger LOG = LoggerFactory.getLogger(OozieResourceSensitivityDataJoinBolt.class);
     private Config config;
     private OutputCollector collector;
 
-    public OozieResourceSensitivityDataJoinBolt(Config config){
+    public OozieResourceSensitivityDataJoinBolt(Config config) {
         super(config, new OozieSensitivityDataEnrichLCM(config));
     }
 
@@ -72,9 +69,9 @@ public class OozieResourceSensitivityDataJoinBolt extends AbstractDataEnrichBolt
             }
             LOG.info("After oozie resource sensitivity lookup: " + newEvent);
             collector.emit(Arrays.asList(newEvent.get("user"), newEvent));
-        }catch(Exception ex){
+        } catch (Exception ex) {
             LOG.error("error join external sensitivity data", ex);
-        }finally {
+        } finally {
             collector.ack(input);
         }
     }

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/6a55b59f/eagle-security/eagle-security-oozie-auditlog/src/main/java/org/apache/eagle/security/oozie/parse/sensitivity/OozieSensitivityDataEnrichLCM.java
----------------------------------------------------------------------
diff --git a/eagle-security/eagle-security-oozie-auditlog/src/main/java/org/apache/eagle/security/oozie/parse/sensitivity/OozieSensitivityDataEnrichLCM.java b/eagle-security/eagle-security-oozie-auditlog/src/main/java/org/apache/eagle/security/oozie/parse/sensitivity/OozieSensitivityDataEnrichLCM.java
index b974c5f..eec4058 100644
--- a/eagle-security/eagle-security-oozie-auditlog/src/main/java/org/apache/eagle/security/oozie/parse/sensitivity/OozieSensitivityDataEnrichLCM.java
+++ b/eagle-security/eagle-security-oozie-auditlog/src/main/java/org/apache/eagle/security/oozie/parse/sensitivity/OozieSensitivityDataEnrichLCM.java
@@ -17,19 +17,22 @@
 
 package org.apache.eagle.security.oozie.parse.sensitivity;
 
-import com.typesafe.config.Config;
+
 import org.apache.eagle.security.enrich.AbstractDataEnrichLCM;
 import org.apache.eagle.security.service.ISecurityDataEnrichServiceClient;
 
+import com.typesafe.config.Config;
+
 import java.util.Collection;
 
 /**
  * Since 8/16/16.
  */
 public class OozieSensitivityDataEnrichLCM extends AbstractDataEnrichLCM {
-    public OozieSensitivityDataEnrichLCM(Config config){
+    public OozieSensitivityDataEnrichLCM(Config config) {
         super(config);
     }
+
     @Override
     protected Collection loadFromService(ISecurityDataEnrichServiceClient client) {
         return null;

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/6a55b59f/eagle-security/eagle-security-oozie-web/src/main/java/org/apache/eagle/service/security/oozie/OozieResourceSensitivityDataJoiner.java
----------------------------------------------------------------------
diff --git a/eagle-security/eagle-security-oozie-web/src/main/java/org/apache/eagle/service/security/oozie/OozieResourceSensitivityDataJoiner.java b/eagle-security/eagle-security-oozie-web/src/main/java/org/apache/eagle/service/security/oozie/OozieResourceSensitivityDataJoiner.java
index 4bfe814..497dba6 100644
--- a/eagle-security/eagle-security-oozie-web/src/main/java/org/apache/eagle/service/security/oozie/OozieResourceSensitivityDataJoiner.java
+++ b/eagle-security/eagle-security-oozie-web/src/main/java/org/apache/eagle/service/security/oozie/OozieResourceSensitivityDataJoiner.java
@@ -29,7 +29,7 @@ import java.util.Map;
 
 public class OozieResourceSensitivityDataJoiner {
 
-    private final static Logger LOG = LoggerFactory.getLogger(OozieResourceSensitivityDataJoiner.class);
+    private static final Logger LOG = LoggerFactory.getLogger(OozieResourceSensitivityDataJoiner.class);
 
     public List<OozieResourceEntity> joinOozieResourceSensitivity(String site, List<CoordinatorJob> coordinatorJobs) {
 

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/6a55b59f/eagle-security/eagle-security-oozie-web/src/main/java/org/apache/eagle/service/security/oozie/dao/BadOozieMetadataAccessConfigException.java
----------------------------------------------------------------------
diff --git a/eagle-security/eagle-security-oozie-web/src/main/java/org/apache/eagle/service/security/oozie/dao/BadOozieMetadataAccessConfigException.java b/eagle-security/eagle-security-oozie-web/src/main/java/org/apache/eagle/service/security/oozie/dao/BadOozieMetadataAccessConfigException.java
index 7c8b650..63fb57e 100644
--- a/eagle-security/eagle-security-oozie-web/src/main/java/org/apache/eagle/service/security/oozie/dao/BadOozieMetadataAccessConfigException.java
+++ b/eagle-security/eagle-security-oozie-web/src/main/java/org/apache/eagle/service/security/oozie/dao/BadOozieMetadataAccessConfigException.java
@@ -16,12 +16,12 @@
  */
 package org.apache.eagle.service.security.oozie.dao;
 
-public class BadOozieMetadataAccessConfigException extends RuntimeException{
-    public BadOozieMetadataAccessConfigException(Exception ex){
+public class BadOozieMetadataAccessConfigException extends RuntimeException {
+    public BadOozieMetadataAccessConfigException(Exception ex) {
         super(ex);
     }
 
-    public BadOozieMetadataAccessConfigException(String msg){
+    public BadOozieMetadataAccessConfigException(String msg) {
         super(msg);
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/6a55b59f/eagle-security/eagle-security-oozie-web/src/main/java/org/apache/eagle/service/security/oozie/dao/OozieMetadataAccessConfig.java
----------------------------------------------------------------------
diff --git a/eagle-security/eagle-security-oozie-web/src/main/java/org/apache/eagle/service/security/oozie/dao/OozieMetadataAccessConfig.java b/eagle-security/eagle-security-oozie-web/src/main/java/org/apache/eagle/service/security/oozie/dao/OozieMetadataAccessConfig.java
index d2e6a76..02115d8 100644
--- a/eagle-security/eagle-security-oozie-web/src/main/java/org/apache/eagle/service/security/oozie/dao/OozieMetadataAccessConfig.java
+++ b/eagle-security/eagle-security-oozie-web/src/main/java/org/apache/eagle/service/security/oozie/dao/OozieMetadataAccessConfig.java
@@ -61,26 +61,26 @@ public class OozieMetadataAccessConfig {
         return "accessType:" + accessType + ",oozieUrl:" + oozieUrl + ",filter:" + filter + ",authType:" + authType;
     }
 
-    public final static class OOZIECONF {
-        public final static String ACCESSTYPE = "accessType";
-        public final static String OOZIEURL = "oozieUrl";
-        public final static String FILTER = "filter";
-        public final static String AUTHTYPE = "authType";
+    public static final class OozieConf {
+        public static final String ACCESSTYPE = "accessType";
+        public static final String OOZIEURL = "oozieUrl";
+        public static final String FILTER = "filter";
+        public static final String AUTHTYPE = "authType";
     }
 
     public static OozieMetadataAccessConfig config2Entity(Config config) {
         OozieMetadataAccessConfig oozieconf = new OozieMetadataAccessConfig();
-        if(config.hasPath(OOZIECONF.ACCESSTYPE)) {
-            oozieconf.setAccessType(config.getString(OOZIECONF.ACCESSTYPE));
+        if (config.hasPath(OozieConf.ACCESSTYPE)) {
+            oozieconf.setAccessType(config.getString(OozieConf.ACCESSTYPE));
         }
-        if(config.hasPath(OOZIECONF.OOZIEURL)) {
-            oozieconf.setOozieUrl(config.getString(OOZIECONF.OOZIEURL));
+        if (config.hasPath(OozieConf.OOZIEURL)) {
+            oozieconf.setOozieUrl(config.getString(OozieConf.OOZIEURL));
         }
-        if(config.hasPath(OOZIECONF.FILTER)) {
-            oozieconf.setFilter(config.getString(OOZIECONF.FILTER));
+        if (config.hasPath(OozieConf.FILTER)) {
+            oozieconf.setFilter(config.getString(OozieConf.FILTER));
         }
-        if(config.hasPath(OOZIECONF.AUTHTYPE)) {
-            oozieconf.setAuthType(config.getString(OOZIECONF.AUTHTYPE));
+        if (config.hasPath(OozieConf.AUTHTYPE)) {
+            oozieconf.setAuthType(config.getString(OozieConf.AUTHTYPE));
         }
         return oozieconf;
     }

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/6a55b59f/eagle-security/eagle-security-oozie-web/src/main/java/org/apache/eagle/service/security/oozie/dao/OozieMetadataAccessConfigDAOImpl.java
----------------------------------------------------------------------
diff --git a/eagle-security/eagle-security-oozie-web/src/main/java/org/apache/eagle/service/security/oozie/dao/OozieMetadataAccessConfigDAOImpl.java b/eagle-security/eagle-security-oozie-web/src/main/java/org/apache/eagle/service/security/oozie/dao/OozieMetadataAccessConfigDAOImpl.java
index 8cab617..320e1b4 100644
--- a/eagle-security/eagle-security-oozie-web/src/main/java/org/apache/eagle/service/security/oozie/dao/OozieMetadataAccessConfigDAOImpl.java
+++ b/eagle-security/eagle-security-oozie-web/src/main/java/org/apache/eagle/service/security/oozie/dao/OozieMetadataAccessConfigDAOImpl.java
@@ -16,12 +16,13 @@
  */
 package org.apache.eagle.service.security.oozie.dao;
 
-import com.typesafe.config.Config;
 import org.apache.eagle.security.resolver.MetadataAccessConfigRepo;
 
-public class OozieMetadataAccessConfigDAOImpl implements OozieMetadataAccessConfigDAO{
+import com.typesafe.config.Config;
+
+public class OozieMetadataAccessConfigDAOImpl implements OozieMetadataAccessConfigDAO {
     @Override
-    public OozieMetadataAccessConfig getConfig(String site) throws Exception{
+    public OozieMetadataAccessConfig getConfig(String site) throws Exception {
         MetadataAccessConfigRepo repo = new MetadataAccessConfigRepo();
         Config config = repo.getConfig("oozieAuditLog", site);
         return OozieMetadataAccessConfig.config2Entity(config);

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/6a55b59f/eagle-security/eagle-security-oozie-web/src/main/java/org/apache/eagle/service/security/oozie/dao/OozieMetadataDAOImpl.java
----------------------------------------------------------------------
diff --git a/eagle-security/eagle-security-oozie-web/src/main/java/org/apache/eagle/service/security/oozie/dao/OozieMetadataDAOImpl.java b/eagle-security/eagle-security-oozie-web/src/main/java/org/apache/eagle/service/security/oozie/dao/OozieMetadataDAOImpl.java
index 65b4bcb..adbf7e6 100644
--- a/eagle-security/eagle-security-oozie-web/src/main/java/org/apache/eagle/service/security/oozie/dao/OozieMetadataDAOImpl.java
+++ b/eagle-security/eagle-security-oozie-web/src/main/java/org/apache/eagle/service/security/oozie/dao/OozieMetadataDAOImpl.java
@@ -32,8 +32,9 @@ public class OozieMetadataDAOImpl implements OozieMetadataDAO {
     private OozieMetadataAccessConfig config;
 
     public OozieMetadataDAOImpl(OozieMetadataAccessConfig config) {
-        if (config.getAccessType() == null)
+        if (config.getAccessType() == null) {
             throw new BadOozieMetadataAccessConfigException("access Type is null, options: [oozie_api]");
+        }
         this.config = config;
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/6a55b59f/eagle-security/eagle-security-oozie-web/src/main/java/org/apache/eagle/service/security/oozie/dao/OozieSensitivityMetadataDAO.java
----------------------------------------------------------------------
diff --git a/eagle-security/eagle-security-oozie-web/src/main/java/org/apache/eagle/service/security/oozie/dao/OozieSensitivityMetadataDAO.java b/eagle-security/eagle-security-oozie-web/src/main/java/org/apache/eagle/service/security/oozie/dao/OozieSensitivityMetadataDAO.java
index dc16d01..972c515 100644
--- a/eagle-security/eagle-security-oozie-web/src/main/java/org/apache/eagle/service/security/oozie/dao/OozieSensitivityMetadataDAO.java
+++ b/eagle-security/eagle-security-oozie-web/src/main/java/org/apache/eagle/service/security/oozie/dao/OozieSensitivityMetadataDAO.java
@@ -19,11 +19,12 @@ package org.apache.eagle.service.security.oozie.dao;
 import java.util.Map;
 
 /**
- * retrieve oozie sensitivity metadata directly from table
+ * retrieve oozie sensitivity metadata directly from table.
  */
 public interface OozieSensitivityMetadataDAO {
     // site to (resource, sensitivityType)
     Map<String, Map<String, String>> getAllOozieSensitivityMap();
+
     // resource to sensitivityType
     Map<String, String> getOozieSensitivityMap(String site);
 }

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/6a55b59f/eagle-security/eagle-security-oozie-web/src/main/java/org/apache/eagle/service/security/oozie/dao/OozieSensitivityMetadataDAOImpl.java
----------------------------------------------------------------------
diff --git a/eagle-security/eagle-security-oozie-web/src/main/java/org/apache/eagle/service/security/oozie/dao/OozieSensitivityMetadataDAOImpl.java b/eagle-security/eagle-security-oozie-web/src/main/java/org/apache/eagle/service/security/oozie/dao/OozieSensitivityMetadataDAOImpl.java
index 247eb90..c25ad6a 100644
--- a/eagle-security/eagle-security-oozie-web/src/main/java/org/apache/eagle/service/security/oozie/dao/OozieSensitivityMetadataDAOImpl.java
+++ b/eagle-security/eagle-security-oozie-web/src/main/java/org/apache/eagle/service/security/oozie/dao/OozieSensitivityMetadataDAOImpl.java
@@ -27,32 +27,32 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 
-public class OozieSensitivityMetadataDAOImpl implements OozieSensitivityMetadataDAO{
+public class OozieSensitivityMetadataDAOImpl implements OozieSensitivityMetadataDAO {
     private static Logger LOG = LoggerFactory.getLogger(OozieSensitivityMetadataDAOImpl.class);
 
     @Override
-    public Map<String, Map<String, String>> getAllOozieSensitivityMap(){
+    public Map<String, Map<String, String>> getAllOozieSensitivityMap() {
         GenericEntityServiceResource resource = new GenericEntityServiceResource();
         /* parameters are: query, startTime, endTime, pageSzie, startRowkey, treeAgg, timeSeries, intervalmin, top, filterIfMissing,
         * parallel, metricName*/
         GenericServiceAPIResponseEntity ret = resource.search("OozieResourceSensitivityService[]{*}", null, null, Integer.MAX_VALUE, null, false, false, 0L, 0, false,
                 0, null, false);
         List<OozieResourceSensitivityAPIEntity> list = (List<OozieResourceSensitivityAPIEntity>) ret.getObj();
-        if( list == null )
+        if (list == null) {
             return Collections.emptyMap();
+        }
         Map<String, Map<String, String>> res = new HashMap<String, Map<String, String>>();
 
-        for(OozieResourceSensitivityAPIEntity entity : list){
+        for (OozieResourceSensitivityAPIEntity entity : list) {
             String site = entity.getTags().get("site");
-            if(entity.getTags().containsKey("oozieResource")) {
-                if(res.get(site) == null){
+            if (entity.getTags().containsKey("oozieResource")) {
+                if (res.get(site) == null) {
                     res.put(site, new HashMap<String, String>());
                 }
                 Map<String, String> resSensitivityMap = res.get(site);
                 resSensitivityMap.put(entity.getTags().get("oozieResource"), entity.getSensitivityType());
-            }
-            else {
-                if(LOG.isDebugEnabled()) {
+            } else {
+                if (LOG.isDebugEnabled()) {
                     LOG.debug("An invalid sensitivity entity is detected" + entity);
                 }
             }
@@ -61,17 +61,18 @@ public class OozieSensitivityMetadataDAOImpl implements OozieSensitivityMetadata
     }
 
     @Override
-    public Map<String, String> getOozieSensitivityMap(String site){
+    public Map<String, String> getOozieSensitivityMap(String site) {
         GenericEntityServiceResource resource = new GenericEntityServiceResource();
         String queryFormat = "OozieResourceSensitivityService[@site=\"%s\"]{*}";
         GenericServiceAPIResponseEntity ret = resource.search(String.format(queryFormat, site), null, null, Integer.MAX_VALUE, null, false, false, 0L, 0, false,
                 0, null, false);
         List<OozieResourceSensitivityAPIEntity> list = (List<OozieResourceSensitivityAPIEntity>) ret.getObj();
-        if( list == null )
+        if (list == null) {
             return Collections.emptyMap();
+        }
         Map<String, String> resSensitivityMap = new HashMap<String, String>();
-        for(OozieResourceSensitivityAPIEntity entity : list){
-            if(entity.getTags().containsKey("oozieResource")) {
+        for (OozieResourceSensitivityAPIEntity entity : list) {
+            if (entity.getTags().containsKey("oozieResource")) {
                 resSensitivityMap.put(entity.getTags().get("oozieResource"), entity.getSensitivityType());
             }
         }

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/6a55b59f/eagle-security/eagle-security-oozie-web/src/test/java/org/apache/eagle/service/security/oozie/TestOozieResourceSensitivityDataJoiner.java
----------------------------------------------------------------------
diff --git a/eagle-security/eagle-security-oozie-web/src/test/java/org/apache/eagle/service/security/oozie/TestOozieResourceSensitivityDataJoiner.java b/eagle-security/eagle-security-oozie-web/src/test/java/org/apache/eagle/service/security/oozie/TestOozieResourceSensitivityDataJoiner.java
index 50cd1f1..ac3cc0b 100644
--- a/eagle-security/eagle-security-oozie-web/src/test/java/org/apache/eagle/service/security/oozie/TestOozieResourceSensitivityDataJoiner.java
+++ b/eagle-security/eagle-security-oozie-web/src/test/java/org/apache/eagle/service/security/oozie/TestOozieResourceSensitivityDataJoiner.java
@@ -16,11 +16,7 @@
  */
 package org.apache.eagle.service.security.oozie;
 
-import org.apache.eagle.log.entity.GenericServiceAPIResponseEntity;
 import org.apache.eagle.security.entity.OozieResourceEntity;
-import org.apache.eagle.security.entity.OozieResourceSensitivityAPIEntity;
-import org.apache.eagle.service.generic.GenericEntityServiceResource;
-import org.apache.eagle.service.security.oozie.dao.OozieSensitivityMetadataDAO;
 import org.apache.eagle.service.security.oozie.dao.OozieSensitivityMetadataDAOImpl;
 import org.apache.oozie.client.CoordinatorJob;
 import org.apache.oozie.client.rest.JsonTags;
@@ -41,9 +37,7 @@ import java.util.List;
 import java.util.Map;
 
 import static org.easymock.EasyMock.expect;
-import static org.powermock.api.easymock.PowerMock.createMock;
-import static org.powermock.api.easymock.PowerMock.expectNew;
-import static org.powermock.api.easymock.PowerMock.replay;
+import static org.powermock.api.easymock.PowerMock.*;
 
 @RunWith(PowerMockRunner.class)
 @PrepareForTest(OozieResourceSensitivityDataJoiner.class)



[02/52] [abbrv] incubator-eagle git commit: [EAGLE-461] Convert MR history app with new app framework

Posted by yo...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/0bde482b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JHFMRVer2EventReader.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JHFMRVer2EventReader.java b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JHFMRVer2EventReader.java
index 2d960b0..4ca0449 100644
--- a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JHFMRVer2EventReader.java
+++ b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JHFMRVer2EventReader.java
@@ -19,8 +19,8 @@
 package org.apache.eagle.jpm.mr.history.parser;
 
 import org.apache.eagle.jpm.mr.history.crawler.JobHistoryContentFilter;
-import org.apache.eagle.jpm.util.jobcounter.JobCounters;
 import org.apache.eagle.jpm.mr.history.parser.JHFMRVer1Parser.Keys;
+import org.apache.eagle.jpm.util.jobcounter.JobCounters;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
@@ -41,6 +41,7 @@ public class JHFMRVer2EventReader extends JHFEventReaderBase {
 
     /**
      * Create a new Event Reader
+     *
      * @throws IOException
      */
     public JHFMRVer2EventReader(Map<String, String> baseTags, Configuration configuration, JobHistoryContentFilter filter) {
@@ -50,86 +51,115 @@ public class JHFMRVer2EventReader extends JHFEventReaderBase {
     @SuppressWarnings("deprecation")
     public void handleEvent(Event wrapper) throws Exception {
         switch (wrapper.type) {
-        case JOB_SUBMITTED:
-            handleJobSubmitted(wrapper); break;
-        case JOB_INITED:
-            handleJobInited(wrapper); break;
-        case JOB_FINISHED:
-            handleJobFinished(wrapper); break;
-        case JOB_PRIORITY_CHANGED:
-            handleJobPriorityChanged(); break;
-        case JOB_STATUS_CHANGED:
-            handleJobStatusChanged(); break;
-        case JOB_FAILED:
-        case JOB_KILLED:
-        case JOB_ERROR:
-            handleJobUnsuccessfulCompletion(wrapper); break;
-        case JOB_INFO_CHANGED:
-            handleJobInfoChanged(); break;
-        case JOB_QUEUE_CHANGED:
-            handleJobQueueChanged();break;
-        case TASK_STARTED:
-            handleTaskStarted(wrapper); break;
-        case TASK_FINISHED:
-            handleTaskFinished(wrapper); break;
-        case TASK_FAILED:
-            handleTaskFailed(wrapper); break;
-        case TASK_UPDATED:
-            handleTaskUpdated(); break;
-
-        // map task
-        case MAP_ATTEMPT_STARTED:
-            handleMapAttemptStarted(wrapper); break;
-        case MAP_ATTEMPT_FINISHED:
-            handleMapAttemptFinished(wrapper); break;
-        case MAP_ATTEMPT_FAILED:
-            handleMapAttemptFailed(wrapper); break;
-        case MAP_ATTEMPT_KILLED:
-            handleMapAttemptKilled(wrapper); break;
-
-        // reduce task
-        case REDUCE_ATTEMPT_STARTED:
-            handleReduceAttemptStarted(wrapper); break;
-        case REDUCE_ATTEMPT_FINISHED:
-            handleReduceAttemptFinished(wrapper); break;
-        case REDUCE_ATTEMPT_FAILED:
-            handleReduceAttemptFailed(wrapper); break;
-        case REDUCE_ATTEMPT_KILLED:
-            handleReduceAttemptKilled(wrapper); break;
-
-        // set up task
-        case SETUP_ATTEMPT_STARTED:
-            break;
-        case SETUP_ATTEMPT_FINISHED:
-            handleSetupAttemptFinished(); break;
-        case SETUP_ATTEMPT_FAILED:
-            handleSetupAttemptFailed(); break;
-        case SETUP_ATTEMPT_KILLED:
-            handleSetupAttemptKilled(); break;
-
-        // clean up task
-        case CLEANUP_ATTEMPT_STARTED:
-            break;
-        case CLEANUP_ATTEMPT_FINISHED:
-            handleCleanupAttemptFinished(); break;
-        case CLEANUP_ATTEMPT_FAILED:
-            handleCleanupAttemptFailed(); break;
-        case CLEANUP_ATTEMPT_KILLED:
-            handleCleanupAttemptKilled(); break;
-
-        case AM_STARTED:
-            handleAMStarted(); break;
-        default:
-            logger.warn("unexpected event type: " + wrapper.type);
+            case JOB_SUBMITTED:
+                handleJobSubmitted(wrapper);
+                break;
+            case JOB_INITED:
+                handleJobInited(wrapper);
+                break;
+            case JOB_FINISHED:
+                handleJobFinished(wrapper);
+                break;
+            case JOB_PRIORITY_CHANGED:
+                handleJobPriorityChanged();
+                break;
+            case JOB_STATUS_CHANGED:
+                handleJobStatusChanged();
+                break;
+            case JOB_FAILED:
+            case JOB_KILLED:
+            case JOB_ERROR:
+                handleJobUnsuccessfulCompletion(wrapper);
+                break;
+            case JOB_INFO_CHANGED:
+                handleJobInfoChanged();
+                break;
+            case JOB_QUEUE_CHANGED:
+                handleJobQueueChanged();
+                break;
+            case TASK_STARTED:
+                handleTaskStarted(wrapper);
+                break;
+            case TASK_FINISHED:
+                handleTaskFinished(wrapper);
+                break;
+            case TASK_FAILED:
+                handleTaskFailed(wrapper);
+                break;
+            case TASK_UPDATED:
+                handleTaskUpdated();
+                break;
+
+            // map task
+            case MAP_ATTEMPT_STARTED:
+                handleMapAttemptStarted(wrapper);
+                break;
+            case MAP_ATTEMPT_FINISHED:
+                handleMapAttemptFinished(wrapper);
+                break;
+            case MAP_ATTEMPT_FAILED:
+                handleMapAttemptFailed(wrapper);
+                break;
+            case MAP_ATTEMPT_KILLED:
+                handleMapAttemptKilled(wrapper);
+                break;
+
+            // reduce task
+            case REDUCE_ATTEMPT_STARTED:
+                handleReduceAttemptStarted(wrapper);
+                break;
+            case REDUCE_ATTEMPT_FINISHED:
+                handleReduceAttemptFinished(wrapper);
+                break;
+            case REDUCE_ATTEMPT_FAILED:
+                handleReduceAttemptFailed(wrapper);
+                break;
+            case REDUCE_ATTEMPT_KILLED:
+                handleReduceAttemptKilled(wrapper);
+                break;
+
+            // set up task
+            case SETUP_ATTEMPT_STARTED:
+                break;
+            case SETUP_ATTEMPT_FINISHED:
+                handleSetupAttemptFinished();
+                break;
+            case SETUP_ATTEMPT_FAILED:
+                handleSetupAttemptFailed();
+                break;
+            case SETUP_ATTEMPT_KILLED:
+                handleSetupAttemptKilled();
+                break;
+
+            // clean up task
+            case CLEANUP_ATTEMPT_STARTED:
+                break;
+            case CLEANUP_ATTEMPT_FINISHED:
+                handleCleanupAttemptFinished();
+                break;
+            case CLEANUP_ATTEMPT_FAILED:
+                handleCleanupAttemptFailed();
+                break;
+            case CLEANUP_ATTEMPT_KILLED:
+                handleCleanupAttemptKilled();
+                break;
+
+            case AM_STARTED:
+                handleAMStarted();
+                break;
+            default:
+                logger.warn("unexpected event type: " + wrapper.type);
         }
     }
 
     private void handleJobPriorityChanged() throws Exception {
         return;
     }
+
     private void handleJobStatusChanged() throws Exception {
         return;
     }
+
     private void handleJobInfoChanged() throws Exception {
         return;
     }
@@ -139,150 +169,285 @@ public class JHFMRVer2EventReader extends JHFEventReaderBase {
     }
 
     private void handleJobSubmitted(Event wrapper) throws Exception {
-        JobSubmitted js = ((JobSubmitted)wrapper.getEvent());
+        JobSubmitted js = ((JobSubmitted) wrapper.getEvent());
         Map<Keys, String> values = new HashMap<>();
-        if (js.getJobid() != null) values.put(Keys.JOBID, js.getJobid().toString());
-        if (js.getJobName() != null) values.put(Keys.JOBNAME, js.getJobName().toString());
-        if (js.getUserName() != null) values.put(Keys.USER, js.getUserName().toString());
-        if (js.getSubmitTime() != null) values.put(Keys.SUBMIT_TIME, js.getSubmitTime().toString());
-        if (js.getJobQueueName() != null) values.put(Keys.JOB_QUEUE, js.getJobQueueName().toString());
+        if (js.getJobid() != null) {
+            values.put(Keys.JOBID, js.getJobid().toString());
+        }
+        if (js.getJobName() != null) {
+            values.put(Keys.JOBNAME, js.getJobName().toString());
+        }
+        if (js.getUserName() != null) {
+            values.put(Keys.USER, js.getUserName().toString());
+        }
+        if (js.getSubmitTime() != null) {
+            values.put(Keys.SUBMIT_TIME, js.getSubmitTime().toString());
+        }
+        if (js.getJobQueueName() != null) {
+            values.put(Keys.JOB_QUEUE, js.getJobQueueName().toString());
+        }
         handleJob(wrapper.getType(), values, null);
     }
 
     private void handleJobInited(Event wrapper) throws Exception {
-        JobInited js = ((JobInited)wrapper.getEvent());
+        JobInited js = ((JobInited) wrapper.getEvent());
         Map<Keys, String> values = new HashMap<>();
-        if (js.getJobid() != null) values.put(Keys.JOBID, js.getJobid().toString());
-        if (js.getLaunchTime() != null) values.put(Keys.LAUNCH_TIME, js.getLaunchTime().toString());
-        if (js.getTotalMaps() != null) values.put(Keys.TOTAL_MAPS, js.getTotalMaps().toString());
-        if (js.getTotalReduces() != null) values.put(Keys.TOTAL_REDUCES, js.getTotalReduces().toString());
-        if (js.getJobStatus() != null) values.put(Keys.JOB_STATUS, js.getJobStatus().toString());
-        if (js.getUberized() != null) values.put(Keys.UBERISED, js.getUberized().toString());
+        if (js.getJobid() != null) {
+            values.put(Keys.JOBID, js.getJobid().toString());
+        }
+        if (js.getLaunchTime() != null) {
+            values.put(Keys.LAUNCH_TIME, js.getLaunchTime().toString());
+        }
+        if (js.getTotalMaps() != null) {
+            values.put(Keys.TOTAL_MAPS, js.getTotalMaps().toString());
+        }
+        if (js.getTotalReduces() != null) {
+            values.put(Keys.TOTAL_REDUCES, js.getTotalReduces().toString());
+        }
+        if (js.getJobStatus() != null) {
+            values.put(Keys.JOB_STATUS, js.getJobStatus().toString());
+        }
+        if (js.getUberized() != null) {
+            values.put(Keys.UBERISED, js.getUberized().toString());
+        }
         handleJob(wrapper.getType(), values, null);
     }
 
     private void handleJobFinished(Event wrapper) throws Exception {
-        JobFinished js = ((JobFinished)wrapper.getEvent());
+        JobFinished js = ((JobFinished) wrapper.getEvent());
         Map<Keys, String> values = new HashMap<>();
-        if (js.getJobid() != null) values.put(Keys.JOBID, js.getJobid().toString());
-        if (js.getFinishTime() != null) values.put(Keys.FINISH_TIME, js.getFinishTime().toString());
-        if (js.getFinishedMaps() != null) values.put(Keys.FINISHED_MAPS, js.getFinishedMaps().toString());
-        if (js.getFinishedReduces() != null) values.put(Keys.FINISHED_REDUCES, js.getFinishedReduces().toString());
-        if (js.getFailedMaps() != null) values.put(Keys.FAILED_MAPS, js.getFailedMaps().toString());
-        if (js.getFailedReduces() != null) values.put(Keys.FAILED_REDUCES, js.getFailedReduces().toString());
+        if (js.getJobid() != null) {
+            values.put(Keys.JOBID, js.getJobid().toString());
+        }
+        if (js.getFinishTime() != null) {
+            values.put(Keys.FINISH_TIME, js.getFinishTime().toString());
+        }
+        if (js.getFinishedMaps() != null) {
+            values.put(Keys.FINISHED_MAPS, js.getFinishedMaps().toString());
+        }
+        if (js.getFinishedReduces() != null) {
+            values.put(Keys.FINISHED_REDUCES, js.getFinishedReduces().toString());
+        }
+        if (js.getFailedMaps() != null) {
+            values.put(Keys.FAILED_MAPS, js.getFailedMaps().toString());
+        }
+        if (js.getFailedReduces() != null) {
+            values.put(Keys.FAILED_REDUCES, js.getFailedReduces().toString());
+        }
         values.put(Keys.JOB_STATUS, EagleJobStatus.SUCCESS.name());
         handleJob(wrapper.getType(), values, js.getTotalCounters());
     }
 
     private void handleJobUnsuccessfulCompletion(Event wrapper) throws Exception {
-        JobUnsuccessfulCompletion js = ((JobUnsuccessfulCompletion)wrapper.getEvent());
+        JobUnsuccessfulCompletion js = ((JobUnsuccessfulCompletion) wrapper.getEvent());
         Map<Keys, String> values = new HashMap<>();
-        if (js.getJobid() != null) values.put(Keys.JOBID, js.getJobid().toString());
-        if (js.getFinishTime() != null) values.put(Keys.FINISH_TIME, js.getFinishTime().toString());
-        if (js.getFinishedMaps() != null) values.put(Keys.FINISHED_MAPS, js.getFinishedMaps().toString());
-        if (js.getFinishedReduces() != null) values.put(Keys.FINISHED_REDUCES, js.getFinishedReduces().toString());
-        if (js.getJobStatus() != null) values.put(Keys.JOB_STATUS, js.getJobStatus().toString());
+        if (js.getJobid() != null) {
+            values.put(Keys.JOBID, js.getJobid().toString());
+        }
+        if (js.getFinishTime() != null) {
+            values.put(Keys.FINISH_TIME, js.getFinishTime().toString());
+        }
+        if (js.getFinishedMaps() != null) {
+            values.put(Keys.FINISHED_MAPS, js.getFinishedMaps().toString());
+        }
+        if (js.getFinishedReduces() != null) {
+            values.put(Keys.FINISHED_REDUCES, js.getFinishedReduces().toString());
+        }
+        if (js.getJobStatus() != null) {
+            values.put(Keys.JOB_STATUS, js.getJobStatus().toString());
+        }
         handleJob(wrapper.getType(), values, null);
     }
 
     private void handleTaskStarted(Event wrapper) throws Exception {
-        TaskStarted js = ((TaskStarted)wrapper.getEvent());
+        TaskStarted js = ((TaskStarted) wrapper.getEvent());
         Map<Keys, String> values = new HashMap<>();
-        if (js.getTaskid() != null) values.put(Keys.TASKID, js.getTaskid().toString());
-        if (js.getTaskType() != null) values.put(Keys.TASK_TYPE, js.getTaskType().toString());
-        if (js.getStartTime() != null) values.put(Keys.START_TIME, js.getStartTime().toString());
-        if (js.getSplitLocations() != null) values.put(Keys.SPLIT_LOCATIONS, js.getSplitLocations().toString());
+        if (js.getTaskid() != null) {
+            values.put(Keys.TASKID, js.getTaskid().toString());
+        }
+        if (js.getTaskType() != null) {
+            values.put(Keys.TASK_TYPE, js.getTaskType().toString());
+        }
+        if (js.getStartTime() != null) {
+            values.put(Keys.START_TIME, js.getStartTime().toString());
+        }
+        if (js.getSplitLocations() != null) {
+            values.put(Keys.SPLIT_LOCATIONS, js.getSplitLocations().toString());
+        }
         handleTask(RecordTypes.Task, wrapper.getType(), values, null);
     }
 
     private void handleTaskFinished(Event wrapper) throws Exception {
-        TaskFinished js = ((TaskFinished)wrapper.getEvent());
+        TaskFinished js = ((TaskFinished) wrapper.getEvent());
         Map<Keys, String> values = new HashMap<>();
-        if (js.getTaskid() != null) values.put(Keys.TASKID, js.getTaskid().toString());
-        if (js.getTaskType() != null) values.put(Keys.TASK_TYPE, js.getTaskType().toString());
-        if (js.getFinishTime() != null) values.put(Keys.FINISH_TIME, js.getFinishTime().toString());
-        if (js.getStatus() != null) values.put(Keys.TASK_STATUS, normalizeTaskStatus(js.getStatus().toString()));
+        if (js.getTaskid() != null) {
+            values.put(Keys.TASKID, js.getTaskid().toString());
+        }
+        if (js.getTaskType() != null) {
+            values.put(Keys.TASK_TYPE, js.getTaskType().toString());
+        }
+        if (js.getFinishTime() != null) {
+            values.put(Keys.FINISH_TIME, js.getFinishTime().toString());
+        }
+        if (js.getStatus() != null) {
+            values.put(Keys.TASK_STATUS, normalizeTaskStatus(js.getStatus().toString()));
+        }
         handleTask(RecordTypes.Task, wrapper.getType(), values, js.getCounters());
     }
 
     private void handleTaskFailed(Event wrapper) throws Exception {
-        TaskFailed js = ((TaskFailed)wrapper.getEvent());
+        TaskFailed js = ((TaskFailed) wrapper.getEvent());
         Map<Keys, String> values = new HashMap<>();
 
-        if (js.getTaskid() != null) values.put(Keys.TASKID, js.getTaskid().toString());
-        if (js.getTaskType() != null) values.put(Keys.TASK_TYPE, js.getTaskType().toString());
-        if (js.getFinishTime() != null) values.put(Keys.FINISH_TIME, js.getFinishTime().toString());
-        if (js.getStatus() != null) values.put(Keys.TASK_STATUS, normalizeTaskStatus(js.getStatus().toString()));
-        if (js.getError() != null) values.put(Keys.ERROR, js.getError().toString());
-        if (js.getFailedDueToAttempt() != null) values.put(Keys.FAILED_DUE_TO_ATTEMPT, js.getFailedDueToAttempt().toString());
+        if (js.getTaskid() != null) {
+            values.put(Keys.TASKID, js.getTaskid().toString());
+        }
+        if (js.getTaskType() != null) {
+            values.put(Keys.TASK_TYPE, js.getTaskType().toString());
+        }
+        if (js.getFinishTime() != null) {
+            values.put(Keys.FINISH_TIME, js.getFinishTime().toString());
+        }
+        if (js.getStatus() != null) {
+            values.put(Keys.TASK_STATUS, normalizeTaskStatus(js.getStatus().toString()));
+        }
+        if (js.getError() != null) {
+            values.put(Keys.ERROR, js.getError().toString());
+        }
+        if (js.getFailedDueToAttempt() != null) {
+            values.put(Keys.FAILED_DUE_TO_ATTEMPT, js.getFailedDueToAttempt().toString());
+        }
         handleTask(RecordTypes.Task, wrapper.getType(), values, js.getCounters());
     }
 
     private String normalizeTaskStatus(String taskStatus) {
-        if (taskStatus.equals("SUCCEEDED"))
+        if (taskStatus.equals("SUCCEEDED")) {
             return EagleTaskStatus.SUCCESS.name();
+        }
         return taskStatus;
     }
 
-    private void handleTaskUpdated(){
+    private void handleTaskUpdated() {
         return;
     }
 
     private void handleMapAttemptStarted(Event wrapper) throws Exception {
         handleTaskAttemptStarted(wrapper, RecordTypes.MapAttempt);
     }
+
     private void handleReduceAttemptStarted(Event wrapper) throws Exception {
         handleTaskAttemptStarted(wrapper, RecordTypes.ReduceAttempt);
     }
 
     private void handleTaskAttemptStarted(Event wrapper, RecordTypes recordType) throws Exception {
-        TaskAttemptStarted js = ((TaskAttemptStarted)wrapper.getEvent());
+        TaskAttemptStarted js = ((TaskAttemptStarted) wrapper.getEvent());
         Map<Keys, String> values = new HashMap<>();
-        if (js.getTaskid() != null) values.put(Keys.TASKID, js.getTaskid().toString());
-        if (js.getTaskType() != null) values.put(Keys.TASK_TYPE, js.getTaskType().toString());
-        if (js.getAttemptId() != null) values.put(Keys.TASK_ATTEMPT_ID, js.getAttemptId().toString());
-        if (js.getStartTime() != null) values.put(Keys.START_TIME, js.getStartTime().toString());
-        if (js.getTrackerName() != null) values.put(Keys.TRACKER_NAME, js.getTrackerName().toString());
-        if (js.getHttpPort() != null) values.put(Keys.HTTP_PORT, js.getHttpPort().toString());
-        if (js.getShufflePort() != null) values.put(Keys.SHUFFLE_PORT, js.getShufflePort().toString());
-        if (js.getLocality() != null) values.put(Keys.LOCALITY, js.getLocality().toString());
-        if (js.getAvataar() != null) values.put(Keys.AVATAAR, js.getAvataar().toString());
-        handleTask(recordType,wrapper.getType(), values, null);
+        if (js.getTaskid() != null) {
+            values.put(Keys.TASKID, js.getTaskid().toString());
+        }
+        if (js.getTaskType() != null) {
+            values.put(Keys.TASK_TYPE, js.getTaskType().toString());
+        }
+        if (js.getAttemptId() != null) {
+            values.put(Keys.TASK_ATTEMPT_ID, js.getAttemptId().toString());
+        }
+        if (js.getStartTime() != null) {
+            values.put(Keys.START_TIME, js.getStartTime().toString());
+        }
+        if (js.getTrackerName() != null) {
+            values.put(Keys.TRACKER_NAME, js.getTrackerName().toString());
+        }
+        if (js.getHttpPort() != null) {
+            values.put(Keys.HTTP_PORT, js.getHttpPort().toString());
+        }
+        if (js.getShufflePort() != null) {
+            values.put(Keys.SHUFFLE_PORT, js.getShufflePort().toString());
+        }
+        if (js.getLocality() != null) {
+            values.put(Keys.LOCALITY, js.getLocality().toString());
+        }
+        if (js.getAvataar() != null) {
+            values.put(Keys.AVATAAR, js.getAvataar().toString());
+        }
+        handleTask(recordType, wrapper.getType(), values, null);
     }
 
     private void handleMapAttemptFinished(Event wrapper) throws Exception {
-        MapAttemptFinished js = ((MapAttemptFinished)wrapper.getEvent());
+        MapAttemptFinished js = ((MapAttemptFinished) wrapper.getEvent());
         Map<Keys, String> values = new HashMap<>();
-        if (js.getTaskid() != null) values.put(Keys.TASKID, js.getTaskid().toString());
-        if (js.getTaskType() != null) values.put(Keys.TASK_TYPE, js.getTaskType().toString());
-        if (js.getTaskStatus() != null) values.put(Keys.TASK_STATUS, normalizeTaskStatus(js.getTaskStatus().toString()));
-        if (js.getAttemptId() != null) values.put(Keys.TASK_ATTEMPT_ID, js.getAttemptId().toString());
-        if (js.getFinishTime() != null) values.put(Keys.FINISH_TIME, js.getFinishTime().toString());
-        if (js.getMapFinishTime() != null) values.put(Keys.MAP_FINISH_TIME, js.getMapFinishTime().toString());
-        if (js.getHostname() != null) values.put(Keys.HOSTNAME, js.getHostname().toString());
-        if (js.getPort() != null) values.put(Keys.PORT, js.getPort().toString());
-        if (js.getRackname() != null) values.put(Keys.RACK_NAME, js.getRackname().toString());
-        if (js.getState() != null) values.put(Keys.STATE_STRING, js.getState().toString());
+        if (js.getTaskid() != null) {
+            values.put(Keys.TASKID, js.getTaskid().toString());
+        }
+        if (js.getTaskType() != null) {
+            values.put(Keys.TASK_TYPE, js.getTaskType().toString());
+        }
+        if (js.getTaskStatus() != null) {
+            values.put(Keys.TASK_STATUS, normalizeTaskStatus(js.getTaskStatus().toString()));
+        }
+        if (js.getAttemptId() != null) {
+            values.put(Keys.TASK_ATTEMPT_ID, js.getAttemptId().toString());
+        }
+        if (js.getFinishTime() != null) {
+            values.put(Keys.FINISH_TIME, js.getFinishTime().toString());
+        }
+        if (js.getMapFinishTime() != null) {
+            values.put(Keys.MAP_FINISH_TIME, js.getMapFinishTime().toString());
+        }
+        if (js.getHostname() != null) {
+            values.put(Keys.HOSTNAME, js.getHostname().toString());
+        }
+        if (js.getPort() != null) {
+            values.put(Keys.PORT, js.getPort().toString());
+        }
+        if (js.getRackname() != null) {
+            values.put(Keys.RACK_NAME, js.getRackname().toString());
+        }
+        if (js.getState() != null) {
+            values.put(Keys.STATE_STRING, js.getState().toString());
+        }
 
         ensureRackAfterAttemptFinish(js.getRackname().toString(), values);
-        handleTask(RecordTypes.MapAttempt,wrapper.getType(), values, js.getCounters());
+        handleTask(RecordTypes.MapAttempt, wrapper.getType(), values, js.getCounters());
     }
+
     private void handleReduceAttemptFinished(Event wrapper) throws Exception {
-        ReduceAttemptFinished js = ((ReduceAttemptFinished)wrapper.getEvent());
+        ReduceAttemptFinished js = ((ReduceAttemptFinished) wrapper.getEvent());
         Map<Keys, String> values = new HashMap<>();
-        if (js.getTaskid() != null) values.put(Keys.TASKID, js.getTaskid().toString());
-        if (js.getTaskType() != null) values.put(Keys.TASK_TYPE, js.getTaskType().toString());
-        if (js.getTaskStatus() != null) values.put(Keys.TASK_STATUS, normalizeTaskStatus(js.getTaskStatus().toString()));
-        if (js.getAttemptId() != null) values.put(Keys.TASK_ATTEMPT_ID, js.getAttemptId().toString());
-        if (js.getFinishTime() != null) values.put(Keys.FINISH_TIME, js.getFinishTime().toString());
-        if (js.getShuffleFinishTime() != null) values.put(Keys.SHUFFLE_FINISHED, js.getShuffleFinishTime().toString());
-        if (js.getSortFinishTime() != null) values.put(Keys.SORT_FINISHED, js.getSortFinishTime().toString());
-        if (js.getHostname() != null) values.put(Keys.HOSTNAME, js.getHostname().toString());
-        if (js.getPort() != null) values.put(Keys.PORT, js.getPort().toString());
-        if (js.getRackname() != null) values.put(Keys.RACK_NAME, js.getRackname().toString());
-        if (js.getState() != null) values.put(Keys.STATE_STRING, js.getState().toString());
+        if (js.getTaskid() != null) {
+            values.put(Keys.TASKID, js.getTaskid().toString());
+        }
+        if (js.getTaskType() != null) {
+            values.put(Keys.TASK_TYPE, js.getTaskType().toString());
+        }
+        if (js.getTaskStatus() != null) {
+            values.put(Keys.TASK_STATUS, normalizeTaskStatus(js.getTaskStatus().toString()));
+        }
+        if (js.getAttemptId() != null) {
+            values.put(Keys.TASK_ATTEMPT_ID, js.getAttemptId().toString());
+        }
+        if (js.getFinishTime() != null) {
+            values.put(Keys.FINISH_TIME, js.getFinishTime().toString());
+        }
+        if (js.getShuffleFinishTime() != null) {
+            values.put(Keys.SHUFFLE_FINISHED, js.getShuffleFinishTime().toString());
+        }
+        if (js.getSortFinishTime() != null) {
+            values.put(Keys.SORT_FINISHED, js.getSortFinishTime().toString());
+        }
+        if (js.getHostname() != null) {
+            values.put(Keys.HOSTNAME, js.getHostname().toString());
+        }
+        if (js.getPort() != null) {
+            values.put(Keys.PORT, js.getPort().toString());
+        }
+        if (js.getRackname() != null) {
+            values.put(Keys.RACK_NAME, js.getRackname().toString());
+        }
+        if (js.getState() != null) {
+            values.put(Keys.STATE_STRING, js.getState().toString());
+        }
         ensureRackAfterAttemptFinish(js.getRackname().toString(), values);
-        handleTask(RecordTypes.ReduceAttempt,wrapper.getType(), values, js.getCounters());
+        handleTask(RecordTypes.ReduceAttempt, wrapper.getType(), values, js.getCounters());
     }
 
     private void ensureRackAfterAttemptFinish(String rackname, Map<Keys, String> values) {
@@ -296,61 +461,89 @@ public class JHFMRVer2EventReader extends JHFEventReaderBase {
     private void handleMapAttemptFailed(Event wrapper) throws Exception {
         handleTaskAttemptFailed(wrapper, RecordTypes.MapAttempt);
     }
+
     private void handleReduceAttemptFailed(Event wrapper) throws Exception {
         handleTaskAttemptFailed(wrapper, RecordTypes.ReduceAttempt);
     }
-    private void handleMapAttemptKilled(Event wrapper)throws Exception {
+
+    private void handleMapAttemptKilled(Event wrapper) throws Exception {
         handleTaskAttemptFailed(wrapper, RecordTypes.MapAttempt);
     }
-    private void handleReduceAttemptKilled(Event wrapper)throws Exception {
+
+    private void handleReduceAttemptKilled(Event wrapper) throws Exception {
         handleTaskAttemptFailed(wrapper, RecordTypes.ReduceAttempt);
     }
-    
+
     private void handleTaskAttemptFailed(Event wrapper, RecordTypes recordType) throws Exception {
-        TaskAttemptUnsuccessfulCompletion js = ((TaskAttemptUnsuccessfulCompletion)wrapper.getEvent());
+        TaskAttemptUnsuccessfulCompletion js = ((TaskAttemptUnsuccessfulCompletion) wrapper.getEvent());
         Map<Keys, String> values = new HashMap<>();
-        if (js.getTaskid() != null) values.put(Keys.TASKID, js.getTaskid().toString());
-        if (js.getTaskType() != null) values.put(Keys.TASK_TYPE, js.getTaskType().toString());
-        if (js.getAttemptId() != null) values.put(Keys.TASK_ATTEMPT_ID, js.getAttemptId().toString());
-        if (js.getFinishTime() != null) values.put(Keys.FINISH_TIME, js.getFinishTime().toString());
-        if (js.getHostname() != null) values.put(Keys.HOSTNAME, js.getHostname().toString());
-        if (js.getPort() != null) values.put(Keys.PORT, js.getPort().toString());
-        if (js.getRackname() != null) values.put(Keys.RACK_NAME, js.getRackname().toString());
-        if (js.getError() != null) values.put(Keys.ERROR, js.getError().toString());
-        if (js.getStatus() != null) values.put(Keys.TASK_STATUS, js.getStatus().toString());
+        if (js.getTaskid() != null) {
+            values.put(Keys.TASKID, js.getTaskid().toString());
+        }
+        if (js.getTaskType() != null) {
+            values.put(Keys.TASK_TYPE, js.getTaskType().toString());
+        }
+        if (js.getAttemptId() != null) {
+            values.put(Keys.TASK_ATTEMPT_ID, js.getAttemptId().toString());
+        }
+        if (js.getFinishTime() != null) {
+            values.put(Keys.FINISH_TIME, js.getFinishTime().toString());
+        }
+        if (js.getHostname() != null) {
+            values.put(Keys.HOSTNAME, js.getHostname().toString());
+        }
+        if (js.getPort() != null) {
+            values.put(Keys.PORT, js.getPort().toString());
+        }
+        if (js.getRackname() != null) {
+            values.put(Keys.RACK_NAME, js.getRackname().toString());
+        }
+        if (js.getError() != null) {
+            values.put(Keys.ERROR, js.getError().toString());
+        }
+        if (js.getStatus() != null) {
+            values.put(Keys.TASK_STATUS, js.getStatus().toString());
+        }
         ensureRackAfterAttemptFinish(js.getRackname().toString(), values);
-        handleTask(recordType,wrapper.getType(), values, js.getCounters());
-    }    
-    private void handleSetupAttemptFinished(){
+        handleTask(recordType, wrapper.getType(), values, js.getCounters());
+    }
+
+    private void handleSetupAttemptFinished() {
         return;
     }
-    private void handleSetupAttemptFailed(){
+
+    private void handleSetupAttemptFailed() {
         return;
     }
-    private void handleSetupAttemptKilled(){
+
+    private void handleSetupAttemptKilled() {
         return;
     }
-    private void handleCleanupAttemptFinished(){
+
+    private void handleCleanupAttemptFinished() {
         return;
     }
-    private void handleCleanupAttemptFailed(){
+
+    private void handleCleanupAttemptFailed() {
         return;
     }
-    private void handleCleanupAttemptKilled(){
+
+    private void handleCleanupAttemptKilled() {
         return;
     }
-    private void handleAMStarted(){
+
+    private void handleAMStarted() {
         return;
     }
 
-    protected JobCounters parseCounters(Object value) throws IOException{
+    protected JobCounters parseCounters(Object value) throws IOException {
         JobCounters jc = new JobCounters();
         Map<String, Map<String, Long>> groups = new HashMap<>();
-        JhCounters counters = (JhCounters)value;
+        JhCounters counters = (JhCounters) value;
         List<JhCounterGroup> list = counters.getGroups();
         for (JhCounterGroup cg : list) {
             String cgName = cg.getName().toString();
-            if(!cgName.equals("org.apache.hadoop.mapreduce.FileSystemCounter")
+            if (!cgName.equals("org.apache.hadoop.mapreduce.FileSystemCounter")
                 && !cgName.equals("org.apache.hadoop.mapreduce.TaskCounter")
                 && !cgName.equals("org.apache.hadoop.mapreduce.JobCounter")
                 && !cgName.equals("org.apache.hadoop.mapreduce.lib.input.FileInputFormatCounter")
@@ -359,7 +552,9 @@ public class JHFMRVer2EventReader extends JHFEventReaderBase {
                 && !cgName.equals("org.apache.hadoop.mapred.Task$Counter")                        // for artemis
                 && !cgName.equals("org.apache.hadoop.mapreduce.lib.input.FileInputFormat$Counter") // for artemis
                 && !cgName.equals("org.apache.hadoop.mapreduce.lib.input.FileOutputFormat$Counter") // for artemis
-            ) continue;
+                ) {
+                continue;
+            }
 
             groups.put(cgName, new HashMap<String, Long>());
             Map<String, Long> counterValues = groups.get(cgName);

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/0bde482b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JHFMRVer2Parser.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JHFMRVer2Parser.java b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JHFMRVer2Parser.java
index c289e4d..2ccbf8d 100644
--- a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JHFMRVer2Parser.java
+++ b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JHFMRVer2Parser.java
@@ -19,8 +19,9 @@
 package org.apache.eagle.jpm.mr.history.parser;
 
 import org.apache.avro.Schema;
-import org.apache.avro.io.*;
-import org.apache.avro.specific.SpecificData;
+import org.apache.avro.io.DatumReader;
+import org.apache.avro.io.DecoderFactory;
+import org.apache.avro.io.JsonDecoder;
 import org.apache.avro.specific.SpecificDatumReader;
 import org.apache.hadoop.mapreduce.jobhistory.Event;
 import org.slf4j.Logger;
@@ -33,35 +34,35 @@ import java.io.InputStream;
 public class JHFMRVer2Parser implements JHFParserBase {
     private static final Logger logger = LoggerFactory.getLogger(JHFMRVer2Parser.class);
     private JHFMRVer2EventReader _reader;
-    
-    public JHFMRVer2Parser(JHFMRVer2EventReader reader){
+
+    public JHFMRVer2Parser(JHFMRVer2EventReader reader) {
         this._reader = reader;
     }
 
-    @SuppressWarnings({ "rawtypes", "deprecation" })
+    @SuppressWarnings( {"rawtypes", "deprecation"})
     @Override
     public void parse(InputStream is) throws Exception {
         int eventCtr = 0;
         try {
-             long start = System.currentTimeMillis();
-             DataInputStream in = new DataInputStream(is);
-             String version = in.readLine();
-             if (!"Avro-Json".equals(version)) {
-                 throw new IOException("Incompatible event log version: " + version);
-             }
-                
-             Schema schema = Schema.parse(in.readLine());
-             SpecificDatumReader datumReader = new SpecificDatumReader(schema);
-             JsonDecoder decoder = DecoderFactory.get().jsonDecoder(schema, in);
+            long start = System.currentTimeMillis();
+            DataInputStream in = new DataInputStream(is);
+            String version = in.readLine();
+            if (!"Avro-Json".equals(version)) {
+                throw new IOException("Incompatible event log version: " + version);
+            }
+
+            Schema schema = Schema.parse(in.readLine());
+            SpecificDatumReader datumReader = new SpecificDatumReader(schema);
+            JsonDecoder decoder = DecoderFactory.get().jsonDecoder(schema, in);
 
-             Event wrapper;
-             while ((wrapper = getNextEvent(datumReader, decoder)) != null) {
-                 ++eventCtr;
-                 _reader.handleEvent(wrapper);
-             }
-             _reader.parseConfiguration();
-             // don't need put to finally as it's a kind of flushing data
-             _reader.close();
+            Event wrapper;
+            while ((wrapper = getNextEvent(datumReader, decoder)) != null) {
+                ++eventCtr;
+                _reader.handleEvent(wrapper);
+            }
+            _reader.parseConfiguration();
+            // don't need put to finally as it's a kind of flushing data
+            _reader.close();
             logger.info("reader used " + (System.currentTimeMillis() - start) + "ms");
         } catch (Exception ioe) {
             logger.error("Caught exception parsing history file after " + eventCtr + " events", ioe);
@@ -71,13 +72,13 @@ public class JHFMRVer2Parser implements JHFParserBase {
                 is.close();
             }
         }
-      }
-    
-    @SuppressWarnings({ "rawtypes", "unchecked" })
+    }
+
+    @SuppressWarnings( {"rawtypes", "unchecked"})
     public Event getNextEvent(DatumReader datumReader, JsonDecoder decoder) throws Exception {
         Event wrapper;
         try {
-            wrapper = (Event)datumReader.read(null, decoder);
+            wrapper = (Event) datumReader.read(null, decoder);
         } catch (java.io.EOFException e) {            // at EOF
             return null;
         }

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/0bde482b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JHFParserBase.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JHFParserBase.java b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JHFParserBase.java
index e4f437f..a2479ab 100644
--- a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JHFParserBase.java
+++ b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JHFParserBase.java
@@ -21,13 +21,12 @@ package org.apache.eagle.jpm.mr.history.parser;
 import java.io.InputStream;
 
 /**
- *
  * @author yonzhang
- *
  */
 public interface JHFParserBase {
     /**
      * this method will ensure to close the inputstream
+     *
      * @param is
      * @throws Exception
      */

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/0bde482b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JHFParserFactory.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JHFParserFactory.java b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JHFParserFactory.java
index dd640c2..718612d 100755
--- a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JHFParserFactory.java
+++ b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JHFParserFactory.java
@@ -18,7 +18,7 @@
 
 package org.apache.eagle.jpm.mr.history.parser;
 
-import org.apache.eagle.jpm.mr.history.common.JHFConfigManager;
+import org.apache.eagle.jpm.mr.history.MRHistoryJobConfig;
 import org.apache.eagle.jpm.mr.history.crawler.JobHistoryContentFilter;
 import org.apache.hadoop.conf.Configuration;
 import org.slf4j.Logger;
@@ -30,7 +30,7 @@ public class JHFParserFactory {
 
     private static final Logger LOG = LoggerFactory.getLogger(JHFParserFactory.class);
 
-    public static JHFParserBase getParser(JHFConfigManager configManager, Map<String, String> baseTags, Configuration configuration, JobHistoryContentFilter filter) {
+    public static JHFParserBase getParser(MRHistoryJobConfig configManager, Map<String, String> baseTags, Configuration configuration, JobHistoryContentFilter filter) {
         String format = configManager.getJobExtractorConfig().mrVersion;
         JHFParserBase parser;
         JHFFormat f;
@@ -40,31 +40,31 @@ public class JHFParserFactory {
             } else {
                 f = JHFFormat.valueOf(format);
             }
-        } catch(IllegalArgumentException ex) {
+        } catch (IllegalArgumentException ex) {
             f = JHFFormat.MRVer1; // fall back to version 1 unless it's specified as version 2
         }
-        
+
         switch (f) {
-        case MRVer2:
-            JHFMRVer2EventReader reader2 = new JHFMRVer2EventReader(baseTags, configuration, filter);
-            reader2.addListener(new JobEntityCreationEagleServiceListener(configManager));
-            reader2.addListener(new TaskFailureListener(configManager));
-            reader2.addListener(new TaskAttemptCounterListener(configManager));
-            reader2.addListener(new JobConfigurationCreationServiceListener(configManager));
+            case MRVer2:
+                JHFMRVer2EventReader reader2 = new JHFMRVer2EventReader(baseTags, configuration, filter);
+                reader2.addListener(new JobEntityCreationEagleServiceListener(configManager));
+                reader2.addListener(new TaskFailureListener(configManager));
+                reader2.addListener(new TaskAttemptCounterListener(configManager));
+                reader2.addListener(new JobConfigurationCreationServiceListener(configManager));
 
-            reader2.register(new JobEntityLifecycleAggregator());
-            parser = new JHFMRVer2Parser(reader2);
-            break;
-        case MRVer1:
-        default:
-            JHFMRVer1EventReader reader1 = new JHFMRVer1EventReader(baseTags, configuration, filter);
-            reader1.addListener(new JobEntityCreationEagleServiceListener(configManager));
-            reader1.addListener(new TaskFailureListener(configManager));
-            reader1.addListener(new TaskAttemptCounterListener(configManager));
+                reader2.register(new JobEntityLifecycleAggregator());
+                parser = new JHFMRVer2Parser(reader2);
+                break;
+            case MRVer1:
+            default:
+                JHFMRVer1EventReader reader1 = new JHFMRVer1EventReader(baseTags, configuration, filter);
+                reader1.addListener(new JobEntityCreationEagleServiceListener(configManager));
+                reader1.addListener(new TaskFailureListener(configManager));
+                reader1.addListener(new TaskAttemptCounterListener(configManager));
 
-            reader1.register(new JobEntityLifecycleAggregator());
-            parser = new JHFMRVer1Parser(reader1);
-            break;
+                reader1.register(new JobEntityLifecycleAggregator());
+                parser = new JHFMRVer1Parser(reader1);
+                break;
         }
         return parser;
     }

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/0bde482b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JHFWriteNotCompletedException.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JHFWriteNotCompletedException.java b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JHFWriteNotCompletedException.java
index 4529f8b..d5ed97a 100644
--- a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JHFWriteNotCompletedException.java
+++ b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JHFWriteNotCompletedException.java
@@ -21,16 +21,16 @@ package org.apache.eagle.jpm.mr.history.parser;
 /**
  * used to warn that one job history file has not yet completed writing to hdfs
  * This happens when feeder catches up and the history file has not been written into hdfs completely
- * @author yonzhang
  *
+ * @author yonzhang
  */
 public class JHFWriteNotCompletedException extends Exception {
     /**
-     * 
+     *
      */
     private static final long serialVersionUID = -3060175780718218490L;
 
-    public JHFWriteNotCompletedException(String msg){
+    public JHFWriteNotCompletedException(String msg) {
         super(msg);
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/0bde482b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JobConfigurationCreationServiceListener.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JobConfigurationCreationServiceListener.java b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JobConfigurationCreationServiceListener.java
index 0334c9b..4163f7b 100644
--- a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JobConfigurationCreationServiceListener.java
+++ b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JobConfigurationCreationServiceListener.java
@@ -18,7 +18,7 @@
 
 package org.apache.eagle.jpm.mr.history.parser;
 
-import org.apache.eagle.jpm.mr.history.common.JHFConfigManager;
+import org.apache.eagle.jpm.mr.history.MRHistoryJobConfig;
 import org.apache.eagle.jpm.mr.historyentity.JobBaseAPIEntity;
 import org.apache.eagle.jpm.mr.historyentity.JobConfigurationAPIEntity;
 import org.apache.eagle.service.client.IEagleServiceClient;
@@ -32,10 +32,10 @@ import java.util.List;
 public class JobConfigurationCreationServiceListener implements HistoryJobEntityLifecycleListener {
     private static final Logger logger = LoggerFactory.getLogger(JobConfigurationCreationServiceListener.class);
     private static final int MAX_RETRY_TIMES = 3;
-    private JHFConfigManager configManager;
+    private MRHistoryJobConfig configManager;
     private JobConfigurationAPIEntity m_jobConfigurationEntity;
 
-    public JobConfigurationCreationServiceListener(JHFConfigManager configManager) {
+    public JobConfigurationCreationServiceListener(MRHistoryJobConfig configManager) {
         this.configManager = configManager;
     }
 
@@ -43,7 +43,7 @@ public class JobConfigurationCreationServiceListener implements HistoryJobEntity
     public void jobEntityCreated(JobBaseAPIEntity entity) throws Exception {
         if (entity != null) {
             if (entity instanceof JobConfigurationAPIEntity) {
-                this.m_jobConfigurationEntity = (JobConfigurationAPIEntity)entity;
+                this.m_jobConfigurationEntity = (JobConfigurationAPIEntity) entity;
             }
         }
     }
@@ -55,13 +55,13 @@ public class JobConfigurationCreationServiceListener implements HistoryJobEntity
 
     @Override
     public void flush() throws Exception {
-        JHFConfigManager.EagleServiceConfig eagleServiceConfig = configManager.getEagleServiceConfig();
-        JHFConfigManager.JobExtractorConfig jobExtractorConfig = configManager.getJobExtractorConfig();
+        MRHistoryJobConfig.EagleServiceConfig eagleServiceConfig = configManager.getEagleServiceConfig();
+        MRHistoryJobConfig.JobExtractorConfig jobExtractorConfig = configManager.getJobExtractorConfig();
         IEagleServiceClient client = new EagleServiceClientImpl(
-                eagleServiceConfig.eagleServiceHost,
-                eagleServiceConfig.eagleServicePort,
-                eagleServiceConfig.username,
-                eagleServiceConfig.password);
+            eagleServiceConfig.eagleServiceHost,
+            eagleServiceConfig.eagleServicePort,
+            eagleServiceConfig.username,
+            eagleServiceConfig.password);
 
         client.getJerseyClient().setReadTimeout(jobExtractorConfig.readTimeoutSeconds * 1000);
         List<JobConfigurationAPIEntity> list = new ArrayList<>();
@@ -76,7 +76,7 @@ public class JobConfigurationCreationServiceListener implements HistoryJobEntity
                 break;
             } catch (Exception ex) {
                 if (tried < MAX_RETRY_TIMES) {
-                    logger.error("Got exception to flush, retry as " + (tried + 1) +" times",ex);
+                    logger.error("Got exception to flush, retry as " + (tried + 1) + " times", ex);
                 } else {
                     logger.error("Got exception to flush, reach max retry times " + MAX_RETRY_TIMES, ex);
                 }
@@ -86,7 +86,7 @@ public class JobConfigurationCreationServiceListener implements HistoryJobEntity
                 client.getJerseyClient().destroy();
                 client.close();
             }
-            tried ++;
+            tried++;
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/0bde482b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JobEntityCreationEagleServiceListener.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JobEntityCreationEagleServiceListener.java b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JobEntityCreationEagleServiceListener.java
index e1a3c69..abddf3b 100644
--- a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JobEntityCreationEagleServiceListener.java
+++ b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JobEntityCreationEagleServiceListener.java
@@ -18,7 +18,7 @@
 
 package org.apache.eagle.jpm.mr.history.parser;
 
-import org.apache.eagle.jpm.mr.history.common.JHFConfigManager;
+import org.apache.eagle.jpm.mr.history.MRHistoryJobConfig;
 import org.apache.eagle.jpm.mr.historyentity.*;
 import org.apache.eagle.log.entity.GenericServiceAPIResponseEntity;
 import org.apache.eagle.service.client.IEagleServiceClient;
@@ -34,23 +34,24 @@ public class JobEntityCreationEagleServiceListener implements HistoryJobEntityCr
     private static final int BATCH_SIZE = 1000;
     private int batchSize;
     private List<JobBaseAPIEntity> list = new ArrayList<>();
-    private JHFConfigManager configManager;
+    private MRHistoryJobConfig configManager;
     List<JobExecutionAPIEntity> jobs = new ArrayList<>();
     List<JobEventAPIEntity> jobEvents = new ArrayList<>();
     List<TaskExecutionAPIEntity> taskExecs = new ArrayList<>();
     List<TaskAttemptExecutionAPIEntity> taskAttemptExecs = new ArrayList<>();
-    
-    public JobEntityCreationEagleServiceListener(JHFConfigManager configManager){
+
+    public JobEntityCreationEagleServiceListener(MRHistoryJobConfig configManager) {
         this(configManager, BATCH_SIZE);
     }
-    
-    public JobEntityCreationEagleServiceListener(JHFConfigManager configManager, int batchSize) {
+
+    public JobEntityCreationEagleServiceListener(MRHistoryJobConfig configManager, int batchSize) {
         this.configManager = configManager;
-        if (batchSize <= 0)
+        if (batchSize <= 0) {
             throw new IllegalArgumentException("batchSize must be greater than 0 when it is provided");
+        }
         this.batchSize = batchSize;
     }
-    
+
     @Override
     public void jobEntityCreated(JobBaseAPIEntity entity) throws Exception {
         list.add(entity);
@@ -65,26 +66,26 @@ public class JobEntityCreationEagleServiceListener implements HistoryJobEntityCr
      */
     @Override
     public void flush() throws Exception {
-        JHFConfigManager.EagleServiceConfig eagleServiceConfig = configManager.getEagleServiceConfig();
-        JHFConfigManager.JobExtractorConfig jobExtractorConfig = configManager.getJobExtractorConfig();
+        MRHistoryJobConfig.EagleServiceConfig eagleServiceConfig = configManager.getEagleServiceConfig();
+        MRHistoryJobConfig.JobExtractorConfig jobExtractorConfig = configManager.getJobExtractorConfig();
         IEagleServiceClient client = new EagleServiceClientImpl(
-                eagleServiceConfig.eagleServiceHost,
-                eagleServiceConfig.eagleServicePort,
-                eagleServiceConfig.username,
-                eagleServiceConfig.password);
+            eagleServiceConfig.eagleServiceHost,
+            eagleServiceConfig.eagleServicePort,
+            eagleServiceConfig.username,
+            eagleServiceConfig.password);
 
         client.getJerseyClient().setReadTimeout(jobExtractorConfig.readTimeoutSeconds * 1000);
         logger.info("start flushing entities of total number " + list.size());
         for (int i = 0; i < list.size(); i++) {
             JobBaseAPIEntity entity = list.get(i);
             if (entity instanceof JobExecutionAPIEntity) {
-                jobs.add((JobExecutionAPIEntity)entity);
-            } else if(entity instanceof JobEventAPIEntity) {
-                jobEvents.add((JobEventAPIEntity)entity);
-            } else if(entity instanceof TaskExecutionAPIEntity) {
-                taskExecs.add((TaskExecutionAPIEntity)entity);
-            } else if(entity instanceof TaskAttemptExecutionAPIEntity) {
-                taskAttemptExecs.add((TaskAttemptExecutionAPIEntity)entity);
+                jobs.add((JobExecutionAPIEntity) entity);
+            } else if (entity instanceof JobEventAPIEntity) {
+                jobEvents.add((JobEventAPIEntity) entity);
+            } else if (entity instanceof TaskExecutionAPIEntity) {
+                taskExecs.add((TaskExecutionAPIEntity) entity);
+            } else if (entity instanceof TaskAttemptExecutionAPIEntity) {
+                taskAttemptExecs.add((TaskAttemptExecutionAPIEntity) entity);
             }
         }
         GenericServiceAPIResponseEntity result;
@@ -117,7 +118,7 @@ public class JobEntityCreationEagleServiceListener implements HistoryJobEntityCr
         client.getJerseyClient().destroy();
         client.close();
     }
-    
+
     private void checkResult(GenericServiceAPIResponseEntity result) throws Exception {
         if (!result.isSuccess()) {
             logger.error(result.getException());

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/0bde482b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JobEntityCreationPublisher.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JobEntityCreationPublisher.java b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JobEntityCreationPublisher.java
index 6e0371c..8b85b26 100644
--- a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JobEntityCreationPublisher.java
+++ b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JobEntityCreationPublisher.java
@@ -24,21 +24,22 @@ import java.util.Vector;
 
 /**
  * not thread safe
- * @author yonzhang
  *
+ * @author yonzhang
  */
 public class JobEntityCreationPublisher {
     private Vector<HistoryJobEntityCreationListener> listeners = new Vector<>(2);
-    public void addListener(HistoryJobEntityCreationListener l){
+
+    public void addListener(HistoryJobEntityCreationListener l) {
         listeners.add(l);
     }
-    
+
     public void notifiyListeners(JobBaseAPIEntity entity) throws Exception {
         for (HistoryJobEntityCreationListener l : listeners) {
             l.jobEntityCreated(entity);
         }
     }
-    
+
     public void flush() throws Exception {
         for (HistoryJobEntityCreationListener l : listeners) {
             l.flush();

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/0bde482b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JobEntityLifecycleAggregator.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JobEntityLifecycleAggregator.java b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JobEntityLifecycleAggregator.java
index d97e2a3..594d8e2 100644
--- a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JobEntityLifecycleAggregator.java
+++ b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JobEntityLifecycleAggregator.java
@@ -53,10 +53,10 @@ public class JobEntityLifecycleAggregator implements HistoryJobEntityLifecycleLi
 
     @Override
     public void jobEntityCreated(JobBaseAPIEntity entity) throws Exception {
-        if (entity != null ) {
+        if (entity != null) {
             if (entity instanceof TaskAttemptExecutionAPIEntity) {
                 taskAttemptEntityCreated((TaskAttemptExecutionAPIEntity) entity);
-            } else if(entity instanceof JobExecutionAPIEntity) {
+            } else if (entity instanceof JobExecutionAPIEntity) {
                 this.m_jobExecutionAPIEntity = (JobExecutionAPIEntity) entity;
             }
         }
@@ -74,24 +74,28 @@ public class JobEntityLifecycleAggregator implements HistoryJobEntityLifecycleLi
             JobCounters jobCounters = m_jobExecutionAPIEntity.getJobCounters();
 
             if (jobCounters == null) {
-                LOG.warn("no job counter found for "+this.m_jobExecutionAPIEntity);
+                LOG.warn("no job counter found for " + this.m_jobExecutionAPIEntity);
                 jobCounters = new JobCounters();
             }
 
-            Map<String,Map<String,Long>> counters = jobCounters.getCounters();
+            Map<String, Map<String, Long>> counters = jobCounters.getCounters();
 
-            Map<String,Long> mapTaskAttemptCounter = this.m_mapTaskAttemptCounterAgg.result();
-            if (mapTaskAttemptCounter == null) mapTaskAttemptCounter = new HashMap<>();
+            Map<String, Long> mapTaskAttemptCounter = this.m_mapTaskAttemptCounterAgg.result();
+            if (mapTaskAttemptCounter == null) {
+                mapTaskAttemptCounter = new HashMap<>();
+            }
             mapTaskAttemptCounter.put(Constants.TaskAttemptCounter.TASK_ATTEMPT_DURATION.toString(), this.m_mapAttemptDuration);
-            counters.put(Constants.MAP_TASK_ATTEMPT_COUNTER,mapTaskAttemptCounter);
+            counters.put(Constants.MAP_TASK_ATTEMPT_COUNTER, mapTaskAttemptCounter);
 
-            Map<String,Long> reduceTaskAttemptCounter = this.m_reduceTaskAttemptCounterAgg.result();
-            if (reduceTaskAttemptCounter == null) reduceTaskAttemptCounter = new HashMap<>();
+            Map<String, Long> reduceTaskAttemptCounter = this.m_reduceTaskAttemptCounterAgg.result();
+            if (reduceTaskAttemptCounter == null) {
+                reduceTaskAttemptCounter = new HashMap<>();
+            }
             reduceTaskAttemptCounter.put(Constants.TaskAttemptCounter.TASK_ATTEMPT_DURATION.toString(), this.m_reduceAttemptDuration);
-            counters.put(Constants.REDUCE_TASK_ATTEMPT_COUNTER,reduceTaskAttemptCounter);
+            counters.put(Constants.REDUCE_TASK_ATTEMPT_COUNTER, reduceTaskAttemptCounter);
 
             counters.put(Constants.MAP_TASK_ATTEMPT_FILE_SYSTEM_COUNTER, this.m_mapFileSystemCounterAgg.result());
-            counters.put(Constants.REDUCE_TASK_ATTEMPT_FILE_SYSTEM_COUNTER,this.m_reduceFileSystemTaskCounterAgg.result());
+            counters.put(Constants.REDUCE_TASK_ATTEMPT_FILE_SYSTEM_COUNTER, this.m_reduceFileSystemTaskCounterAgg.result());
 
             jobCounters.setCounters(counters);
 
@@ -122,9 +126,9 @@ public class JobEntityLifecycleAggregator implements HistoryJobEntityLifecycleLi
 
         ObjectMapper objectMapper = new ObjectMapper();
         try {
-            LOG.warn("Unknown task type of task attempt execution entity: "+objectMapper.writeValueAsString(entity));
+            LOG.warn("Unknown task type of task attempt execution entity: " + objectMapper.writeValueAsString(entity));
         } catch (Exception e) {
-            LOG.error(e.getMessage(),e);
+            LOG.error(e.getMessage(), e);
         }
     }
 
@@ -137,13 +141,14 @@ public class JobEntityLifecycleAggregator implements HistoryJobEntityLifecycleLi
 
     interface JobCounterAggregateFunction {
         void accumulate(Map<String, Long> mapCounters);
-        Map<String,Long> result();
+
+        Map<String, Long> result();
     }
 
     static class JobCounterSumFunction implements JobCounterAggregateFunction {
         final Map<String, Long> result;
 
-        public JobCounterSumFunction(){
+        public JobCounterSumFunction() {
             result = new HashMap<>();
         }
 
@@ -153,7 +158,7 @@ public class JobEntityLifecycleAggregator implements HistoryJobEntityLifecycleLi
         @Override
         public void accumulate(Map<String, Long> counters) {
             if (counters != null) {
-                for (Map.Entry<String,Long> taskEntry: counters.entrySet()) {
+                for (Map.Entry<String, Long> taskEntry : counters.entrySet()) {
                     String counterName = taskEntry.getKey();
                     long counterValue = taskEntry.getValue();
 

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/0bde482b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/MRErrorClassifier.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/MRErrorClassifier.java b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/MRErrorClassifier.java
index a2f4c3a..e65ec17 100644
--- a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/MRErrorClassifier.java
+++ b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/MRErrorClassifier.java
@@ -28,13 +28,13 @@ import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 
 public final class MRErrorClassifier {
-    
+
     private final List<ErrorCategory> categories = new ArrayList<ErrorCategory>();
 
     public MRErrorClassifier(InputStream configStream) throws IOException {
         final BufferedReader reader = new BufferedReader(new InputStreamReader(configStream));
         String line;
-        while((line = reader.readLine()) != null) {
+        while ((line = reader.readLine()) != null) {
             line = line.trim();
             if (line.isEmpty() || line.startsWith("#")) {
                 continue;
@@ -58,7 +58,7 @@ public final class MRErrorClassifier {
             categories.add(category);
         }
     }
-    
+
     public List<ErrorCategory> getErrorCategories() {
         return categories;
     }
@@ -72,31 +72,36 @@ public final class MRErrorClassifier {
         }
         return "UNKNOWN";
     }
-    
+
     public static class ErrorCategory {
         private String name;
         private Pattern pattern;
         private boolean needTransform;
-        
+
         public String getName() {
             return name;
         }
+
         public void setName(String name) {
             this.name = name;
         }
+
         public Pattern getPattern() {
             return pattern;
         }
+
         public void setPattern(Pattern pattern) {
             this.pattern = pattern;
         }
+
         public boolean isNeedTransform() {
             return needTransform;
         }
+
         public void setNeedTransform(boolean needTransform) {
             this.needTransform = needTransform;
         }
-        
+
         public String classify(String error) {
             Matcher matcher = pattern.matcher(error);
             if (matcher.find()) {

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/0bde482b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/RecordTypes.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/RecordTypes.java b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/RecordTypes.java
index d2ac944..be23051 100644
--- a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/RecordTypes.java
+++ b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/RecordTypes.java
@@ -17,9 +17,10 @@
 */
 
 package org.apache.eagle.jpm.mr.history.parser;
+
 /**
- * Record types are identifiers for each line of log in history files. 
- * A record type appears as the first token in a single line of log. 
+ * Record types are identifiers for each line of log in history files.
+ * A record type appears as the first token in a single line of log.
  */
 public enum RecordTypes {
     Jobtracker, Job, Task, MapAttempt, ReduceAttempt, Meta

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/0bde482b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/TaskAttemptCounterListener.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/TaskAttemptCounterListener.java b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/TaskAttemptCounterListener.java
index d819baf..efc43c5 100755
--- a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/TaskAttemptCounterListener.java
+++ b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/TaskAttemptCounterListener.java
@@ -18,7 +18,7 @@
 
 package org.apache.eagle.jpm.mr.history.parser;
 
-import org.apache.eagle.jpm.mr.history.common.JHFConfigManager;
+import org.apache.eagle.jpm.mr.history.MRHistoryJobConfig;
 import org.apache.eagle.jpm.mr.historyentity.JobBaseAPIEntity;
 import org.apache.eagle.jpm.mr.historyentity.TaskAttemptCounterAPIEntity;
 import org.apache.eagle.jpm.mr.historyentity.TaskAttemptExecutionAPIEntity;
@@ -34,46 +34,48 @@ public class TaskAttemptCounterListener implements HistoryJobEntityCreationListe
     private static final Logger logger = LoggerFactory.getLogger(TaskAttemptCounterListener.class);
     private static final int BATCH_SIZE = 1000;
     private Map<CounterKey, CounterValue> counters = new HashMap<>();
-    private JHFConfigManager configManager;
+    private MRHistoryJobConfig configManager;
 
-    public TaskAttemptCounterListener(JHFConfigManager configManager) {
+    public TaskAttemptCounterListener(MRHistoryJobConfig configManager) {
         this.configManager = configManager;
     }
 
     private static class CounterKey {
         Map<String, String> tags = new HashMap<>();
         long timestamp;
-        
+
         @Override
         public boolean equals(Object thatKey) {
-            if (!(thatKey instanceof CounterKey))
+            if (!(thatKey instanceof CounterKey)) {
                 return false;
-            CounterKey that = (CounterKey)thatKey;
-            if (that.tags.equals(this.tags) && that.timestamp == this.timestamp)
+            }
+            CounterKey that = (CounterKey) thatKey;
+            if (that.tags.equals(this.tags) && that.timestamp == this.timestamp) {
                 return true;
+            }
             return false;
         }
-        
+
         @Override
-        public int hashCode(){
+        public int hashCode() {
             return tags.hashCode() ^ Long.valueOf(timestamp).hashCode();
         }
     }
-    
+
     private static class CounterValue {
         int totalCount;
         int failedCount;
         int killedCount;
     }
-    
+
     @Override
     public void jobEntityCreated(JobBaseAPIEntity entity) throws Exception {
         if (!(entity instanceof TaskAttemptExecutionAPIEntity)) {
             return;
         }
-        
-        TaskAttemptExecutionAPIEntity e = (TaskAttemptExecutionAPIEntity)entity;
-        
+
+        TaskAttemptExecutionAPIEntity e = (TaskAttemptExecutionAPIEntity) entity;
+
         Map<String, String> tags = new HashMap<>();
         tags.put(MRJobTagName.SITE.toString(), e.getTags().get(MRJobTagName.SITE.toString()));
         tags.put(MRJobTagName.JOD_DEF_ID.toString(), e.getTags().get(MRJobTagName.JOD_DEF_ID.toString()));
@@ -85,21 +87,21 @@ public class TaskAttemptCounterListener implements HistoryJobEntityCreationListe
         CounterKey key = new CounterKey();
         key.tags = tags;
         key.timestamp = roundToMinute(e.getEndTime());
-        
+
         CounterValue value = counters.get(key);
         if (value == null) {
             value = new CounterValue();
             counters.put(key, value);
         }
-        
+
         if (e.getTaskStatus().equals(EagleTaskStatus.FAILED.name())) {
             value.failedCount++;
-        } else if(e.getTaskStatus().equals(EagleTaskStatus.KILLED.name())) {
+        } else if (e.getTaskStatus().equals(EagleTaskStatus.KILLED.name())) {
             value.killedCount++;
         }
         value.totalCount++;
     }
-    
+
     private long roundToMinute(long timestamp) {
         GregorianCalendar cal = new GregorianCalendar();
         cal.setTimeInMillis(timestamp);
@@ -107,16 +109,16 @@ public class TaskAttemptCounterListener implements HistoryJobEntityCreationListe
         cal.set(Calendar.MILLISECOND, 0);
         return cal.getTimeInMillis();
     }
-    
+
     @Override
     public void flush() throws Exception {
-        JHFConfigManager.EagleServiceConfig eagleServiceConfig = configManager.getEagleServiceConfig();
-        JHFConfigManager.JobExtractorConfig jobExtractorConfig = configManager.getJobExtractorConfig();
+        MRHistoryJobConfig.EagleServiceConfig eagleServiceConfig = configManager.getEagleServiceConfig();
+        MRHistoryJobConfig.JobExtractorConfig jobExtractorConfig = configManager.getJobExtractorConfig();
         IEagleServiceClient client = new EagleServiceClientImpl(
-                eagleServiceConfig.eagleServiceHost,
-                eagleServiceConfig.eagleServicePort,
-                eagleServiceConfig.username,
-                eagleServiceConfig.password);
+            eagleServiceConfig.eagleServiceHost,
+            eagleServiceConfig.eagleServicePort,
+            eagleServiceConfig.username,
+            eagleServiceConfig.password);
 
         client.getJerseyClient().setReadTimeout(jobExtractorConfig.readTimeoutSeconds * 1000);
         List<TaskAttemptCounterAPIEntity> list = new ArrayList<>();
@@ -132,7 +134,7 @@ public class TaskAttemptCounterListener implements HistoryJobEntityCreationListe
             entity.setFailedCount(value.failedCount);
             entity.setKilledCount(value.killedCount);
             list.add(entity);
-            
+
             if (list.size() >= BATCH_SIZE) {
                 logger.info("start flushing TaskAttemptCounter " + list.size());
                 client.create(list);

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/0bde482b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/TaskFailureListener.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/TaskFailureListener.java b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/TaskFailureListener.java
index 6e42fe2..e0c3c6b 100755
--- a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/TaskFailureListener.java
+++ b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/TaskFailureListener.java
@@ -18,7 +18,7 @@
 
 package org.apache.eagle.jpm.mr.history.parser;
 
-import org.apache.eagle.jpm.mr.history.common.JHFConfigManager;
+import org.apache.eagle.jpm.mr.history.MRHistoryJobConfig;
 import org.apache.eagle.jpm.mr.historyentity.JobBaseAPIEntity;
 import org.apache.eagle.jpm.mr.historyentity.TaskAttemptExecutionAPIEntity;
 import org.apache.eagle.jpm.mr.historyentity.TaskFailureCountAPIEntity;
@@ -44,94 +44,98 @@ public class TaskFailureListener implements HistoryJobEntityCreationListener {
 
     private final List<TaskFailureCountAPIEntity> failureTasks = new ArrayList<TaskFailureCountAPIEntity>();
     private final MRErrorClassifier classifier;
-	private JHFConfigManager configManager;
+    private MRHistoryJobConfig configManager;
 
-    public TaskFailureListener(JHFConfigManager configManager) {
+    public TaskFailureListener(MRHistoryJobConfig configManager) {
         this.configManager = configManager;
-    	InputStream is = null;
-    	try {
-    		is = TaskFailureListener.class.getClassLoader().getResourceAsStream(MR_ERROR_CATEGORY_CONFIG_FILE_NAME);
+        InputStream is = null;
+        try {
+            is = TaskFailureListener.class.getClassLoader().getResourceAsStream(MR_ERROR_CATEGORY_CONFIG_FILE_NAME);
             URL url = TaskFailureListener.class.getClassLoader().getResource(MR_ERROR_CATEGORY_CONFIG_FILE_NAME);
             if (url != null) {
                 logger.info("Feeder is going to load configuration file: " + url.toString());
             }
-    		classifier = new MRErrorClassifier(is);
-    	} catch (IOException ex) {
-    		throw new RuntimeException("Can't find MRErrorCategory.config file to configure MRErrorCategory");
-    	} finally {
-    		if (is != null) {
-    			try {
-    				is.close();
-    			} catch (IOException e) {
-    			}
-    		}
-    	}
+            classifier = new MRErrorClassifier(is);
+        } catch (IOException ex) {
+            throw new RuntimeException("Can't find MRErrorCategory.config file to configure MRErrorCategory");
+        } finally {
+            if (is != null) {
+                try {
+                    is.close();
+                } catch (IOException e) {
+                }
+            }
+        }
     }
 
     @Override
     public void jobEntityCreated(JobBaseAPIEntity entity) throws Exception {
-    	if (!(entity instanceof TaskAttemptExecutionAPIEntity))
-    		return;
+        if (!(entity instanceof TaskAttemptExecutionAPIEntity)) {
+            return;
+        }
 
-    	TaskAttemptExecutionAPIEntity e = (TaskAttemptExecutionAPIEntity)entity;
-    	// only store those killed or failed tasks
-    	if (!e.getTaskStatus().equals(EagleTaskStatus.FAILED.name()) && !e.getTaskStatus().equals(EagleTaskStatus.KILLED.name()))
-    		return;
+        TaskAttemptExecutionAPIEntity e = (TaskAttemptExecutionAPIEntity) entity;
+        // only store those killed or failed tasks
+        if (!e.getTaskStatus().equals(EagleTaskStatus.FAILED.name()) && !e.getTaskStatus().equals(EagleTaskStatus.KILLED.name())) {
+            return;
+        }
 
-    	TaskFailureCountAPIEntity failureTask = new TaskFailureCountAPIEntity();
-    	Map<String, String> tags = new HashMap<>();
-    	failureTask.setTags(tags);
-    	tags.put(MRJobTagName.SITE.toString(), e.getTags().get(MRJobTagName.SITE.toString()));
-    	tags.put(MRJobTagName.JOD_DEF_ID.toString(), e.getTags().get(MRJobTagName.JOD_DEF_ID.toString()));
-    	tags.put(MRJobTagName.RACK.toString(), e.getTags().get(MRJobTagName.RACK.toString()));
-    	tags.put(MRJobTagName.HOSTNAME.toString(), e.getTags().get(MRJobTagName.HOSTNAME.toString()));
-    	tags.put(MRJobTagName.JOB_ID.toString(), e.getTags().get(MRJobTagName.JOB_ID.toString()));
-    	tags.put(MRJobTagName.TASK_ATTEMPT_ID.toString(), e.getTaskAttemptID());
-    	tags.put(MRJobTagName.TASK_TYPE.toString(), e.getTags().get(MRJobTagName.TASK_TYPE.toString()));
+        TaskFailureCountAPIEntity failureTask = new TaskFailureCountAPIEntity();
+        Map<String, String> tags = new HashMap<>();
+        failureTask.setTags(tags);
+        tags.put(MRJobTagName.SITE.toString(), e.getTags().get(MRJobTagName.SITE.toString()));
+        tags.put(MRJobTagName.JOD_DEF_ID.toString(), e.getTags().get(MRJobTagName.JOD_DEF_ID.toString()));
+        tags.put(MRJobTagName.RACK.toString(), e.getTags().get(MRJobTagName.RACK.toString()));
+        tags.put(MRJobTagName.HOSTNAME.toString(), e.getTags().get(MRJobTagName.HOSTNAME.toString()));
+        tags.put(MRJobTagName.JOB_ID.toString(), e.getTags().get(MRJobTagName.JOB_ID.toString()));
+        tags.put(MRJobTagName.TASK_ATTEMPT_ID.toString(), e.getTaskAttemptID());
+        tags.put(MRJobTagName.TASK_TYPE.toString(), e.getTags().get(MRJobTagName.TASK_TYPE.toString()));
 
-    	//TODO need optimize, match and then capture the data
-    	final String errCategory = classifier.classifyError(e.getError());
-    	tags.put(MRJobTagName.ERROR_CATEGORY.toString(), errCategory);
+        //TODO need optimize, match and then capture the data
+        final String errCategory = classifier.classifyError(e.getError());
+        tags.put(MRJobTagName.ERROR_CATEGORY.toString(), errCategory);
 
-    	failureTask.setError(e.getError());
-    	failureTask.setFailureCount(1); // hard coded to 1 unless we do pre-aggregation in the future
-    	failureTask.setTimestamp(e.getTimestamp());
-    	failureTask.setTaskStatus(e.getTaskStatus());
-    	failureTasks.add(failureTask);
+        failureTask.setError(e.getError());
+        failureTask.setFailureCount(1); // hard coded to 1 unless we do pre-aggregation in the future
+        failureTask.setTimestamp(e.getTimestamp());
+        failureTask.setTaskStatus(e.getTaskStatus());
+        failureTasks.add(failureTask);
 
-    	if (failureTasks.size() >= BATCH_SIZE) flush();
+        if (failureTasks.size() >= BATCH_SIZE) {
+            flush();
+        }
     }
-    
+
     @Override
     public void flush() throws Exception {
-		JHFConfigManager.EagleServiceConfig eagleServiceConfig = configManager.getEagleServiceConfig();
-		JHFConfigManager.JobExtractorConfig jobExtractorConfig = configManager.getJobExtractorConfig();
-		IEagleServiceClient client = new EagleServiceClientImpl(
-				eagleServiceConfig.eagleServiceHost,
-				eagleServiceConfig.eagleServicePort,
-				eagleServiceConfig.username,
-				eagleServiceConfig.password);
+        MRHistoryJobConfig.EagleServiceConfig eagleServiceConfig = configManager.getEagleServiceConfig();
+        MRHistoryJobConfig.JobExtractorConfig jobExtractorConfig = configManager.getJobExtractorConfig();
+        IEagleServiceClient client = new EagleServiceClientImpl(
+            eagleServiceConfig.eagleServiceHost,
+            eagleServiceConfig.eagleServicePort,
+            eagleServiceConfig.username,
+            eagleServiceConfig.password);
 
-		client.getJerseyClient().setReadTimeout(jobExtractorConfig.readTimeoutSeconds * 1000);
+        client.getJerseyClient().setReadTimeout(jobExtractorConfig.readTimeoutSeconds * 1000);
 
-    	int tried = 0;
-    	while (tried <= MAX_RETRY_TIMES) {
-    		try {
-    			logger.info("start flushing entities of total number " + failureTasks.size());
-    			client.create(failureTasks);
-    			logger.info("finish flushing entities of total number " + failureTasks.size());
-    			failureTasks.clear();
-				break;
-    		} catch (Exception ex) {
-    			if (tried < MAX_RETRY_TIMES) {
-    				logger.error("Got exception to flush, retry as " + (tried + 1) + " times", ex);
-    			} else {
-    				logger.error("Got exception to flush, reach max retry times " + MAX_RETRY_TIMES, ex);
-    				throw ex;
-    			}
-    		}
-			tried ++;
-    	}
+        int tried = 0;
+        while (tried <= MAX_RETRY_TIMES) {
+            try {
+                logger.info("start flushing entities of total number " + failureTasks.size());
+                client.create(failureTasks);
+                logger.info("finish flushing entities of total number " + failureTasks.size());
+                failureTasks.clear();
+                break;
+            } catch (Exception ex) {
+                if (tried < MAX_RETRY_TIMES) {
+                    logger.error("Got exception to flush, retry as " + (tried + 1) + " times", ex);
+                } else {
+                    logger.error("Got exception to flush, reach max retry times " + MAX_RETRY_TIMES, ex);
+                    throw ex;
+                }
+            }
+            tried++;
+        }
         client.getJerseyClient().destroy();
         client.close();
     }


[34/52] [abbrv] incubator-eagle git commit: EAGLE-519: no data alert enhancement

Posted by yo...@apache.org.
EAGLE-519: no data alert enhancement

Author: Li, Garrett
Reviewer: ralphsu

This closes #413


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

Branch: refs/heads/master
Commit: ebbaad091b797945efa7147698de554a53117955
Parents: b2b16b7
Author: Ralph, Su <su...@gmail.com>
Authored: Fri Sep 2 16:33:23 2016 +0800
Committer: Ralph, Su <su...@gmail.com>
Committed: Fri Sep 2 16:33:23 2016 +0800

----------------------------------------------------------------------
 .../engine/coordinator/PolicyDefinition.java    |   3 +-
 .../alert/engine/coordinator/StreamColumn.java  |  25 +-
 .../eagle/alert/coordinator/Coordinator.java    |   2 +-
 .../coordinator/impl/GreedyPolicyScheduler.java |   4 +
 .../provider/NodataMetadataGenerator.java       | 343 +++++++++++++++++++
 .../provider/ScheduleContextBuilder.java        |  14 +-
 .../coordinator/trigger/CoordinatorTrigger.java |   2 +-
 .../NodataMetadataGeneratorTest.java            | 103 ++++++
 .../coordinator/ScheduleContextBuilderTest.java |  93 ++++-
 .../TestGreedyScheduleCoordinator.java          | 131 +++++++
 .../src/test/resources/application.conf         |   9 +
 .../src/test/resources/test-application.conf    |   9 +
 .../engine/evaluator/PolicyStreamHandlers.java  |   9 +-
 .../nodata/DistinctValuesInTimeBatchWindow.java | 128 +++++++
 .../nodata/NoDataPolicyTimeBatchHandler.java    | 168 +++++++++
 .../publisher/impl/JsonEventSerializer.java     |   2 +-
 .../AttributeCollectWithDistinctAggregator.java | 124 +++++++
 .../src/main/resources/eagle.siddhiext          |   1 +
 .../TestDistinctValuesInTimeBatchWindow.java    |  89 +++++
 .../TestNoDataPolicyTimeBatchHandler.java       | 158 +++++++++
 .../src/test/resources/eagle.siddhiext          |  18 +
 .../simple/application-integration.conf         |  16 +-
 .../src/test/resources/simple/topologies.json   |  14 +-
 .../src/main/resources/application.conf         |  11 +-
 24 files changed, 1442 insertions(+), 34 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/ebbaad09/eagle-core/eagle-alert-parent/eagle-alert/alert-common/src/main/java/org/apache/eagle/alert/engine/coordinator/PolicyDefinition.java
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-alert-parent/eagle-alert/alert-common/src/main/java/org/apache/eagle/alert/engine/coordinator/PolicyDefinition.java b/eagle-core/eagle-alert-parent/eagle-alert/alert-common/src/main/java/org/apache/eagle/alert/engine/coordinator/PolicyDefinition.java
index 0dca247..bc389a2 100644
--- a/eagle-core/eagle-alert-parent/eagle-alert/alert-common/src/main/java/org/apache/eagle/alert/engine/coordinator/PolicyDefinition.java
+++ b/eagle-core/eagle-alert-parent/eagle-alert/alert-common/src/main/java/org/apache/eagle/alert/engine/coordinator/PolicyDefinition.java
@@ -144,7 +144,8 @@ public class PolicyDefinition implements Serializable{
         if(! (that instanceof PolicyDefinition))
             return false;
         PolicyDefinition another = (PolicyDefinition)that;
-        if(another.name.equals(this.name) &&
+        if(Objects.equals(another.name, this.name) &&
+        		Objects.equals(another.description, this.description) &&
                 CollectionUtils.isEqualCollection(another.inputStreams, this.inputStreams) &&
                 CollectionUtils.isEqualCollection(another.outputStreams, this.outputStreams) &&
                 another.definition.equals(this.definition) &&

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/ebbaad09/eagle-core/eagle-alert-parent/eagle-alert/alert-common/src/main/java/org/apache/eagle/alert/engine/coordinator/StreamColumn.java
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-alert-parent/eagle-alert/alert-common/src/main/java/org/apache/eagle/alert/engine/coordinator/StreamColumn.java b/eagle-core/eagle-alert-parent/eagle-alert/alert-common/src/main/java/org/apache/eagle/alert/engine/coordinator/StreamColumn.java
index b11729d..c0d355e 100644
--- a/eagle-core/eagle-alert-parent/eagle-alert/alert-common/src/main/java/org/apache/eagle/alert/engine/coordinator/StreamColumn.java
+++ b/eagle-core/eagle-alert-parent/eagle-alert/alert-common/src/main/java/org/apache/eagle/alert/engine/coordinator/StreamColumn.java
@@ -16,15 +16,15 @@
  */
 package org.apache.eagle.alert.engine.coordinator;
 
-import com.fasterxml.jackson.databind.ObjectMapper;
-
-import javax.xml.bind.annotation.XmlType;
-import javax.xml.bind.annotation.adapters.XmlAdapter;
-import javax.xml.bind.annotation.adapters.XmlJavaTypeAdapter;
 import java.io.IOException;
 import java.io.Serializable;
 import java.util.HashMap;
 
+import javax.xml.bind.annotation.adapters.XmlAdapter;
+import javax.xml.bind.annotation.adapters.XmlJavaTypeAdapter;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+
 
 public class StreamColumn implements Serializable {
     private static final long serialVersionUID = -5457861313624389106L;
@@ -33,13 +33,22 @@ public class StreamColumn implements Serializable {
     private Object defaultValue;
     private boolean required;
     private String description;
+    private String nodataExpression;
 
     public String toString() {
-        return String.format("StreamColumn=name[%s], type=[%s], defaultValue=[%s], required=[%s]", name, type,
-                defaultValue, required);
+        return String.format("StreamColumn=name[%s], type=[%s], defaultValue=[%s], required=[%s], nodataExpression=[%s]", 
+        		name, type, defaultValue, required, nodataExpression);
     }
 
-    public String getName() {
+    public String getNodataExpression() {
+		return nodataExpression;
+	}
+
+	public void setNodataExpression(String nodataExpression) {
+		this.nodataExpression = nodataExpression;
+	}
+
+	public String getName() {
         return name;
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/ebbaad09/eagle-core/eagle-alert-parent/eagle-alert/alert-coordinator/src/main/java/org/apache/eagle/alert/coordinator/Coordinator.java
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-alert-parent/eagle-alert/alert-coordinator/src/main/java/org/apache/eagle/alert/coordinator/Coordinator.java b/eagle-core/eagle-alert-parent/eagle-alert/alert-coordinator/src/main/java/org/apache/eagle/alert/coordinator/Coordinator.java
index f46e4c2..5c455f6 100644
--- a/eagle-core/eagle-alert-parent/eagle-alert/alert-coordinator/src/main/java/org/apache/eagle/alert/coordinator/Coordinator.java
+++ b/eagle-core/eagle-alert-parent/eagle-alert/alert-coordinator/src/main/java/org/apache/eagle/alert/coordinator/Coordinator.java
@@ -128,7 +128,7 @@ public class Coordinator {
     		ScheduleState state = null;
     		try {
     			Stopwatch watch = Stopwatch.createStarted();
-    	        IScheduleContext context = new ScheduleContextBuilder(client).buildContext();
+    	        IScheduleContext context = new ScheduleContextBuilder(config, client).buildContext();
     	        TopologyMgmtService mgmtService = new TopologyMgmtService();
     	        IPolicyScheduler scheduler = PolicySchedulerFactory.createScheduler();
     	

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/ebbaad09/eagle-core/eagle-alert-parent/eagle-alert/alert-coordinator/src/main/java/org/apache/eagle/alert/coordinator/impl/GreedyPolicyScheduler.java
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-alert-parent/eagle-alert/alert-coordinator/src/main/java/org/apache/eagle/alert/coordinator/impl/GreedyPolicyScheduler.java b/eagle-core/eagle-alert-parent/eagle-alert/alert-coordinator/src/main/java/org/apache/eagle/alert/coordinator/impl/GreedyPolicyScheduler.java
index 6c98fa6..ebc533e 100644
--- a/eagle-core/eagle-alert-parent/eagle-alert/alert-coordinator/src/main/java/org/apache/eagle/alert/coordinator/impl/GreedyPolicyScheduler.java
+++ b/eagle-core/eagle-alert-parent/eagle-alert/alert-coordinator/src/main/java/org/apache/eagle/alert/coordinator/impl/GreedyPolicyScheduler.java
@@ -298,6 +298,10 @@ public class GreedyPolicyScheduler implements IPolicyScheduler {
      * @return
      */
     private int getQueueSize(int hint) {
+    	if (hint == 0) {
+    		// some policies require single bolt to execute
+    		return 1;
+    	}
         return initialQueueSize * ((hint + initialQueueSize - 1) / initialQueueSize);
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/ebbaad09/eagle-core/eagle-alert-parent/eagle-alert/alert-coordinator/src/main/java/org/apache/eagle/alert/coordinator/provider/NodataMetadataGenerator.java
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-alert-parent/eagle-alert/alert-coordinator/src/main/java/org/apache/eagle/alert/coordinator/provider/NodataMetadataGenerator.java b/eagle-core/eagle-alert-parent/eagle-alert/alert-coordinator/src/main/java/org/apache/eagle/alert/coordinator/provider/NodataMetadataGenerator.java
new file mode 100644
index 0000000..67dedeb
--- /dev/null
+++ b/eagle-core/eagle-alert-parent/eagle-alert/alert-coordinator/src/main/java/org/apache/eagle/alert/coordinator/provider/NodataMetadataGenerator.java
@@ -0,0 +1,343 @@
+/*
+ * 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.eagle.alert.coordinator.provider;
+
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.eagle.alert.coordination.model.Kafka2TupleMetadata;
+import org.apache.eagle.alert.coordination.model.Tuple2StreamMetadata;
+import org.apache.eagle.alert.engine.coordinator.PolicyDefinition;
+import org.apache.eagle.alert.engine.coordinator.Publishment;
+import org.apache.eagle.alert.engine.coordinator.StreamColumn;
+import org.apache.eagle.alert.engine.coordinator.StreamDefinition;
+import org.apache.eagle.alert.engine.coordinator.StreamPartition;
+import org.apache.eagle.alert.utils.TimePeriodUtils;
+import org.joda.time.Period;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.typesafe.config.Config;
+
+public class NodataMetadataGenerator {
+
+	private static final Logger LOG = LoggerFactory.getLogger(NodataMetadataGenerator.class);
+	
+	private static final String NODATA_ALERT_AGGR_STREAM = "nodata_alert_aggregation_stream";
+	private static final String NODATA_ALERT_AGGR_OUTPUT_STREAM = "nodata_alert_aggregation_output_stream";
+	private static final String NODATA_ALERT_AGGR_DATASOURCE_NAME = "nodata_alert_aggregation_ds";
+	private static final String NODATA_ALERT_AGGR_OUTPUT_DATASOURCE_NAME = "nodata_alert_aggregation_output_ds";
+	private static final String NODATA_ALERT_AGGR_TOPIC_NAME = "nodata_alert_aggregation";
+	private static final String NODATA_ALERT_AGGR_OUTPUT_TOPIC_NAME = "nodata_alert";
+	
+	private static final String DATASOURCE_TYPE = "KAFKA";
+	private static final String DATASOURCE_SCHEME_CLS = "org.apache.eagle.alert.engine.scheme.JsonScheme";
+	
+	private static final String NODATA_ALERT_AGGR_POLICY_TYPE = "nodataalert";
+	private static final String NODATA_ALERT_AGGR_OUTPUT_POLICY_TYPE = "siddhi";
+	
+	private static final String JSON_STRING_STREAM_NAME_SELECTOR_CLS = "org.apache.eagle.alert.engine.scheme.JsonStringStreamNameSelector";
+	private static final String STREAM_TIMESTAMP_COLUMN_NAME = "timestamp";
+	private static final String STREAM_TIMESTAMP_FORMAT = "";
+	
+	private static final String KAFKA_PUBLISHMENT_TYPE = "org.apache.eagle.alert.engine.publisher.impl.AlertKafkaPublisher";
+	private static final String EMAIL_PUBLISHMENT_TYPE = "org.apache.eagle.alert.engine.publisher.impl.AlertEmailPublisher";
+	
+	private static final String PUBLISHMENT_DEDUP_DURATION = "PT0M";
+	private static final String PUBLISHMENT_SERIALIZER = "org.apache.eagle.alert.engine.publisher.impl.JsonEventSerializer";
+	
+	public NodataMetadataGenerator() {}
+	
+	public void execute(Config config, Map<String, StreamDefinition> streamDefinitionsMap, 
+			Map<String, Kafka2TupleMetadata> kafkaSources, 
+			Map<String, PolicyDefinition> policies, Map<String, Publishment> publishments) {
+		Collection<StreamDefinition> streamDefinitions = streamDefinitionsMap.values();
+		for (StreamDefinition streamDefinition : streamDefinitions) {
+    		StreamColumn columnWithNodataExpression = null;
+    		for (StreamColumn column : streamDefinition.getColumns()) {
+    			if (StringUtils.isNotBlank(column.getNodataExpression())) {
+    				// has nodata alert setting, needs to generate the nodata alert policy
+    				if (columnWithNodataExpression != null) {
+    					columnWithNodataExpression = null;
+    					LOG.warn("Only one column in one stream is allowed to configure nodata alert");
+    					break;
+    				}
+    				columnWithNodataExpression = column;
+    			}
+    		}
+    		if (columnWithNodataExpression != null) {
+    			String streamName = streamDefinition.getStreamId();
+    			
+    			// create nodata alert aggr stream
+    			if (streamDefinitionsMap.containsKey(NODATA_ALERT_AGGR_STREAM)) {
+    				LOG.info("Nodata alert aggregation stream: {} already exists", NODATA_ALERT_AGGR_STREAM);
+    			} else {
+    				streamDefinitionsMap.put(NODATA_ALERT_AGGR_STREAM, buildAggregationStream());
+    				LOG.info("Created nodata alert aggregation stream: {}", NODATA_ALERT_AGGR_STREAM);
+    			}
+    			
+    			// create nodata alert aggr output stream
+    			if (streamDefinitionsMap.containsKey(NODATA_ALERT_AGGR_OUTPUT_STREAM)) {
+    				LOG.info("Nodata alert aggregation output stream: {} already exists", NODATA_ALERT_AGGR_OUTPUT_STREAM);
+    			} else {
+    				streamDefinitionsMap.put(NODATA_ALERT_AGGR_OUTPUT_STREAM, buildAggregationOutputStream());
+    				LOG.info("Created nodata alert aggregation output stream: {}", NODATA_ALERT_AGGR_OUTPUT_STREAM);
+    			}
+    			
+    			// create nodata alert data source
+    			if (kafkaSources.containsKey(NODATA_ALERT_AGGR_DATASOURCE_NAME)) {
+    				LOG.info("Stream: {} nodata alert aggregation datasource: {} already exists", 
+    						NODATA_ALERT_AGGR_STREAM, NODATA_ALERT_AGGR_DATASOURCE_NAME);
+    			} else {
+	    			kafkaSources.put(NODATA_ALERT_AGGR_DATASOURCE_NAME, buildAggregationDatasource());
+	    			LOG.info("Created nodata alert aggregation datasource {} for stream {}", 
+	    					NODATA_ALERT_AGGR_DATASOURCE_NAME, NODATA_ALERT_AGGR_STREAM);
+    			}
+    			
+    			// create nodata alert aggregation output datasource
+    			if (kafkaSources.containsKey(NODATA_ALERT_AGGR_OUTPUT_DATASOURCE_NAME)) {
+    				LOG.info("Stream: {} nodata alert aggregation output datasource: {} already exists", 
+    						NODATA_ALERT_AGGR_OUTPUT_STREAM, NODATA_ALERT_AGGR_OUTPUT_DATASOURCE_NAME);
+    			} else {
+	    			kafkaSources.put(NODATA_ALERT_AGGR_OUTPUT_DATASOURCE_NAME, buildAggregationOutputDatasource());
+	    			LOG.info("Created nodata alert aggregation output datasource {} for stream {}", 
+	    					NODATA_ALERT_AGGR_DATASOURCE_NAME, NODATA_ALERT_AGGR_OUTPUT_STREAM);
+    			}
+    			
+    			// create nodata alert policy
+    			String policyName = streamName + "_nodata_alert";
+    			String nodataExpression = columnWithNodataExpression.getNodataExpression();
+    			String[] segments = nodataExpression.split(",");
+    			long windowPeriodInSeconds = TimePeriodUtils.getSecondsOfPeriod(Period.parse(segments[0]));
+    			if (policies.containsKey(policyName)) {
+    				LOG.info("Stream: {} nodata alert policy: {} already exists", streamName, policyName);
+    			} else {
+    				policies.put(policyName, buildDynamicNodataPolicy(
+    						streamName,
+    						policyName, 
+    						columnWithNodataExpression.getName(),
+    						nodataExpression,
+    						Arrays.asList(streamName)));
+    				LOG.info("Created nodata alert policy {} with expression {} for stream {}", 
+    						policyName, nodataExpression, streamName);
+    			}
+    			
+    			// create nodata alert aggregation
+    			String aggrPolicyName = NODATA_ALERT_AGGR_STREAM + "_policy";
+    			if (policies.containsKey(aggrPolicyName)) {
+    				LOG.info("Stream: {} nodata alert aggregation policy: {} already exists", 
+    						NODATA_ALERT_AGGR_OUTPUT_STREAM, aggrPolicyName);
+    			} else {
+    				policies.put(aggrPolicyName, buildAggregationPolicy(
+    						aggrPolicyName, 
+    						columnWithNodataExpression.getName(),
+    						windowPeriodInSeconds));
+    				LOG.info("Created nodata alert aggregation policy {} for stream {}", 
+    						aggrPolicyName, NODATA_ALERT_AGGR_OUTPUT_STREAM);
+    			}
+    			
+    			// create nodata alert publish
+    			String publishmentName = policyName + "_publish";
+    			if (publishments.containsKey(publishmentName)) {
+    				LOG.info("Stream: {} nodata alert publishment: {} already exists", streamName, publishmentName);
+    			} else {
+	    			String kafkaBroker = config.getString("kafkaProducer.bootstrapServers");
+	    			publishments.put(publishmentName, buildKafkaAlertPublishment(
+	    					publishmentName, policyName, kafkaBroker, NODATA_ALERT_AGGR_TOPIC_NAME));
+	    			publishments.put(publishmentName + "_email", buildEmailAlertPublishment(config, 
+	    					publishmentName + "_email", policyName, kafkaBroker, NODATA_ALERT_AGGR_TOPIC_NAME));
+	    			LOG.info("Created nodata alert publishment {} for stream {}", policyName + "_publish", streamName);
+    			}
+    			
+    			// create nodata alert aggregation publish
+    			String aggrPublishName = aggrPolicyName + "_publish";
+    			if (publishments.containsKey(aggrPublishName)) {
+    				LOG.info("Stream: {} publishment: {} already exists", NODATA_ALERT_AGGR_STREAM, aggrPublishName);
+    			} else {
+	    			String kafkaBroker = config.getString("kafkaProducer.bootstrapServers");
+	    			publishments.put(aggrPublishName, buildKafkaAlertPublishment(
+	    					aggrPublishName, aggrPolicyName, kafkaBroker, NODATA_ALERT_AGGR_OUTPUT_TOPIC_NAME));
+	    			publishments.put(aggrPublishName + "_email", buildEmailAlertPublishment(config, 
+	    					aggrPublishName + "_email", aggrPolicyName, kafkaBroker, NODATA_ALERT_AGGR_OUTPUT_TOPIC_NAME));
+	    			LOG.info("Created nodata alert publishment {} for stream {}", policyName + "_publish", streamName);
+    			}
+    		}
+    	}
+	}
+	
+	private Kafka2TupleMetadata buildAggregationDatasource() {
+		Kafka2TupleMetadata datasource = new Kafka2TupleMetadata();
+		datasource.setName(NODATA_ALERT_AGGR_DATASOURCE_NAME);
+		datasource.setType(DATASOURCE_TYPE);
+		datasource.setSchemeCls(DATASOURCE_SCHEME_CLS);
+		datasource.setTopic(NODATA_ALERT_AGGR_TOPIC_NAME);
+		Tuple2StreamMetadata codec = new Tuple2StreamMetadata();
+		codec.setStreamNameSelectorCls(JSON_STRING_STREAM_NAME_SELECTOR_CLS);
+		codec.setTimestampColumn(STREAM_TIMESTAMP_COLUMN_NAME);
+		codec.setTimestampFormat(STREAM_TIMESTAMP_FORMAT);
+		Properties codecProperties = new Properties();
+		codecProperties.put("userProvidedStreamName", NODATA_ALERT_AGGR_STREAM);
+		codecProperties.put("streamNameFormat", "%s");
+		codec.setStreamNameSelectorProp(codecProperties);
+		datasource.setCodec(codec);
+		return datasource;
+	}
+	
+	private Kafka2TupleMetadata buildAggregationOutputDatasource() {
+		Kafka2TupleMetadata datasource = new Kafka2TupleMetadata();
+		datasource.setName(NODATA_ALERT_AGGR_OUTPUT_DATASOURCE_NAME);
+		datasource.setType(DATASOURCE_TYPE);
+		datasource.setSchemeCls(DATASOURCE_SCHEME_CLS);
+		datasource.setTopic(NODATA_ALERT_AGGR_OUTPUT_TOPIC_NAME);
+		Tuple2StreamMetadata codec = new Tuple2StreamMetadata();
+		codec.setStreamNameSelectorCls(JSON_STRING_STREAM_NAME_SELECTOR_CLS);
+		codec.setTimestampColumn(STREAM_TIMESTAMP_COLUMN_NAME);
+		codec.setTimestampFormat(STREAM_TIMESTAMP_FORMAT);
+		Properties codecProperties = new Properties();
+		codecProperties.put("userProvidedStreamName", NODATA_ALERT_AGGR_OUTPUT_STREAM);
+		codecProperties.put("streamNameFormat", "%s");
+		codec.setStreamNameSelectorProp(codecProperties);
+		datasource.setCodec(codec);
+		return datasource;
+	}
+	
+	private PolicyDefinition buildDynamicNodataPolicy(String streamName, String policyName, 
+    		String columnName, String expression, List<String> inputStream) {
+		PolicyDefinition pd = new PolicyDefinition();
+		PolicyDefinition.Definition def = new PolicyDefinition.Definition();
+		//expression, something like "PT5S,dynamic,1,host"
+		def.setValue(expression);
+		def.setType(NODATA_ALERT_AGGR_POLICY_TYPE);
+		pd.setDefinition(def);
+		pd.setInputStreams(inputStream);
+		pd.setOutputStreams(Arrays.asList(NODATA_ALERT_AGGR_STREAM));
+		pd.setName(policyName);
+		pd.setDescription(String.format("Nodata alert policy for stream %s", streamName));
+		
+		StreamPartition sp = new StreamPartition();
+		sp.setStreamId(streamName);
+		sp.setColumns(Arrays.asList(columnName));
+		sp.setType(StreamPartition.Type.GROUPBY);
+		pd.addPartition(sp);
+		return pd;
+	}
+    
+    private PolicyDefinition buildAggregationPolicy(String policyName, String columnName, 
+    		long windowPeriodInSeconds) {
+    	PolicyDefinition pd = new PolicyDefinition();
+		PolicyDefinition.Definition def = new PolicyDefinition.Definition();
+		String SiddhiQL = String.format(
+				"from %s#window.timeBatch(%s sec) select eagle:collectWithDistinct(%s) as hosts, "
+				+ "originalStreamName as streamName group by originalStreamName insert into %s", 
+				NODATA_ALERT_AGGR_STREAM, windowPeriodInSeconds * 2, 
+				columnName, NODATA_ALERT_AGGR_OUTPUT_STREAM);
+		LOG.info("Generated SiddhiQL {} for stream: {}", SiddhiQL, NODATA_ALERT_AGGR_STREAM);
+		def.setValue(SiddhiQL);
+		def.setType(NODATA_ALERT_AGGR_OUTPUT_POLICY_TYPE);
+		pd.setDefinition(def);
+		pd.setInputStreams(Arrays.asList(NODATA_ALERT_AGGR_STREAM));
+		pd.setOutputStreams(Arrays.asList(NODATA_ALERT_AGGR_OUTPUT_STREAM));
+		pd.setName(policyName);
+		pd.setDescription("Nodata alert aggregation policy, used to merge alerts from multiple bolts");
+		
+		StreamPartition sp = new StreamPartition();
+		sp.setStreamId(NODATA_ALERT_AGGR_STREAM);
+		sp.setColumns(Arrays.asList(columnName));
+		sp.setType(StreamPartition.Type.GROUPBY);
+		pd.addPartition(sp);
+		pd.setParallelismHint(0);
+    	return pd;
+    }
+	
+	private Publishment buildKafkaAlertPublishment(String publishmentName, String policyName, String kafkaBroker, String topic) {
+		Publishment publishment = new Publishment();
+		publishment.setName(publishmentName);
+		publishment.setType(KAFKA_PUBLISHMENT_TYPE);
+		publishment.setPolicyIds(Arrays.asList(policyName));
+		publishment.setDedupIntervalMin(PUBLISHMENT_DEDUP_DURATION);
+		Map<String, String> publishmentProperties = new HashMap<String, String>();
+		publishmentProperties.put("kafka_broker", kafkaBroker);
+		publishmentProperties.put("topic", topic);
+		publishment.setProperties(publishmentProperties);
+		publishment.setSerializer(PUBLISHMENT_SERIALIZER);
+		return publishment;
+	}
+	
+	private Publishment buildEmailAlertPublishment(Config config, 
+			String publishmentName, String policyName, String kafkaBroker, String topic) {
+		Publishment publishment = new Publishment();
+		publishment.setName(publishmentName);
+		publishment.setType(EMAIL_PUBLISHMENT_TYPE);
+		publishment.setPolicyIds(Arrays.asList(policyName));
+		publishment.setDedupIntervalMin(PUBLISHMENT_DEDUP_DURATION);
+		Map<String, String> publishmentProperties = new HashMap<String, String>();
+		publishmentProperties.put("subject", String.format("Eagle Alert - %s", topic));
+		publishmentProperties.put("template", "");
+		publishmentProperties.put("sender", config.getString("email.sender"));
+		publishmentProperties.put("recipients", config.getString("email.recipients"));
+		publishmentProperties.put("mail.smtp.host", config.getString("email.mailSmtpHost"));
+		publishmentProperties.put("mail.smtp.port", config.getString("email.mailSmtpPort"));
+		publishmentProperties.put("connection", "plaintext");
+		publishment.setProperties(publishmentProperties);
+		publishment.setSerializer(PUBLISHMENT_SERIALIZER);
+		return publishment;
+	}
+	
+	private StreamDefinition buildAggregationStream() {
+		StreamDefinition sd = new StreamDefinition();
+		StreamColumn tsColumn = new StreamColumn();
+		tsColumn.setName("timestamp");
+		tsColumn.setType(StreamColumn.Type.LONG);
+		
+		StreamColumn hostColumn = new StreamColumn();
+		hostColumn.setName("host");
+		hostColumn.setType(StreamColumn.Type.STRING);
+
+		StreamColumn originalStreamNameColumn = new StreamColumn();
+		originalStreamNameColumn.setName("originalStreamName");
+		originalStreamNameColumn.setType(StreamColumn.Type.STRING);
+		
+		sd.setColumns(Arrays.asList(tsColumn, hostColumn, originalStreamNameColumn));
+		sd.setDataSource(NODATA_ALERT_AGGR_DATASOURCE_NAME);
+		sd.setStreamId(NODATA_ALERT_AGGR_STREAM);
+		sd.setDescription("Nodata alert aggregation stream");
+		return sd;
+	}
+	
+	private StreamDefinition buildAggregationOutputStream() {
+		StreamDefinition sd = new StreamDefinition();
+		StreamColumn hostColumn = new StreamColumn();
+		hostColumn.setName("hosts");
+		hostColumn.setType(StreamColumn.Type.STRING);
+		
+		StreamColumn osnColumn = new StreamColumn();
+		osnColumn.setName("streamName");
+		osnColumn.setType(StreamColumn.Type.STRING);
+		
+		sd.setColumns(Arrays.asList(hostColumn, osnColumn));
+		sd.setDataSource(NODATA_ALERT_AGGR_OUTPUT_DATASOURCE_NAME);
+		sd.setStreamId(NODATA_ALERT_AGGR_OUTPUT_STREAM);
+		sd.setDescription("Nodata alert aggregation output stream");
+		return sd;
+	}
+    
+}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/ebbaad09/eagle-core/eagle-alert-parent/eagle-alert/alert-coordinator/src/main/java/org/apache/eagle/alert/coordinator/provider/ScheduleContextBuilder.java
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-alert-parent/eagle-alert/alert-coordinator/src/main/java/org/apache/eagle/alert/coordinator/provider/ScheduleContextBuilder.java b/eagle-core/eagle-alert-parent/eagle-alert/alert-coordinator/src/main/java/org/apache/eagle/alert/coordinator/provider/ScheduleContextBuilder.java
index 9013f04..dd38395 100644
--- a/eagle-core/eagle-alert-parent/eagle-alert/alert-coordinator/src/main/java/org/apache/eagle/alert/coordinator/provider/ScheduleContextBuilder.java
+++ b/eagle-core/eagle-alert-parent/eagle-alert/alert-coordinator/src/main/java/org/apache/eagle/alert/coordinator/provider/ScheduleContextBuilder.java
@@ -61,6 +61,7 @@ public class ScheduleContextBuilder {
     private static final Logger LOG = LoggerFactory.getLogger(ScheduleContextBuilder.class);
     private static final String UNIQUE_BOLT_ID = "%s-%s";// toponame-boltname
 
+    private Config config;
     private IMetadataServiceClient client;
 
     private Map<String, Topology> topologies;
@@ -73,10 +74,12 @@ public class ScheduleContextBuilder {
     private Map<String, TopologyUsage> usages;
 
     public ScheduleContextBuilder(Config config) {
+    	this.config = config;
         client = new MetadataServiceClientImpl(config);
     }
 
-    public ScheduleContextBuilder(IMetadataServiceClient client) {
+    public ScheduleContextBuilder(Config config, IMetadataServiceClient client) {
+    	this.config = config;
         this.client = client;
     }
 
@@ -89,10 +92,13 @@ public class ScheduleContextBuilder {
         topologies = listToMap(client.listTopologies());
         kafkaSources = listToMap(client.listDataSources());
         // filter out disabled policies
-        policies = listToMap(client.listPolicies().stream().filter(
-        		(t) -> t.getPolicyStatus() != PolicyStatus.DISABLED).collect(Collectors.toList()));
+        List<PolicyDefinition> enabledPolicies = client.listPolicies().stream().filter(
+        		(t) -> t.getPolicyStatus() != PolicyStatus.DISABLED).collect(Collectors.toList());
+        policies = listToMap(enabledPolicies);
         publishments = listToMap(client.listPublishment());
         streamDefinitions = listToMap(client.listStreams());
+        // generate data sources, policies, publishments for nodata alert
+        new NodataMetadataGenerator().execute(config, streamDefinitions, kafkaSources, policies, publishments);
         
         // TODO: See ScheduleState comments on how to improve the storage
         ScheduleState state = client.getVersionedSpec();
@@ -114,7 +120,7 @@ public class ScheduleContextBuilder {
         return new InMemScheduleConext(topologies, assignments, kafkaSources, policies, publishments,
                 streamDefinitions, monitoredStreamMap, usages);
     }
-
+    
     /**
      * 1.
      * <pre>

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/ebbaad09/eagle-core/eagle-alert-parent/eagle-alert/alert-coordinator/src/main/java/org/apache/eagle/alert/coordinator/trigger/CoordinatorTrigger.java
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-alert-parent/eagle-alert/alert-coordinator/src/main/java/org/apache/eagle/alert/coordinator/trigger/CoordinatorTrigger.java b/eagle-core/eagle-alert-parent/eagle-alert/alert-coordinator/src/main/java/org/apache/eagle/alert/coordinator/trigger/CoordinatorTrigger.java
index c489a0e..4e17179 100644
--- a/eagle-core/eagle-alert-parent/eagle-alert/alert-coordinator/src/main/java/org/apache/eagle/alert/coordinator/trigger/CoordinatorTrigger.java
+++ b/eagle-core/eagle-alert-parent/eagle-alert/alert-coordinator/src/main/java/org/apache/eagle/alert/coordinator/trigger/CoordinatorTrigger.java
@@ -64,7 +64,7 @@ public class CoordinatorTrigger implements Runnable {
                 Stopwatch watch = Stopwatch.createStarted();
 
                 // schedule
-                IScheduleContext context = new ScheduleContextBuilder(client).buildContext();
+                IScheduleContext context = new ScheduleContextBuilder(config, client).buildContext();
                 TopologyMgmtService mgmtService = new TopologyMgmtService();
                 IPolicyScheduler scheduler = PolicySchedulerFactory.createScheduler();
 

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/ebbaad09/eagle-core/eagle-alert-parent/eagle-alert/alert-coordinator/src/test/java/org/apache/alert/coordinator/NodataMetadataGeneratorTest.java
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-alert-parent/eagle-alert/alert-coordinator/src/test/java/org/apache/alert/coordinator/NodataMetadataGeneratorTest.java b/eagle-core/eagle-alert-parent/eagle-alert/alert-coordinator/src/test/java/org/apache/alert/coordinator/NodataMetadataGeneratorTest.java
new file mode 100644
index 0000000..43dc9c1
--- /dev/null
+++ b/eagle-core/eagle-alert-parent/eagle-alert/alert-coordinator/src/test/java/org/apache/alert/coordinator/NodataMetadataGeneratorTest.java
@@ -0,0 +1,103 @@
+/*
+ * 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.alert.coordinator;
+
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.commons.lang3.builder.ToStringBuilder;
+import org.apache.eagle.alert.coordination.model.Kafka2TupleMetadata;
+import org.apache.eagle.alert.coordinator.provider.NodataMetadataGenerator;
+import org.apache.eagle.alert.engine.coordinator.PolicyDefinition;
+import org.apache.eagle.alert.engine.coordinator.Publishment;
+import org.apache.eagle.alert.engine.coordinator.StreamColumn;
+import org.apache.eagle.alert.engine.coordinator.StreamDefinition;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.typesafe.config.Config;
+import com.typesafe.config.ConfigFactory;
+
+public class NodataMetadataGeneratorTest {
+	
+	private static final Logger LOG = LoggerFactory.getLogger(NodataMetadataGeneratorTest.class);
+
+	Config config = ConfigFactory.load().getConfig("coordinator");
+	private NodataMetadataGenerator generator;
+	
+	@Before
+	public void setup() {
+		generator = new NodataMetadataGenerator();
+	}
+	
+	@Test
+	public void testNormal() throws Exception {
+		StreamDefinition sd = createStreamDefinitionWithNodataAlert();
+		Map<String, StreamDefinition> streamDefinitionsMap = new HashMap<String, StreamDefinition>();
+		streamDefinitionsMap.put(sd.getStreamId(), sd);
+		
+		Map<String, Kafka2TupleMetadata> kafkaSources = new HashMap<String, Kafka2TupleMetadata>();
+		Map<String, PolicyDefinition> policies = new HashMap<String, PolicyDefinition>();
+		Map<String, Publishment> publishments = new HashMap<String, Publishment>();
+		
+		generator.execute(config, streamDefinitionsMap, kafkaSources, policies, publishments);
+		
+		Assert.assertEquals(2, kafkaSources.size());
+		
+		kafkaSources.forEach((key, value) -> {
+			LOG.info("KafkaSources > {}: {}", key, ToStringBuilder.reflectionToString(value));
+		});
+		
+		Assert.assertEquals(2, policies.size());
+		
+		policies.forEach((key, value) -> {
+			LOG.info("Policies > {}: {}", key, ToStringBuilder.reflectionToString(value));
+		});
+		
+		Assert.assertEquals(4, publishments.size());
+		
+		publishments.forEach((key, value) -> {
+			LOG.info("Publishments > {}: {}", key, ToStringBuilder.reflectionToString(value));
+		});
+	}
+	
+	private StreamDefinition createStreamDefinitionWithNodataAlert() {
+		StreamDefinition sd = new StreamDefinition();
+		StreamColumn tsColumn = new StreamColumn();
+		tsColumn.setName("timestamp");
+		tsColumn.setType(StreamColumn.Type.LONG);
+
+		StreamColumn hostColumn = new StreamColumn();
+		hostColumn.setName("host");
+		hostColumn.setType(StreamColumn.Type.STRING);
+		hostColumn.setNodataExpression("PT1M,dynamic,1,host");
+
+		StreamColumn valueColumn = new StreamColumn();
+		valueColumn.setName("value");
+		valueColumn.setType(StreamColumn.Type.DOUBLE);
+
+		sd.setColumns(Arrays.asList(tsColumn, hostColumn, valueColumn));
+		sd.setDataSource("testDataSource");
+		sd.setStreamId("testStreamId");
+		return sd;
+	}
+	
+}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/ebbaad09/eagle-core/eagle-alert-parent/eagle-alert/alert-coordinator/src/test/java/org/apache/alert/coordinator/ScheduleContextBuilderTest.java
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-alert-parent/eagle-alert/alert-coordinator/src/test/java/org/apache/alert/coordinator/ScheduleContextBuilderTest.java b/eagle-core/eagle-alert-parent/eagle-alert/alert-coordinator/src/test/java/org/apache/alert/coordinator/ScheduleContextBuilderTest.java
index ed9d7b7..84153f6 100644
--- a/eagle-core/eagle-alert-parent/eagle-alert/alert-coordinator/src/test/java/org/apache/alert/coordinator/ScheduleContextBuilderTest.java
+++ b/eagle-core/eagle-alert-parent/eagle-alert/alert-coordinator/src/test/java/org/apache/alert/coordinator/ScheduleContextBuilderTest.java
@@ -21,6 +21,7 @@ import java.util.Arrays;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.Map.Entry;
 import java.util.Set;
 
 import org.apache.alert.coordinator.mock.InMemMetadataServiceClient;
@@ -48,17 +49,22 @@ import org.apache.eagle.alert.engine.coordinator.StreamSortSpec;
 import org.junit.Assert;
 import org.junit.Test;
 
+import com.typesafe.config.Config;
+import com.typesafe.config.ConfigFactory;
+
 /**
  * @since May 5, 2016
  *
  */
 public class ScheduleContextBuilderTest {
 
+	Config config = ConfigFactory.load().getConfig("coordinator");
+	
     @Test
     public void test() {
         InMemMetadataServiceClient client = getSampleMetadataService();
 
-        ScheduleContextBuilder builder = new ScheduleContextBuilder(client);
+        ScheduleContextBuilder builder = new ScheduleContextBuilder(config, client);
 
         IScheduleContext context = builder.buildContext();
 
@@ -84,7 +90,7 @@ public class ScheduleContextBuilderTest {
     @Test
     public void test_remove_policy() {
         InMemMetadataServiceClient client = getSampleMetadataService();
-        ScheduleContextBuilder builder = new ScheduleContextBuilder(client);
+        ScheduleContextBuilder builder = new ScheduleContextBuilder(config, client);
 
         PolicyAssignment assignment1 = client.getVersionedSpec().getAssignments().get(0);
 
@@ -105,7 +111,7 @@ public class ScheduleContextBuilderTest {
     @Test
     public void test_changed_policy_partition() {
         InMemMetadataServiceClient client = getSampleMetadataService();
-        ScheduleContextBuilder builder = new ScheduleContextBuilder(client);
+        ScheduleContextBuilder builder = new ScheduleContextBuilder(config, client);
         PolicyAssignment assignment1 = client.getVersionedSpec().getAssignments().get(0);
 
         IScheduleContext context = builder.buildContext();
@@ -143,7 +149,7 @@ public class ScheduleContextBuilderTest {
     @Test
     public void test_changed_policy_parallelism() {
         InMemMetadataServiceClient client = getSampleMetadataService();
-        ScheduleContextBuilder builder = new ScheduleContextBuilder(client);
+        ScheduleContextBuilder builder = new ScheduleContextBuilder(config, client);
         PolicyAssignment assignment1 = client.getVersionedSpec().getAssignments().get(0);
 
         IScheduleContext context = builder.buildContext();
@@ -171,7 +177,7 @@ public class ScheduleContextBuilderTest {
     @Test
     public void test_changed_policy_definition() {
         InMemMetadataServiceClient client = getSampleMetadataService();
-        ScheduleContextBuilder builder = new ScheduleContextBuilder(client);
+        ScheduleContextBuilder builder = new ScheduleContextBuilder(config, client);
         PolicyAssignment assignment1 = client.getVersionedSpec().getAssignments().get(0);
 
         IScheduleContext context = builder.buildContext();
@@ -189,11 +195,59 @@ public class ScheduleContextBuilderTest {
         // just to make sure queueNew is present
         Assert.assertEquals(queue.getQueueId(), queueNew.getQueueId());
     }
+    
+    @Test
+    public void test_stream_noalert_policies_generation() throws Exception {
+    	InMemMetadataServiceClient client = getSampleMetadataServiceWithNodataAlert();
+    	
+        ScheduleContextBuilder builder = new ScheduleContextBuilder(config, client);
+        IScheduleContext context = builder.buildContext();
+        
+        PolicyDefinition policyDefinition = null;
+        PolicyDefinition aggrPolicyDefinition = null;
+        for (Entry<String, PolicyDefinition> entry : context.getPolicies().entrySet()) {
+        	if (entry.getKey().endsWith("_nodata_alert")) {
+        		policyDefinition = entry.getValue();
+        		continue;
+        	}
+        	if (entry.getKey().endsWith("_aggregation_stream_policy")) {
+        		aggrPolicyDefinition = entry.getValue();
+        		continue;
+        	}
+        }
+        Assert.assertEquals(3, context.getPolicies().size());
+        
+        Assert.assertNotNull(policyDefinition);
+        Assert.assertEquals("nodataalert", policyDefinition.getDefinition().getType());
+        Assert.assertEquals("PT5S,dynamic,1," + COL1, policyDefinition.getDefinition().getValue());
+        
+        Assert.assertNotNull(aggrPolicyDefinition);
+        Assert.assertEquals("siddhi", aggrPolicyDefinition.getDefinition().getType());
+        
+        Kafka2TupleMetadata datasource = null;
+        for (Entry<String, Kafka2TupleMetadata> entry : context.getDataSourceMetadata().entrySet()) {
+        	if ("nodata_alert_aggregation_ds".equals(entry.getKey())) {
+        		datasource = entry.getValue();
+        		break;
+        	}
+        }
+        Assert.assertNotNull(datasource);
+        
+        String publishmentName = policyDefinition.getName() + "_publish";
+        Publishment publishment = null;
+        for (Entry<String, Publishment> entry : context.getPublishments().entrySet()) {
+        	if (publishmentName.equals(entry.getKey())) {
+        		publishment = entry.getValue();
+        		break;
+        	}
+        }
+        Assert.assertNotNull(publishment);
+    }
 
     @Test
     public void test_renamed_topologies() {
         InMemMetadataServiceClient client = getSampleMetadataService();
-        ScheduleContextBuilder builder = new ScheduleContextBuilder(client);
+        ScheduleContextBuilder builder = new ScheduleContextBuilder(config, client);
 
         IScheduleContext context = builder.buildContext();
         Assert.assertTrue(context.getPolicyAssignments().containsKey(TEST_POLICY_1));
@@ -226,6 +280,33 @@ public class ScheduleContextBuilderTest {
         client.addScheduleState(createScheduleState());
         return client;
     }
+    
+    public static InMemMetadataServiceClient getSampleMetadataServiceWithNodataAlert() {
+        InMemMetadataServiceClient client = new InMemMetadataServiceClient();
+        client.addTopology(createSampleTopology());
+        client.addDataSource(createKafka2TupleMetadata());
+        client.addPolicy(createPolicy());
+        client.addPublishment(createPublishment());
+        client.addStreamDefinition(createStreamDefinitionWithNodataAlert());
+        client.addScheduleState(createScheduleState());
+        return client;
+    }
+    
+    private static StreamDefinition createStreamDefinitionWithNodataAlert() {
+        StreamDefinition def = new StreamDefinition();
+        def.setStreamId(TEST_STREAM_DEF_1);
+        def.setDataSource(TEST_DATASOURCE_1);
+
+        StreamColumn col = new StreamColumn();
+        col.setName(COL1);
+        col.setRequired(true);
+        col.setType(Type.STRING);
+        col.setNodataExpression("PT5S,dynamic,1," + COL1);
+        def.getColumns().add(col);
+
+        return def;
+    }
+
 
     private static ScheduleState createScheduleState() {
         ScheduleState ss = new ScheduleState();

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/ebbaad09/eagle-core/eagle-alert-parent/eagle-alert/alert-coordinator/src/test/java/org/apache/alert/coordinator/TestGreedyScheduleCoordinator.java
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-alert-parent/eagle-alert/alert-coordinator/src/test/java/org/apache/alert/coordinator/TestGreedyScheduleCoordinator.java b/eagle-core/eagle-alert-parent/eagle-alert/alert-coordinator/src/test/java/org/apache/alert/coordinator/TestGreedyScheduleCoordinator.java
new file mode 100644
index 0000000..a86b13a
--- /dev/null
+++ b/eagle-core/eagle-alert-parent/eagle-alert/alert-coordinator/src/test/java/org/apache/alert/coordinator/TestGreedyScheduleCoordinator.java
@@ -0,0 +1,131 @@
+/*
+ * 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.alert.coordinator;
+
+import org.apache.eagle.alert.coordinator.ExclusiveExecutor;
+import org.apache.eagle.alert.utils.ZookeeperEmbedded;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+public class TestGreedyScheduleCoordinator {
+	
+	public static class ScheduleZkState {
+    	volatile boolean scheduleAcquired = false;
+        volatile boolean scheduleCompleted = false;
+    }
+
+	public static class GreedyScheduleCoordinator {
+		
+	    public int schedule(int input) {
+	    	ScheduleZkState scheduleZkState = new ScheduleZkState();
+	    	ExclusiveExecutor.Runnable exclusiveRunnable = new ExclusiveExecutor.Runnable() {
+				@Override
+				public void run() throws Exception {
+					scheduleZkState.scheduleAcquired = true;
+					
+					while (!scheduleZkState.scheduleCompleted) {
+						Thread.sleep(2000);
+					}
+				}
+	    	};
+	    	ExclusiveExecutor.execute("/alert/test", exclusiveRunnable);
+	    	int waitMaxTimes = 0;
+	    	while (waitMaxTimes < 90) { //about 3 minutes waiting
+	    		if (!scheduleZkState.scheduleAcquired) {
+	    			waitMaxTimes ++;
+	    			try {
+						Thread.sleep(2000);
+					} catch (InterruptedException e) {}
+	    			continue;
+	    		}
+	    		try {
+	    			return input;
+	    		} finally {
+	    			//schedule completed
+	    			scheduleZkState.scheduleCompleted = true;
+	    		}
+	    	}
+	    	throw new RuntimeException("Acquire greedy scheduler lock failed, please retry later");
+	    }
+		
+	}
+	
+	ZookeeperEmbedded zkEmbed;
+
+	@Before
+	public void setUp() throws Exception {
+		zkEmbed = new ZookeeperEmbedded(2181);
+		zkEmbed.start();
+
+		Thread.sleep(2000);
+	}
+
+	@After
+	public void tearDown() throws Exception {
+		zkEmbed.shutdown();
+	}
+	
+	@Test
+	public void testMain() throws Exception {
+		final GreedyScheduleCoordinator coordinator = new GreedyScheduleCoordinator();
+		
+		
+		new Thread(new Runnable() {
+
+			@Override
+			public void run() {
+				System.out.println("output: " + coordinator.schedule(1));
+				
+				try {
+					Thread.sleep(1000);
+				} catch (InterruptedException e) {}
+			}
+			
+		}).start();
+		
+		new Thread(new Runnable() {
+
+			@Override
+			public void run() {
+				System.out.println("output: " + coordinator.schedule(2));
+				
+				try {
+					Thread.sleep(1000);
+				} catch (InterruptedException e) {}
+			}
+			
+		}).start();
+		
+		new Thread(new Runnable() {
+
+			@Override
+			public void run() {
+				System.out.println("output: " + coordinator.schedule(3));
+				
+				try {
+					Thread.sleep(1000);
+				} catch (InterruptedException e) {}
+			}
+			
+		}).start();
+		
+		Thread.sleep(15000);
+	}
+	
+	
+}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/ebbaad09/eagle-core/eagle-alert-parent/eagle-alert/alert-coordinator/src/test/resources/application.conf
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-alert-parent/eagle-alert/alert-coordinator/src/test/resources/application.conf b/eagle-core/eagle-alert-parent/eagle-alert/alert-coordinator/src/test/resources/application.conf
index 363e661..1ef71a0 100644
--- a/eagle-core/eagle-alert-parent/eagle-alert/alert-coordinator/src/test/resources/application.conf
+++ b/eagle-core/eagle-alert-parent/eagle-alert/alert-coordinator/src/test/resources/application.conf
@@ -37,6 +37,15 @@
 		"metadataDynamicCheck" : {
 			"initDelayMillis" : 1000,
 			"delayMillis" : 30000
+		},
+		"kafkaProducer": {
+	  		"bootstrapServers": "localhost:9092"
+	  	},
+		"email": {
+			"sender": "eagle@eagle.com",
+			"recipients": "test@eagle.com",
+			"mailSmtpHost": "test.eagle.com",
+			"mailSmtpPort": "25"
 		}
 	}
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/ebbaad09/eagle-core/eagle-alert-parent/eagle-alert/alert-coordinator/src/test/resources/test-application.conf
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-alert-parent/eagle-alert/alert-coordinator/src/test/resources/test-application.conf b/eagle-core/eagle-alert-parent/eagle-alert/alert-coordinator/src/test/resources/test-application.conf
index 361d6d1..63be6a9 100644
--- a/eagle-core/eagle-alert-parent/eagle-alert/alert-coordinator/src/test/resources/test-application.conf
+++ b/eagle-core/eagle-alert-parent/eagle-alert/alert-coordinator/src/test/resources/test-application.conf
@@ -37,6 +37,15 @@
 		"metadataDynamicCheck" : {
 			"initDelayMillis" : 1000,
 			"delayMillis" : 30000
+		},
+		"kafkaProducer": {
+	  		"bootstrapServers": "localhost:9092"
+	  	},
+		"email": {
+			"sender": "eagle@eagle.com",
+			"recipients": "test@eagle.com",
+			"mailSmtpHost": "test.eagle.com",
+			"mailSmtpPort": "25"
 		}
 	}
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/ebbaad09/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/main/java/org/apache/eagle/alert/engine/evaluator/PolicyStreamHandlers.java
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/main/java/org/apache/eagle/alert/engine/evaluator/PolicyStreamHandlers.java b/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/main/java/org/apache/eagle/alert/engine/evaluator/PolicyStreamHandlers.java
index ef9caf0..1e7aacc 100644
--- a/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/main/java/org/apache/eagle/alert/engine/evaluator/PolicyStreamHandlers.java
+++ b/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/main/java/org/apache/eagle/alert/engine/evaluator/PolicyStreamHandlers.java
@@ -16,17 +16,17 @@
  */
 package org.apache.eagle.alert.engine.evaluator;
 
+import java.util.Map;
+
 import org.apache.eagle.alert.engine.coordinator.PolicyDefinition;
 import org.apache.eagle.alert.engine.coordinator.StreamDefinition;
 import org.apache.eagle.alert.engine.evaluator.absence.AbsencePolicyHandler;
 import org.apache.eagle.alert.engine.evaluator.impl.SiddhiPolicyHandler;
 import org.apache.eagle.alert.engine.evaluator.impl.SiddhiPolicyStateHandler;
-import org.apache.eagle.alert.engine.evaluator.nodata.NoDataPolicyHandler;
+import org.apache.eagle.alert.engine.evaluator.nodata.NoDataPolicyTimeBatchHandler;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.util.Map;
-
 /**
  * TODO/FIXME: to support multiple stage definition in single policy. The methods in this class is not good to understand now.(Hard code of 0/1).
  */
@@ -42,7 +42,8 @@ public class PolicyStreamHandlers {
         if (SIDDHI_ENGINE.equals(definition.getType())) {
             return new SiddhiPolicyHandler(sds, 0);// // FIXME: 8/2/16 
         } else if (NO_DATA_ALERT_ENGINE.equals(definition.getType())) {
-            return new NoDataPolicyHandler(sds);
+        	// no data for an entire stream won't trigger gap alert  (use local time & batch window instead)
+        	return new NoDataPolicyTimeBatchHandler(sds);
         } else if (ABSENCE_ALERT_ENGINE.equals(definition.getType())) {
             return new AbsencePolicyHandler(sds);
         } else if (CUSTOMIZED_ENGINE.equals(definition.getType())) {

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/ebbaad09/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/main/java/org/apache/eagle/alert/engine/evaluator/nodata/DistinctValuesInTimeBatchWindow.java
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/main/java/org/apache/eagle/alert/engine/evaluator/nodata/DistinctValuesInTimeBatchWindow.java b/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/main/java/org/apache/eagle/alert/engine/evaluator/nodata/DistinctValuesInTimeBatchWindow.java
new file mode 100644
index 0000000..357504e
--- /dev/null
+++ b/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/main/java/org/apache/eagle/alert/engine/evaluator/nodata/DistinctValuesInTimeBatchWindow.java
@@ -0,0 +1,128 @@
+/*
+ * 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.eagle.alert.engine.evaluator.nodata;
+
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.eagle.alert.engine.model.StreamEvent;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class DistinctValuesInTimeBatchWindow {
+
+	private static final Logger LOG = LoggerFactory.getLogger(DistinctValuesInTimeBatchWindow.class);
+	
+	private final ScheduledExecutorService scheduler = Executors.newScheduledThreadPool(1);
+	
+	// wisb (what is should be) set for expected full set value of multiple columns
+	@SuppressWarnings("rawtypes")
+	private volatile Set wisb = new HashSet();
+	
+	private NoDataPolicyTimeBatchHandler handler;
+	
+	/**
+	 * map from value to max timestamp for this value
+	 */
+	private Map<Object, Long> valueMaxTimeMap = new HashMap<>();
+	
+	private long startTime = -1;
+	private long nextEmitTime = -1;
+	private long timeInMilliSeconds;
+
+	public DistinctValuesInTimeBatchWindow(NoDataPolicyTimeBatchHandler handler, 
+			long timeInMilliSeconds, @SuppressWarnings("rawtypes") Set wisb) {
+		this.handler = handler;
+		this.timeInMilliSeconds = timeInMilliSeconds;
+		if (wisb != null) {
+			this.wisb = wisb;
+		}
+	}
+
+	public Map<Object, Long> distinctValues() {
+		return valueMaxTimeMap;
+	}
+	
+	public void send(StreamEvent event, Object value, long timestamp) {
+		synchronized(this) {
+			if (startTime < 0) {
+				startTime = System.currentTimeMillis();
+				
+				scheduler.scheduleAtFixedRate(new Runnable() {
+
+					@SuppressWarnings({ "unchecked", "rawtypes" })
+					@Override
+					public void run() {
+						try {
+							LOG.info("{}/{}: {}", startTime, nextEmitTime, valueMaxTimeMap.keySet());
+							synchronized (valueMaxTimeMap) {
+								boolean sendAlerts = false;
+								
+								if (nextEmitTime < 0) {
+									nextEmitTime = startTime + timeInMilliSeconds;
+								}
+								
+								if (System.currentTimeMillis() > nextEmitTime) {
+									startTime = nextEmitTime;
+									nextEmitTime += timeInMilliSeconds;
+									sendAlerts = true;
+								} else {
+									sendAlerts = false;
+								}
+								
+								if (sendAlerts) {
+									// alert
+									handler.compareAndEmit(wisb, distinctValues().keySet(), event);
+									LOG.info("alert for wiri: {} compares to wisb: {}", distinctValues().keySet(), wisb);
+									
+									if (distinctValues().keySet().size() > 0) {
+										wisb = new HashSet(distinctValues().keySet());
+									}
+									valueMaxTimeMap.clear();
+									LOG.info("Clear wiri & update wisb to {}", wisb);
+								}
+							}
+						} catch (Throwable t) {
+							LOG.error("failed to run batch window for gap alert", t);
+						}
+					}
+					
+				}, 0, timeInMilliSeconds / 2, TimeUnit.MILLISECONDS);
+			}
+		}
+		
+		if (valueMaxTimeMap.containsKey(value)) {
+			// remove that entry with old timestamp in timeSortedMap
+			long oldTime = valueMaxTimeMap.get(value);
+			if (oldTime >= timestamp) {
+				// no any effect as the new timestamp is equal or even less than
+				// old timestamp
+				return;
+			}
+		}
+		// update new timestamp in valueMaxTimeMap
+		valueMaxTimeMap.put(value, timestamp);
+		
+		LOG.info("sent: {} with start: {}/next: {}", valueMaxTimeMap.keySet(), startTime, nextEmitTime);
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/ebbaad09/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/main/java/org/apache/eagle/alert/engine/evaluator/nodata/NoDataPolicyTimeBatchHandler.java
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/main/java/org/apache/eagle/alert/engine/evaluator/nodata/NoDataPolicyTimeBatchHandler.java b/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/main/java/org/apache/eagle/alert/engine/evaluator/nodata/NoDataPolicyTimeBatchHandler.java
new file mode 100644
index 0000000..741fce4
--- /dev/null
+++ b/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/main/java/org/apache/eagle/alert/engine/evaluator/nodata/NoDataPolicyTimeBatchHandler.java
@@ -0,0 +1,168 @@
+/*
+ * 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.eagle.alert.engine.evaluator.nodata;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.eagle.alert.engine.Collector;
+import org.apache.eagle.alert.engine.coordinator.PolicyDefinition;
+import org.apache.eagle.alert.engine.coordinator.StreamDefinition;
+import org.apache.eagle.alert.engine.evaluator.PolicyHandlerContext;
+import org.apache.eagle.alert.engine.evaluator.PolicyStreamHandler;
+import org.apache.eagle.alert.engine.model.AlertStreamEvent;
+import org.apache.eagle.alert.engine.model.StreamEvent;
+import org.apache.eagle.alert.utils.TimePeriodUtils;
+import org.apache.storm.guava.base.Joiner;
+import org.joda.time.Period;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class NoDataPolicyTimeBatchHandler implements PolicyStreamHandler {
+	
+	private static final Logger LOG = LoggerFactory.getLogger(NoDataPolicyTimeBatchHandler.class);
+	private Map<String, StreamDefinition> sds;
+	
+	private volatile List<Integer> wisbFieldIndices = new ArrayList<>();
+	// reuse PolicyDefinition.defintion.value field to store full set of values
+	// separated by comma
+	private volatile PolicyDefinition policyDef;
+	private volatile Collector<AlertStreamEvent> collector;
+	private volatile PolicyHandlerContext context;
+	private volatile NoDataWisbType wisbType;
+	private volatile DistinctValuesInTimeBatchWindow distinctWindow;
+
+	public NoDataPolicyTimeBatchHandler(Map<String, StreamDefinition> sds){
+        this.sds = sds;
+    }
+
+	@Override
+	public void prepare(Collector<AlertStreamEvent> collector, PolicyHandlerContext context) throws Exception {
+		this.collector = collector;
+		this.context = context;
+		this.policyDef = context.getPolicyDefinition();
+		List<String> inputStreams = policyDef.getInputStreams();
+		// validate inputStreams has to contain only one stream
+		if (inputStreams.size() != 1)
+			throw new IllegalArgumentException("policy inputStream size has to be 1 for no data alert");
+		// validate outputStream has to contain only one stream
+		if (policyDef.getOutputStreams().size() != 1)
+			throw new IllegalArgumentException("policy outputStream size has to be 1 for no data alert");
+
+		String is = inputStreams.get(0);
+		StreamDefinition sd = sds.get(is);
+
+		String policyValue = policyDef.getDefinition().getValue();
+		// assume that no data alert policy value consists of "windowPeriod,
+		// type, numOfFields, f1_name, f2_name, f1_value, f2_value, f1_value,
+		// f2_value}
+		String[] segments = policyValue.split(",");
+		this.wisbType = NoDataWisbType.valueOf(segments[1]);
+		// for provided wisb values, need to parse, for dynamic wisb values, it
+		// is computed through a window
+		@SuppressWarnings("rawtypes")
+		Set wisbValues = null;
+		if (wisbType == NoDataWisbType.provided) {
+			wisbValues = new NoDataWisbProvidedParser().parse(segments);
+		}
+		long windowPeriod = TimePeriodUtils.getMillisecondsOfPeriod(Period.parse(segments[0]));
+		distinctWindow = new DistinctValuesInTimeBatchWindow(this, windowPeriod, wisbValues);
+		// populate wisb field names
+		int numOfFields = Integer.parseInt(segments[2]);
+		for (int i = 3; i < 3 + numOfFields; i++) {
+			String fn = segments[i];
+			wisbFieldIndices.add(sd.getColumnIndex(fn));
+		}
+	}
+
+	@Override
+	public void send(StreamEvent event) throws Exception {
+		Object[] data = event.getData();
+		
+		List<Object> columnValues = new ArrayList<>();
+		for (int i = 0; i < wisbFieldIndices.size(); i++) {
+			Object o = data[wisbFieldIndices.get(i)];
+			// convert value to string
+			columnValues.add(o.toString());
+		}
+		// use local timestamp rather than event timestamp
+		distinctWindow.send(event, columnValues, System.currentTimeMillis());
+		LOG.debug("event sent to window with wiri: {}", distinctWindow.distinctValues());
+	}
+	
+	@SuppressWarnings("rawtypes")
+	public void compareAndEmit(Set wisb, Set wiri, StreamEvent event) {
+		// compare with wisbValues if wisbValues are already there for dynamic
+		// type
+		Collection noDataValues = CollectionUtils.subtract(wisb, wiri);
+		LOG.debug("nodatavalues:" + noDataValues + ", wisb: " + wisb + ", wiri: " + wiri);
+		if (noDataValues != null && noDataValues.size() > 0) {
+			LOG.info("No data alert is triggered with no data values {} and wisb {}", noDataValues, wisb);
+			
+			String is = policyDef.getOutputStreams().get(0);
+			StreamDefinition sd = sds.get(is);
+			int timestampIndex = sd.getColumnIndex("timestamp");
+			int hostIndex = sd.getColumnIndex("host");
+			int originalStreamNameIndex = sd.getColumnIndex("originalStreamName");
+			
+			for (Object one : noDataValues) {
+				Object[] triggerEvent = new Object[sd.getColumns().size()];
+				for (int i = 0; i < sd.getColumns().size(); i ++) {
+					if (i == timestampIndex) {
+						triggerEvent[i] = System.currentTimeMillis();
+					} else if (i == hostIndex) {
+						triggerEvent[hostIndex] = ((List) one).get(0);
+					} else if (i == originalStreamNameIndex) {
+						triggerEvent[originalStreamNameIndex] = event.getStreamId();
+					} else if (sd.getColumns().size() < i) {
+						LOG.error("strema event data have different lenght compare to column definition!");
+					} else {
+						triggerEvent[i] = sd.getColumns().get(i).getDefaultValue();
+					}
+				}
+				AlertStreamEvent alertEvent = createAlertEvent(sd, event.getTimestamp(), triggerEvent);
+				LOG.info(String.format("Nodata alert %s generated and will be emitted", Joiner.on(",").join(triggerEvent)));
+				collector.emit(alertEvent);
+			}
+			
+		}
+	}
+
+	private AlertStreamEvent createAlertEvent(StreamDefinition sd, long timestamp, Object[] triggerEvent) {
+		AlertStreamEvent event = new AlertStreamEvent();
+		event.setTimestamp(timestamp);
+		event.setData(triggerEvent);
+		event.setStreamId(policyDef.getOutputStreams().get(0));
+		event.setPolicy(context.getPolicyDefinition());
+		if (this.context.getPolicyEvaluator() != null) {
+			event.setCreatedBy(context.getPolicyEvaluator().getName());
+		}
+		event.setCreatedTime(System.currentTimeMillis());
+		event.setSchema(sd);
+		return event;
+	}
+
+	@Override
+	public void close() throws Exception {
+
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/ebbaad09/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/main/java/org/apache/eagle/alert/engine/publisher/impl/JsonEventSerializer.java
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/main/java/org/apache/eagle/alert/engine/publisher/impl/JsonEventSerializer.java b/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/main/java/org/apache/eagle/alert/engine/publisher/impl/JsonEventSerializer.java
index bf2a954..f30bf8f 100644
--- a/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/main/java/org/apache/eagle/alert/engine/publisher/impl/JsonEventSerializer.java
+++ b/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/main/java/org/apache/eagle/alert/engine/publisher/impl/JsonEventSerializer.java
@@ -43,7 +43,7 @@ public class JsonEventSerializer implements IEventSerializer {
     public Object serialize(AlertStreamEvent event) {
         String result = streamEventToJson(event);
         if (LOG.isDebugEnabled()) {
-            LOG.debug("serialized alert event : ", result);
+            LOG.debug("serialized alert event : {}", result);
         }
         return result;
     }

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/ebbaad09/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/main/java/org/apache/eagle/alert/engine/siddhi/extension/AttributeCollectWithDistinctAggregator.java
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/main/java/org/apache/eagle/alert/engine/siddhi/extension/AttributeCollectWithDistinctAggregator.java b/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/main/java/org/apache/eagle/alert/engine/siddhi/extension/AttributeCollectWithDistinctAggregator.java
new file mode 100644
index 0000000..43400c7
--- /dev/null
+++ b/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/main/java/org/apache/eagle/alert/engine/siddhi/extension/AttributeCollectWithDistinctAggregator.java
@@ -0,0 +1,124 @@
+/*
+ * 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.eagle.alert.engine.siddhi.extension;
+
+import java.util.LinkedList;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.wso2.siddhi.core.config.ExecutionPlanContext;
+import org.wso2.siddhi.core.executor.ExpressionExecutor;
+import org.wso2.siddhi.core.query.selector.attribute.aggregator.AttributeAggregator;
+import org.wso2.siddhi.query.api.definition.Attribute;
+import org.wso2.siddhi.query.api.definition.Attribute.Type;
+
+import com.google.common.collect.ImmutableList;
+
+public class AttributeCollectWithDistinctAggregator extends AttributeAggregator {
+
+    private static final Logger LOG = LoggerFactory.getLogger(AttributeCollectAggregator.class);
+
+    private LinkedList<Object> value;
+
+    public AttributeCollectWithDistinctAggregator() {
+        value = new LinkedList<Object>();
+    }
+
+    @Override
+    public void start() {
+    }
+
+    @Override
+    public void stop() {
+    }
+
+    @Override
+    public Object[] currentState() {
+        return value.toArray();
+    }
+
+    @Override
+    public void restoreState(Object[] arg0) {
+        value = new LinkedList<Object>();
+        if (arg0 != null) {
+            for (Object o : arg0) {
+                value.add(o);
+            }
+        }
+    }
+
+    @Override
+    public Type getReturnType() {
+        return Attribute.Type.OBJECT;
+    }
+
+    @Override
+    protected void init(ExpressionExecutor[] arg0, ExecutionPlanContext arg1) {
+        // TODO: Support max of elements?
+    }
+
+    @Override
+    public Object processAdd(Object arg0) {
+    	// AttributeAggregator.process is already synchronized
+    	if (value.contains(arg0)) {
+    		value.remove(arg0);
+    	}
+    	value.add(arg0);
+        if (LOG.isDebugEnabled()) {
+            LOG.debug("processAdd: current values are : " + value);
+        }
+        return ImmutableList.copyOf(value);
+    }
+
+    @Override
+    public Object processAdd(Object[] arg0) {
+    	// AttributeAggregator.process is already synchronized
+    	if (value.contains(arg0)) {
+    		value.remove(arg0);
+    	}
+    	value.add(arg0);
+        if (LOG.isDebugEnabled()) {
+            LOG.debug("processAdd: current values are : " + value);
+        }
+        return ImmutableList.copyOf(value);
+    }
+
+    // / NOTICE: non O(1)
+    @Override
+    public Object processRemove(Object arg0) {
+        value.remove(arg0);
+        if (LOG.isDebugEnabled()) {
+            LOG.debug("processRemove: current values are : " + value);
+        }
+        return ImmutableList.copyOf(value);
+    }
+
+    // / NOTICE: non O(1)
+    @Override
+    public Object processRemove(Object[] arg0) {
+        value.remove(arg0);
+        LOG.info("processRemove: current values are : " + value);
+        return ImmutableList.copyOf(value);
+    }
+
+    @Override
+    public Object reset() {
+        value.clear();
+        return value;
+    }
+    
+}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/ebbaad09/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/main/resources/eagle.siddhiext
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/main/resources/eagle.siddhiext b/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/main/resources/eagle.siddhiext
index 4ce9805..16569a4 100644
--- a/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/main/resources/eagle.siddhiext
+++ b/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/main/resources/eagle.siddhiext
@@ -16,3 +16,4 @@
 #
 
 collect=org.apache.eagle.alert.engine.siddhi.extension.AttributeCollectAggregator
+collectWithDistinct=org.apache.eagle.alert.engine.siddhi.extension.AttributeCollectWithDistinctAggregator
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/ebbaad09/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/test/java/org/apache/eagle/alert/engine/nodata/TestDistinctValuesInTimeBatchWindow.java
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/test/java/org/apache/eagle/alert/engine/nodata/TestDistinctValuesInTimeBatchWindow.java b/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/test/java/org/apache/eagle/alert/engine/nodata/TestDistinctValuesInTimeBatchWindow.java
new file mode 100644
index 0000000..72ef02b
--- /dev/null
+++ b/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/test/java/org/apache/eagle/alert/engine/nodata/TestDistinctValuesInTimeBatchWindow.java
@@ -0,0 +1,89 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.eagle.alert.engine.nodata;
+
+import static org.mockito.Matchers.anyObject;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.times;
+
+import org.apache.eagle.alert.engine.evaluator.nodata.DistinctValuesInTimeBatchWindow;
+import org.apache.eagle.alert.engine.evaluator.nodata.NoDataPolicyTimeBatchHandler;
+import org.apache.eagle.alert.engine.model.StreamEvent;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+public class TestDistinctValuesInTimeBatchWindow {
+
+	private static final String inputStream = "testInputStream";
+
+	private NoDataPolicyTimeBatchHandler handler;
+
+	@Before
+	public void setup() {
+		handler = mock(NoDataPolicyTimeBatchHandler.class);
+	}
+
+	@After
+	public void teardown() {
+	}
+
+	@Test
+	public void testNormal() throws Exception {
+		// wisb is null since it is dynamic mode
+		DistinctValuesInTimeBatchWindow window = new DistinctValuesInTimeBatchWindow(handler, 5 * 1000, null);
+
+		long now = System.currentTimeMillis();
+
+		// handler.compareAndEmit(anyObject(), anyObject(), anyObject());
+
+		// event time
+		sendEventToWindow(window, now, "host1", 95.5);
+
+		Thread.sleep(6000);
+
+		sendEventToWindow(window, now, "host1", 91.0);
+		sendEventToWindow(window, now, "host2", 95.5);
+		sendEventToWindow(window, now, "host2", 97.1);
+
+		Thread.sleep(3000);
+
+		sendEventToWindow(window, now, "host1", 90.7);
+
+		Thread.sleep(4000);
+
+		sendEventToWindow(window, now, "host1", 90.7);
+		
+		Thread.sleep(3000);
+
+		verify(handler, times(3)).compareAndEmit(anyObject(), anyObject(), anyObject());
+	}
+
+	private void sendEventToWindow(DistinctValuesInTimeBatchWindow window, long ts, String host, double value) {
+		window.send(buildStreamEvent(ts, host, value), host, ts);
+	}
+
+	private StreamEvent buildStreamEvent(long ts, String host, double value) {
+		StreamEvent e = new StreamEvent();
+		e.setData(new Object[] { ts, host, value });
+		e.setStreamId(inputStream);
+		e.setTimestamp(ts);
+		return e;
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/ebbaad09/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/test/java/org/apache/eagle/alert/engine/nodata/TestNoDataPolicyTimeBatchHandler.java
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/test/java/org/apache/eagle/alert/engine/nodata/TestNoDataPolicyTimeBatchHandler.java b/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/test/java/org/apache/eagle/alert/engine/nodata/TestNoDataPolicyTimeBatchHandler.java
new file mode 100644
index 0000000..02d19b4
--- /dev/null
+++ b/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/test/java/org/apache/eagle/alert/engine/nodata/TestNoDataPolicyTimeBatchHandler.java
@@ -0,0 +1,158 @@
+/*
+ * 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.eagle.alert.engine.nodata;
+
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.eagle.alert.engine.Collector;
+import org.apache.eagle.alert.engine.coordinator.PolicyDefinition;
+import org.apache.eagle.alert.engine.coordinator.StreamColumn;
+import org.apache.eagle.alert.engine.coordinator.StreamDefinition;
+import org.apache.eagle.alert.engine.evaluator.PolicyHandlerContext;
+import org.apache.eagle.alert.engine.evaluator.nodata.NoDataPolicyTimeBatchHandler;
+import org.apache.eagle.alert.engine.model.AlertStreamEvent;
+import org.apache.eagle.alert.engine.model.StreamEvent;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class TestNoDataPolicyTimeBatchHandler {
+
+	private static final Logger LOG = LoggerFactory.getLogger(TestNoDataPolicyTimeBatchHandler.class);
+	
+	private static final String inputStream = "testInputStream";
+	private static final String outputStream = "testOutputStream";
+
+	@Before
+	public void setup() {
+	}
+
+	@SuppressWarnings("unchecked")
+	@Test
+	public void testDynamic1() throws Exception {
+		Map<String, StreamDefinition> sds = new HashMap<>();
+		sds.put("testInputStream", buildStreamDef());
+		sds.put("testOutputStream", buildOutputStreamDef());
+		NoDataPolicyTimeBatchHandler handler = new NoDataPolicyTimeBatchHandler(sds);
+
+		PolicyHandlerContext context = new PolicyHandlerContext();
+		context.setPolicyDefinition(buildPolicyDef_dynamic());
+		handler.prepare(new TestCollector(), context);
+
+		long now = System.currentTimeMillis();
+		
+		handler.send(buildStreamEvt(now, "host1", 12.5));
+		
+		Thread.sleep(2000);
+		
+		handler.send(buildStreamEvt(now, "host2", 12.6));
+		handler.send(buildStreamEvt(now, "host1", 20.9));
+		handler.send(buildStreamEvt(now, "host2", 22.1));
+		handler.send(buildStreamEvt(now, "host2", 22.1));
+		
+		Thread.sleep(5000);
+		
+		handler.send(buildStreamEvt(now, "host2", 22.1));
+		handler.send(buildStreamEvt(now, "host2", 22.3));
+		
+		Thread.sleep(5000);
+		
+		handler.send(buildStreamEvt(now, "host2", 22.9));
+		handler.send(buildStreamEvt(now, "host1", 41.6));
+		handler.send(buildStreamEvt(now, "host2", 45.6));
+		
+		Thread.sleep(1000);
+	}
+	
+	@SuppressWarnings("rawtypes")
+    private static class TestCollector implements Collector{
+        @Override
+        public void emit(Object o) {
+            AlertStreamEvent e = (AlertStreamEvent)o;
+            Object[] data = e.getData();
+            
+            LOG.info("alert data: {}, {}", data[1], data[0]);
+            
+            Assert.assertEquals("host1", data[1]);
+        }
+    }
+
+	private PolicyDefinition buildPolicyDef_dynamic() {
+		PolicyDefinition pd = new PolicyDefinition();
+		PolicyDefinition.Definition def = new PolicyDefinition.Definition();
+		def.setValue("PT5S,dynamic,1,host");
+		def.setType("nodataalert");
+		pd.setDefinition(def);
+		pd.setInputStreams(Arrays.asList(inputStream));
+		pd.setOutputStreams(Arrays.asList(outputStream));
+		pd.setName("nodataalert-test");
+		return pd;
+	}
+
+	private StreamDefinition buildStreamDef() {
+		StreamDefinition sd = new StreamDefinition();
+		StreamColumn tsColumn = new StreamColumn();
+		tsColumn.setName("timestamp");
+		tsColumn.setType(StreamColumn.Type.LONG);
+
+		StreamColumn hostColumn = new StreamColumn();
+		hostColumn.setName("host");
+		hostColumn.setType(StreamColumn.Type.STRING);
+
+		StreamColumn valueColumn = new StreamColumn();
+		valueColumn.setName("value");
+		valueColumn.setType(StreamColumn.Type.DOUBLE);
+
+		sd.setColumns(Arrays.asList(tsColumn, hostColumn, valueColumn));
+		sd.setDataSource("testDataSource");
+		sd.setStreamId("testInputStream");
+		return sd;
+	}
+	
+	private StreamDefinition buildOutputStreamDef() {
+		StreamDefinition sd = new StreamDefinition();
+		StreamColumn tsColumn = new StreamColumn();
+		tsColumn.setName("timestamp");
+		tsColumn.setType(StreamColumn.Type.LONG);
+
+		StreamColumn hostColumn = new StreamColumn();
+		hostColumn.setName("host");
+		hostColumn.setType(StreamColumn.Type.STRING);
+
+		StreamColumn valueColumn = new StreamColumn();
+		valueColumn.setName("originalStreamName");
+		valueColumn.setType(StreamColumn.Type.STRING);
+
+		sd.setColumns(Arrays.asList(tsColumn, hostColumn, valueColumn));
+		sd.setDataSource("testDataSource");
+		sd.setStreamId("testOutputStream");
+		return sd;
+	}
+
+	private StreamEvent buildStreamEvt(long ts, String host, double value) {
+		StreamEvent e = new StreamEvent();
+		e.setData(new Object[] { ts, host, value });
+		e.setStreamId(inputStream);
+		e.setTimestamp(ts);
+		return e;
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/ebbaad09/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/test/resources/eagle.siddhiext
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/test/resources/eagle.siddhiext b/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/test/resources/eagle.siddhiext
new file mode 100644
index 0000000..4ce9805
--- /dev/null
+++ b/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/test/resources/eagle.siddhiext
@@ -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.
+#
+
+collect=org.apache.eagle.alert.engine.siddhi.extension.AttributeCollectAggregator

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/ebbaad09/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/test/resources/simple/application-integration.conf
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/test/resources/simple/application-integration.conf b/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/test/resources/simple/application-integration.conf
index 73e5b30..3f2fbc3 100644
--- a/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/test/resources/simple/application-integration.conf
+++ b/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/test/resources/simple/application-integration.conf
@@ -18,12 +18,12 @@
     "numOfTotalWorkers": 20,
     "numOfSpoutTasks" : 1,
     "numOfRouterBolts" : 4,
-    "numOfAlertBolts" : 10,
+    "numOfAlertBolts" : 20,
     "numOfPublishTasks" : 1,
     "localMode" : "true"
   },
   "spout" : {
-    "kafkaBrokerZkQuorum": "localhost:2181",
+    "kafkaBrokerZkQuorum": "127.0.0.1:2181",
     "kafkaBrokerZkBasePath": "/brokers",
     "stormKafkaUseSameZkQuorumWithKafkaBroker": true,
     "stormKafkaTransactionZkQuorum": "",
@@ -33,7 +33,7 @@
     "stormKafkaFetchSizeBytes": 1048586,
   },
   "zkConfig" : {
-    "zkQuorum" : "localhost:2181",
+    "zkQuorum" : "127.0.0.1:2181",
     "zkRoot" : "/alert",
     "zkSessionTimeoutMs" : 10000,
     "connectionTimeoutMs" : 10000,
@@ -55,6 +55,12 @@
   	"context" : "/rest"
   },
   "kafkaProducer": {
-  	"bootstrapServers": "localhost:9092"
-  }
+  	"bootstrapServers": "127.0.0.1:9092"
+  },
+	"email": {
+		"sender": "eagle@eagle.com",
+		"recipients": "test@eagle.com",
+		"mail.smtp.host": "test.eagle.com",
+		"mail.smtp.port": "25"
+	}
 }

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/ebbaad09/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/test/resources/simple/topologies.json
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/test/resources/simple/topologies.json b/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/test/resources/simple/topologies.json
index 411cc48..984fcdb 100644
--- a/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/test/resources/simple/topologies.json
+++ b/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/test/resources/simple/topologies.json
@@ -2,7 +2,7 @@
 {
 	"name": "alertUnitTopology_1",
 	"numOfSpout":1,
-	"numOfAlertBolt": 10,
+	"numOfAlertBolt": 20,
 	"numOfGroupBolt": 4,
 	"spoutId": "alertEngineSpout",
 	"groupNodeIds" : [
@@ -21,7 +21,17 @@
 		"alertBolt6",
 		"alertBolt7",
 		"alertBolt8",
-		"alertBolt9"
+		"alertBolt9",
+		"alertBolt10",
+		"alertBolt11",
+		"alertBolt12",
+		"alertBolt13",
+		"alertBolt14",
+		"alertBolt15",
+		"alertBolt16",
+		"alertBolt17",
+		"alertBolt18",
+		"alertBolt19"
 	],
 	"pubBoltId" : "alertPublishBolt",
 	"spoutParallelism": 1,

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/ebbaad09/eagle-core/eagle-alert-parent/eagle-alert/alert-service/src/main/resources/application.conf
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-alert-parent/eagle-alert/alert-service/src/main/resources/application.conf b/eagle-core/eagle-alert-parent/eagle-alert/alert-service/src/main/resources/application.conf
index 97edc5a..72a731a 100644
--- a/eagle-core/eagle-alert-parent/eagle-alert/alert-service/src/main/resources/application.conf
+++ b/eagle-core/eagle-alert-parent/eagle-alert/alert-service/src/main/resources/application.conf
@@ -20,7 +20,7 @@
 		"policyDefaultParallelism" : 5,
 		"boltLoadUpbound": 0.8,
 		"topologyLoadUpbound" : 0.8,
-		"numOfAlertBoltsPerTopology" : 5,
+		"numOfAlertBoltsPerTopology" : 20,
 		"zkConfig" : {
 			"zkQuorum" : "127.0.0.1:2181",
 			"zkRoot" : "/alert",
@@ -37,6 +37,15 @@
 		"metadataDynamicCheck" : {
 			"initDelayMillis" : 1000,
 			"delayMillis" : 30000
+		},
+		"kafkaProducer": {
+			"bootstrapServers": "127.0.0.1:9092"
+		},
+		"email": {
+			"sender": "eagle@eagle.com",
+			"recipients": "test@eagle.com",
+			"mailSmtpHost": "test.eagle.com",
+			"mailSmtpPort": "25"
 		}
 	},
 	"datastore": {



[03/52] [abbrv] incubator-eagle git commit: [EAGLE-461] Convert MR history app with new app framework

Posted by yo...@apache.org.
[EAGLE-461] Convert MR history app with new app framework

https://issues.apache.org/jira/browse/EAGLE-461

Author: Hao Chen <ha...@apache.org>

Closes #380 from haoch/EAGLE-461.


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

Branch: refs/heads/master
Commit: 0bde482be40f208ba944d32ece23533714c87133
Parents: b52405f
Author: Hao Chen <ha...@apache.org>
Authored: Wed Aug 24 15:48:58 2016 +0800
Committer: Hao Chen <ha...@apache.org>
Committed: Wed Aug 24 15:48:58 2016 +0800

----------------------------------------------------------------------
 eagle-jpm/eagle-jpm-mr-history/pom.xml          |   5 +
 .../jpm/mr/history/MRHistoryJobApplication.java |  75 +++
 .../MRHistoryJobApplicationProvider.java        |  26 +
 .../jpm/mr/history/MRHistoryJobConfig.java      | 208 +++++++
 .../eagle/jpm/mr/history/MRHistoryJobMain.java  |  70 +--
 .../jpm/mr/history/common/JHFConfigManager.java | 182 ------
 .../crawler/DefaultJHFInputStreamCallback.java  |   8 +-
 .../history/crawler/JHFCrawlerDriverImpl.java   |   6 +-
 .../mr/history/crawler/JobHistoryDAOImpl.java   |   2 +-
 .../HistoryJobEntityCreationListener.java       |   6 +-
 .../HistoryJobEntityLifecycleListener.java      |   5 +-
 .../jpm/mr/history/parser/ImportException.java  |   2 -
 .../mr/history/parser/JHFEventReaderBase.java   | 302 +++++-----
 .../mr/history/parser/JHFMRVer1EventReader.java |  30 +-
 .../jpm/mr/history/parser/JHFMRVer1Parser.java  | 319 ++++++-----
 .../parser/JHFMRVer1PerLineListener.java        |  14 +-
 .../mr/history/parser/JHFMRVer2EventReader.java | 553 +++++++++++++------
 .../jpm/mr/history/parser/JHFMRVer2Parser.java  |  55 +-
 .../jpm/mr/history/parser/JHFParserBase.java    |   3 +-
 .../jpm/mr/history/parser/JHFParserFactory.java |  44 +-
 .../parser/JHFWriteNotCompletedException.java   |   6 +-
 ...JobConfigurationCreationServiceListener.java |  24 +-
 .../JobEntityCreationEagleServiceListener.java  |  45 +-
 .../parser/JobEntityCreationPublisher.java      |   9 +-
 .../parser/JobEntityLifecycleAggregator.java    |  37 +-
 .../mr/history/parser/MRErrorClassifier.java    |  17 +-
 .../jpm/mr/history/parser/RecordTypes.java      |   5 +-
 .../parser/TaskAttemptCounterListener.java      |  54 +-
 .../mr/history/parser/TaskFailureListener.java  | 142 ++---
 .../jpm/mr/history/storm/JobHistorySpout.java   | 122 ++--
 .../mr/history/zkres/JobHistoryZKStateLCM.java  |   8 +
 .../history/zkres/JobHistoryZKStateManager.java |  90 +--
 ....history.MRHistoryJobApplicationProvider.xml | 154 ++++++
 ...org.apache.eagle.app.spi.ApplicationProvider |  16 +
 .../src/main/resources/application.conf         |  20 +-
 .../MRHistoryJobApplicationProviderTest.java    |  33 ++
 .../mr/history/MRHistoryJobApplicationTest.java |  27 +
 37 files changed, 1633 insertions(+), 1091 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/0bde482b/eagle-jpm/eagle-jpm-mr-history/pom.xml
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-mr-history/pom.xml b/eagle-jpm/eagle-jpm-mr-history/pom.xml
index bfd4cf2..1ffda6a 100644
--- a/eagle-jpm/eagle-jpm-mr-history/pom.xml
+++ b/eagle-jpm/eagle-jpm-mr-history/pom.xml
@@ -104,6 +104,11 @@
             <artifactId>hadoop-mapreduce-client-core</artifactId>
             <version>${hadoop.version}</version>
         </dependency>
+        <dependency>
+            <groupId>org.apache.eagle</groupId>
+            <artifactId>eagle-app-base</artifactId>
+            <version>${project.version}</version>
+        </dependency>
     </dependencies>
 
     <build>

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/0bde482b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/MRHistoryJobApplication.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/MRHistoryJobApplication.java b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/MRHistoryJobApplication.java
new file mode 100644
index 0000000..08607a1
--- /dev/null
+++ b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/MRHistoryJobApplication.java
@@ -0,0 +1,75 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.eagle.jpm.mr.history;
+
+import org.apache.eagle.app.StormApplication;
+import org.apache.eagle.app.environment.impl.StormEnvironment;
+import org.apache.eagle.jpm.mr.history.crawler.JobHistoryContentFilter;
+import org.apache.eagle.jpm.mr.history.crawler.JobHistoryContentFilterBuilder;
+import org.apache.eagle.jpm.mr.history.storm.JobHistorySpout;
+import org.apache.eagle.jpm.util.Constants;
+
+import backtype.storm.generated.StormTopology;
+import backtype.storm.topology.TopologyBuilder;
+import com.typesafe.config.Config;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.regex.Pattern;
+
+public class MRHistoryJobApplication extends StormApplication {
+    @Override
+    public StormTopology execute(Config config, StormEnvironment environment) {
+        //1. trigger init conf
+        MRHistoryJobConfig appConfig = MRHistoryJobConfig.getInstance(config);
+        com.typesafe.config.Config jhfAppConf = appConfig.getConfig();
+
+        //2. init JobHistoryContentFilter
+        final JobHistoryContentFilterBuilder builder = JobHistoryContentFilterBuilder.newBuilder().acceptJobFile().acceptJobConfFile();
+        String[] confKeyPatternsSplit = jhfAppConf.getString("MRConfigureKeys.jobConfigKey").split(",");
+        List<String> confKeyPatterns = new ArrayList<>(confKeyPatternsSplit.length);
+        for (String confKeyPattern : confKeyPatternsSplit) {
+            confKeyPatterns.add(confKeyPattern.trim());
+        }
+        confKeyPatterns.add(Constants.JobConfiguration.CASCADING_JOB);
+        confKeyPatterns.add(Constants.JobConfiguration.HIVE_JOB);
+        confKeyPatterns.add(Constants.JobConfiguration.PIG_JOB);
+        confKeyPatterns.add(Constants.JobConfiguration.SCOOBI_JOB);
+
+        String jobNameKey = jhfAppConf.getString("MRConfigureKeys.jobNameKey");
+        builder.setJobNameKey(jobNameKey);
+
+        for (String key : confKeyPatterns) {
+            builder.includeJobKeyPatterns(Pattern.compile(key));
+        }
+        JobHistoryContentFilter filter = builder.build();
+        //3. init topology
+        TopologyBuilder topologyBuilder = new TopologyBuilder();
+        String spoutName = "mrHistoryJobExecutor";
+        int parallelism = jhfAppConf.getInt("envContextConfig.parallelismConfig." + spoutName);
+        int tasks = jhfAppConf.getInt("envContextConfig.tasks." + spoutName);
+        if (parallelism > tasks) {
+            parallelism = tasks;
+        }
+        topologyBuilder.setSpout(
+            spoutName,
+            new JobHistorySpout(filter, appConfig),
+            parallelism
+        ).setNumTasks(tasks);
+        return topologyBuilder.createTopology();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/0bde482b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/MRHistoryJobApplicationProvider.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/MRHistoryJobApplicationProvider.java b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/MRHistoryJobApplicationProvider.java
new file mode 100644
index 0000000..9aa1c61
--- /dev/null
+++ b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/MRHistoryJobApplicationProvider.java
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.eagle.jpm.mr.history;
+
+import org.apache.eagle.app.spi.AbstractApplicationProvider;
+
+public class MRHistoryJobApplicationProvider extends AbstractApplicationProvider<MRHistoryJobApplication> {
+    @Override
+    public MRHistoryJobApplication getApplication() {
+        return new MRHistoryJobApplication();
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/0bde482b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/MRHistoryJobConfig.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/MRHistoryJobConfig.java b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/MRHistoryJobConfig.java
new file mode 100644
index 0000000..ae86904
--- /dev/null
+++ b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/MRHistoryJobConfig.java
@@ -0,0 +1,208 @@
+/*
+ * 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.eagle.jpm.mr.history;
+
+import org.apache.eagle.common.config.ConfigOptionParser;
+import org.apache.eagle.jpm.util.DefaultJobIdPartitioner;
+import org.apache.eagle.jpm.util.JobIdPartitioner;
+
+import com.typesafe.config.Config;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Serializable;
+
+public class MRHistoryJobConfig implements Serializable {
+    private static final Logger LOG = LoggerFactory.getLogger(MRHistoryJobConfig.class);
+
+    private static final String JOB_CONFIGURE_KEY_CONF_FILE = "JobConfigKeys.conf";
+
+    public String getEnv() {
+        return env;
+    }
+
+    private String env;
+
+    public ZKStateConfig getZkStateConfig() {
+        return zkStateConfig;
+    }
+
+    private ZKStateConfig zkStateConfig;
+
+    public JobHistoryEndpointConfig getJobHistoryEndpointConfig() {
+        return jobHistoryEndpointConfig;
+    }
+
+    private JobHistoryEndpointConfig jobHistoryEndpointConfig;
+
+    public ControlConfig getControlConfig() {
+        return controlConfig;
+    }
+
+    private ControlConfig controlConfig;
+
+    public JobExtractorConfig getJobExtractorConfig() {
+        return jobExtractorConfig;
+    }
+
+    private JobExtractorConfig jobExtractorConfig;
+
+    public EagleServiceConfig getEagleServiceConfig() {
+        return eagleServiceConfig;
+    }
+
+    private EagleServiceConfig eagleServiceConfig;
+
+    public Config getConfig() {
+        return config;
+    }
+
+    private Config config;
+
+    public static class ZKStateConfig implements Serializable {
+        public String zkQuorum;
+        public String zkRoot;
+        public int zkSessionTimeoutMs;
+        public int zkRetryTimes;
+        public int zkRetryInterval;
+        public String zkPort;
+    }
+
+    public static class JobHistoryEndpointConfig implements Serializable {
+        public String nnEndpoint;
+        public String basePath;
+        public boolean pathContainsJobTrackerName;
+        public String jobTrackerName;
+        public String principal;
+        public String keyTab;
+    }
+
+    public static class ControlConfig implements Serializable {
+        public boolean dryRun;
+        public Class<? extends JobIdPartitioner> partitionerCls;
+        public boolean zeroBasedMonth;
+        public String timeZone;
+    }
+
+    public static class JobExtractorConfig implements Serializable {
+        public String site;
+        public String mrVersion;
+        public int readTimeoutSeconds;
+    }
+
+    public static class EagleServiceConfig implements Serializable {
+        public String eagleServiceHost;
+        public int eagleServicePort;
+        public String username;
+        public String password;
+    }
+
+    private static MRHistoryJobConfig manager = new MRHistoryJobConfig();
+
+    /**
+     * As this is singleton object and constructed while this class is being initialized,
+     * so any exception within this constructor will be wrapped with java.lang.ExceptionInInitializerError.
+     * And this is unrecoverable and hard to troubleshooting.
+     */
+    private MRHistoryJobConfig() {
+        this.zkStateConfig = new ZKStateConfig();
+        this.jobHistoryEndpointConfig = new JobHistoryEndpointConfig();
+        this.controlConfig = new ControlConfig();
+        this.jobExtractorConfig = new JobExtractorConfig();
+        this.eagleServiceConfig = new EagleServiceConfig();
+    }
+
+    public static MRHistoryJobConfig getInstance(String[] args) {
+        manager.init(args);
+        return manager;
+    }
+
+    public static MRHistoryJobConfig getInstance(Config config) {
+        manager.init(config);
+        return manager;
+    }
+
+    /**
+     * read configuration file and load hbase config etc.
+     */
+    private void init(String[] args) {
+        // TODO: Probably we can remove the properties file path check in future
+        try {
+            LOG.info("Loading from configuration file");
+            init(new ConfigOptionParser().load(args));
+        } catch (Exception e) {
+            LOG.error("failed to load config");
+        }
+    }
+
+    /**
+     * read configuration file and load hbase config etc.
+     */
+    private void init(Config config) {
+        this.config = config;
+        this.env = config.getString("envContextConfig.env");
+        //parse eagle job extractor
+        this.jobExtractorConfig.site = config.getString("jobExtractorConfig.site");
+        this.jobExtractorConfig.mrVersion = config.getString("jobExtractorConfig.mrVersion");
+        this.jobExtractorConfig.readTimeoutSeconds = config.getInt("jobExtractorConfig.readTimeOutSeconds");
+        //parse eagle zk
+        this.zkStateConfig.zkQuorum = config.getString("dataSourceConfig.zkQuorum");
+        this.zkStateConfig.zkPort = config.getString("dataSourceConfig.zkPort");
+        this.zkStateConfig.zkSessionTimeoutMs = config.getInt("dataSourceConfig.zkSessionTimeoutMs");
+        this.zkStateConfig.zkRetryTimes = config.getInt("dataSourceConfig.zkRetryTimes");
+        this.zkStateConfig.zkRetryInterval = config.getInt("dataSourceConfig.zkRetryInterval");
+        this.zkStateConfig.zkRoot = config.getString("dataSourceConfig.zkRoot");
+
+        //parse job history endpoint
+        this.jobHistoryEndpointConfig.basePath = config.getString("dataSourceConfig.basePath");
+        this.jobHistoryEndpointConfig.jobTrackerName = config.getString("dataSourceConfig.jobTrackerName");
+        this.jobHistoryEndpointConfig.nnEndpoint = config.getString("dataSourceConfig.nnEndpoint");
+        this.jobHistoryEndpointConfig.pathContainsJobTrackerName = config.getBoolean("dataSourceConfig.pathContainsJobTrackerName");
+        this.jobHistoryEndpointConfig.principal = config.getString("dataSourceConfig.principal");
+        this.jobHistoryEndpointConfig.keyTab = config.getString("dataSourceConfig.keytab");
+
+        //parse control config
+        this.controlConfig.dryRun = config.getBoolean("dataSourceConfig.dryRun");
+        try {
+            this.controlConfig.partitionerCls = (Class<? extends JobIdPartitioner>) Class.forName(config.getString("dataSourceConfig.partitionerCls"));
+            assert this.controlConfig.partitionerCls != null;
+        } catch (Exception e) {
+            LOG.warn("can not initialize partitioner class, use org.apache.eagle.jpm.util.DefaultJobIdPartitioner", e);
+            this.controlConfig.partitionerCls = DefaultJobIdPartitioner.class;
+        } finally {
+            LOG.info("Loaded partitioner class: {}", this.controlConfig.partitionerCls);
+        }
+        this.controlConfig.zeroBasedMonth = config.getBoolean("dataSourceConfig.zeroBasedMonth");
+        this.controlConfig.timeZone = config.getString("dataSourceConfig.timeZone");
+
+        // parse eagle service endpoint
+        this.eagleServiceConfig.eagleServiceHost = config.getString("eagleProps.eagleService.host");
+        String port = config.getString("eagleProps.eagleService.port");
+        this.eagleServiceConfig.eagleServicePort = (port == null ? 8080 : Integer.parseInt(port));
+        this.eagleServiceConfig.username = config.getString("eagleProps.eagleService.username");
+        this.eagleServiceConfig.password = config.getString("eagleProps.eagleService.password");
+
+        LOG.info("Successfully initialized MRHistoryJobConfig");
+        LOG.info("env: " + this.env);
+        LOG.info("zookeeper.quorum: " + this.zkStateConfig.zkQuorum);
+        LOG.info("zookeeper.property.clientPort: " + this.zkStateConfig.zkPort);
+        LOG.info("eagle.service.host: " + this.eagleServiceConfig.eagleServiceHost);
+        LOG.info("eagle.service.port: " + this.eagleServiceConfig.eagleServicePort);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/0bde482b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/MRHistoryJobMain.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/MRHistoryJobMain.java b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/MRHistoryJobMain.java
index 9f030a7..bef72cc 100644
--- a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/MRHistoryJobMain.java
+++ b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/MRHistoryJobMain.java
@@ -18,74 +18,8 @@
 
 package org.apache.eagle.jpm.mr.history;
 
-import backtype.storm.Config;
-import backtype.storm.LocalCluster;
-import backtype.storm.StormSubmitter;
-import backtype.storm.topology.TopologyBuilder;
-import org.apache.eagle.jpm.mr.history.common.JHFConfigManager;
-import org.apache.eagle.jpm.mr.history.crawler.JobHistoryContentFilter;
-import org.apache.eagle.jpm.mr.history.crawler.JobHistoryContentFilterBuilder;
-import org.apache.eagle.jpm.mr.history.storm.JobHistorySpout;
-import org.apache.eagle.jpm.util.Constants;
-
-import java.util.List;
-import java.util.regex.Pattern;
-
 public class MRHistoryJobMain {
     public static void main(String []args) {
-        try {
-            //1. trigger init conf
-            JHFConfigManager jhfConfigManager = JHFConfigManager.getInstance(args);
-            com.typesafe.config.Config jhfAppConf = jhfConfigManager.getConfig();
-
-            //2. init JobHistoryContentFilter
-            JobHistoryContentFilterBuilder builder = JobHistoryContentFilterBuilder.newBuilder().acceptJobFile().acceptJobConfFile();
-            List<String> confKeyPatterns = jhfAppConf.getStringList("MRConfigureKeys.jobConfigKey");
-            confKeyPatterns.add(Constants.JobConfiguration.CASCADING_JOB);
-            confKeyPatterns.add(Constants.JobConfiguration.HIVE_JOB);
-            confKeyPatterns.add(Constants.JobConfiguration.PIG_JOB);
-            confKeyPatterns.add(Constants.JobConfiguration.SCOOBI_JOB);
-
-            String jobNameKey = jhfAppConf.getString("MRConfigureKeys.jobNameKey");
-            builder.setJobNameKey(jobNameKey);
-
-            for (String key : confKeyPatterns) {
-                builder.includeJobKeyPatterns(Pattern.compile(key));
-            }
-            JobHistoryContentFilter filter = builder.build();
-
-            //3. init topology
-            TopologyBuilder topologyBuilder = new TopologyBuilder();
-            String topologyName = "mrHistoryJobTopology";
-            if (jhfAppConf.hasPath("envContextConfig.topologyName")) {
-                topologyName = jhfAppConf.getString("envContextConfig.topologyName");
-            }
-            String spoutName = "mrHistoryJobExecutor";
-            int parallelism = jhfAppConf.getInt("envContextConfig.parallelismConfig." + spoutName);
-            int tasks = jhfAppConf.getInt("envContextConfig.tasks." + spoutName);
-            if (parallelism > tasks) {
-                parallelism = tasks;
-            }
-            topologyBuilder.setSpout(
-                    spoutName,
-                    new JobHistorySpout(filter, jhfConfigManager),
-                    parallelism
-            ).setNumTasks(tasks);
-
-            Config config = new backtype.storm.Config();
-            config.setNumWorkers(jhfAppConf.getInt("envContextConfig.workers"));
-            config.put(Config.TOPOLOGY_DEBUG, true);
-            if (!jhfConfigManager.getEnv().equals("local")) {
-                //cluster mode
-                //parse conf here
-                StormSubmitter.submitTopology(topologyName, config, topologyBuilder.createTopology());
-            } else {
-                //local mode
-                LocalCluster cluster = new LocalCluster();
-                cluster.submitTopology(topologyName, config, topologyBuilder.createTopology());
-            }
-        } catch (Exception e) {
-            e.printStackTrace();
-        }
+        new MRHistoryJobApplication().run(args);
     }
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/0bde482b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/common/JHFConfigManager.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/common/JHFConfigManager.java b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/common/JHFConfigManager.java
deleted file mode 100644
index c99891b..0000000
--- a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/common/JHFConfigManager.java
+++ /dev/null
@@ -1,182 +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.eagle.jpm.mr.history.common;
-
-import com.typesafe.config.Config;
-import org.apache.eagle.common.config.ConfigOptionParser;
-import org.apache.eagle.jpm.util.DefaultJobIdPartitioner;
-import org.apache.eagle.jpm.util.JobIdPartitioner;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.Serializable;
-
-public class JHFConfigManager implements Serializable {
-    private static final Logger LOG = LoggerFactory.getLogger(JHFConfigManager.class);
-
-    private static final String JOB_CONFIGURE_KEY_CONF_FILE = "JobConfigKeys.conf";
-
-    public String getEnv() {
-        return env;
-    }
-    private String env;
-
-    public ZKStateConfig getZkStateConfig() { return zkStateConfig; }
-    private ZKStateConfig zkStateConfig;
-
-    public JobHistoryEndpointConfig getJobHistoryEndpointConfig() { return jobHistoryEndpointConfig; }
-    private JobHistoryEndpointConfig jobHistoryEndpointConfig;
-
-    public ControlConfig getControlConfig() { return controlConfig; }
-    private ControlConfig controlConfig;
-
-    public JobExtractorConfig getJobExtractorConfig() { return jobExtractorConfig; }
-    private JobExtractorConfig jobExtractorConfig;
-
-    public EagleServiceConfig getEagleServiceConfig() {
-        return eagleServiceConfig;
-    }
-    private EagleServiceConfig eagleServiceConfig;
-
-    public Config getConfig() {
-        return config;
-    }
-    private Config config;
-
-    public static class ZKStateConfig implements Serializable {
-        public String zkQuorum;
-        public String zkRoot;
-        public int zkSessionTimeoutMs;
-        public int zkRetryTimes;
-        public int zkRetryInterval;
-        public String zkPort;
-    }
-
-    public static class JobHistoryEndpointConfig implements Serializable {
-        public String nnEndpoint;
-        public String basePath;
-        public boolean pathContainsJobTrackerName;
-        public String jobTrackerName;
-        public String principal;
-        public String keyTab;
-    }
-
-    public static class ControlConfig implements Serializable {
-        public boolean dryRun;
-        public Class<? extends JobIdPartitioner> partitionerCls;
-        public boolean zeroBasedMonth;
-        public String timeZone;
-    }
-
-    public static class JobExtractorConfig implements Serializable {
-        public String site;
-        public String mrVersion;
-        public int readTimeoutSeconds;
-    }
-
-    public static class EagleServiceConfig implements Serializable {
-        public String eagleServiceHost;
-        public int eagleServicePort;
-        public String username;
-        public String password;
-    }
-
-    private static JHFConfigManager manager = new JHFConfigManager();
-
-    /**
-     * As this is singleton object and constructed while this class is being initialized,
-     * so any exception within this constructor will be wrapped with java.lang.ExceptionInInitializerError.
-     * And this is unrecoverable and hard to troubleshooting.
-     */
-    private JHFConfigManager() {
-        this.zkStateConfig = new ZKStateConfig();
-        this.jobHistoryEndpointConfig = new JobHistoryEndpointConfig();
-        this.controlConfig = new ControlConfig();
-        this.jobExtractorConfig = new JobExtractorConfig();
-        this.eagleServiceConfig = new EagleServiceConfig();
-    }
-
-    public static JHFConfigManager getInstance(String []args) {
-        manager.init(args);
-        return manager;
-    }
-
-    /**
-     * read configuration file and load hbase config etc
-     */
-    private void init(String[] args) {
-        // TODO: Probably we can remove the properties file path check in future
-        try {
-            LOG.info("Loading from configuration file");
-            this.config = new ConfigOptionParser().load(args);
-        } catch (Exception e) {
-            LOG.error("failed to load config");
-        }
-
-        this.env = config.getString("envContextConfig.env");
-
-        //parse eagle job extractor
-        this.jobExtractorConfig.site = config.getString("jobExtractorConfig.site");
-        this.jobExtractorConfig.mrVersion = config.getString("jobExtractorConfig.mrVersion");
-        this.jobExtractorConfig.readTimeoutSeconds = config.getInt("jobExtractorConfig.readTimeOutSeconds");
-        //parse eagle zk
-        this.zkStateConfig.zkQuorum = config.getString("dataSourceConfig.zkQuorum");
-        this.zkStateConfig.zkPort = config.getString("dataSourceConfig.zkPort");
-        this.zkStateConfig.zkSessionTimeoutMs = config.getInt("dataSourceConfig.zkSessionTimeoutMs");
-        this.zkStateConfig.zkRetryTimes = config.getInt("dataSourceConfig.zkRetryTimes");
-        this.zkStateConfig.zkRetryInterval = config.getInt("dataSourceConfig.zkRetryInterval");
-        this.zkStateConfig.zkRoot = config.getString("dataSourceConfig.zkRoot");
-
-        //parse job history endpoint
-        this.jobHistoryEndpointConfig.basePath = config.getString("dataSourceConfig.basePath");
-        this.jobHistoryEndpointConfig.jobTrackerName = config.getString("dataSourceConfig.jobTrackerName");
-        this.jobHistoryEndpointConfig.nnEndpoint = config.getString("dataSourceConfig.nnEndpoint");
-        this.jobHistoryEndpointConfig.pathContainsJobTrackerName = config.getBoolean("dataSourceConfig.pathContainsJobTrackerName");
-        this.jobHistoryEndpointConfig.principal = config.getString("dataSourceConfig.principal");
-        this.jobHistoryEndpointConfig.keyTab = config.getString("dataSourceConfig.keytab");
-
-        //parse control config
-        this.controlConfig.dryRun = config.getBoolean("dataSourceConfig.dryRun");
-        try {
-            this.controlConfig.partitionerCls = (Class<? extends JobIdPartitioner>) Class.forName(config.getString("dataSourceConfig.partitionerCls"));
-            assert this.controlConfig.partitionerCls != null;
-        } catch (Exception e) {
-            LOG.warn("can not initialize partitioner class, use org.apache.eagle.jpm.util.DefaultJobIdPartitioner", e);
-            this.controlConfig.partitionerCls = DefaultJobIdPartitioner.class;
-        } finally {
-            LOG.info("Loaded partitioner class: {}",this.controlConfig.partitionerCls);
-        }
-        this.controlConfig.zeroBasedMonth = config.getBoolean("dataSourceConfig.zeroBasedMonth");
-        this.controlConfig.timeZone = config.getString("dataSourceConfig.timeZone");
-
-        // parse eagle service endpoint
-        this.eagleServiceConfig.eagleServiceHost = config.getString("eagleProps.eagleService.host");
-        String port = config.getString("eagleProps.eagleService.port");
-        this.eagleServiceConfig.eagleServicePort = (port == null ? 8080 : Integer.parseInt(port));
-        this.eagleServiceConfig.username = config.getString("eagleProps.eagleService.username");
-        this.eagleServiceConfig.password = config.getString("eagleProps.eagleService.password");
-
-        LOG.info("Successfully initialized JHFConfigManager");
-        LOG.info("env: " + this.env);
-        LOG.info("zookeeper.quorum: " + this.zkStateConfig.zkQuorum);
-        LOG.info("zookeeper.property.clientPort: " + this.zkStateConfig.zkPort);
-        LOG.info("eagle.service.host: " + this.eagleServiceConfig.eagleServiceHost);
-        LOG.info("eagle.service.port: " + this.eagleServiceConfig.eagleServicePort);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/0bde482b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/crawler/DefaultJHFInputStreamCallback.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/crawler/DefaultJHFInputStreamCallback.java b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/crawler/DefaultJHFInputStreamCallback.java
index 6f85149..aeb35fd 100644
--- a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/crawler/DefaultJHFInputStreamCallback.java
+++ b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/crawler/DefaultJHFInputStreamCallback.java
@@ -18,7 +18,7 @@
 
 package org.apache.eagle.jpm.mr.history.crawler;
 
-import org.apache.eagle.jpm.mr.history.common.JHFConfigManager;
+import org.apache.eagle.jpm.mr.history.MRHistoryJobConfig;
 import org.apache.eagle.jpm.mr.history.parser.JHFParserBase;
 import org.apache.eagle.jpm.mr.history.parser.JHFParserFactory;
 import org.slf4j.Logger;
@@ -33,16 +33,16 @@ public class DefaultJHFInputStreamCallback implements JHFInputStreamCallback {
 
 
     private JobHistoryContentFilter m_filter;
-    private JHFConfigManager m_configManager;
+    private MRHistoryJobConfig m_configManager;
 
-    public DefaultJHFInputStreamCallback(JobHistoryContentFilter filter, JHFConfigManager configManager, EagleOutputCollector eagleCollector) {
+    public DefaultJHFInputStreamCallback(JobHistoryContentFilter filter, MRHistoryJobConfig configManager, EagleOutputCollector eagleCollector) {
         this.m_filter = filter;
         this.m_configManager = configManager;
     }
 
     @Override
     public void onInputStream(InputStream jobFileInputStream, org.apache.hadoop.conf.Configuration conf) throws Exception {
-        final JHFConfigManager.JobExtractorConfig jobExtractorConfig = m_configManager.getJobExtractorConfig();
+        final MRHistoryJobConfig.JobExtractorConfig jobExtractorConfig = m_configManager.getJobExtractorConfig();
         @SuppressWarnings("serial")
         Map<String, String> baseTags = new HashMap<String, String>() { {
             put("site", jobExtractorConfig.site);

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/0bde482b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/crawler/JHFCrawlerDriverImpl.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/crawler/JHFCrawlerDriverImpl.java b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/crawler/JHFCrawlerDriverImpl.java
index d3e1816..52bd8ea 100644
--- a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/crawler/JHFCrawlerDriverImpl.java
+++ b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/crawler/JHFCrawlerDriverImpl.java
@@ -19,7 +19,7 @@
 package org.apache.eagle.jpm.mr.history.crawler;
 
 import org.apache.commons.lang3.tuple.Pair;
-import org.apache.eagle.jpm.mr.history.common.JHFConfigManager;
+import org.apache.eagle.jpm.mr.history.MRHistoryJobConfig;
 import org.apache.eagle.jpm.util.JobIdFilter;
 import org.apache.eagle.jpm.mr.history.zkres.JobHistoryZKStateLCM;
 import org.slf4j.Logger;
@@ -61,8 +61,8 @@ public class JHFCrawlerDriverImpl implements JHFCrawlerDriver {
     private int m_partitionId;
     private TimeZone m_timeZone;
 
-    public JHFCrawlerDriverImpl(JHFConfigManager.JobHistoryEndpointConfig jobHistoryConfig,
-                                JHFConfigManager.ControlConfig controlConfig, JHFInputStreamCallback reader,
+    public JHFCrawlerDriverImpl(MRHistoryJobConfig.JobHistoryEndpointConfig jobHistoryConfig,
+                                MRHistoryJobConfig.ControlConfig controlConfig, JHFInputStreamCallback reader,
                                 JobHistoryZKStateLCM zkStateLCM,
                                 JobHistoryLCM historyLCM, JobIdFilter jobFilter, int partitionId) throws Exception {
         this.m_zeroBasedMonth = controlConfig.zeroBasedMonth;

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/0bde482b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/crawler/JobHistoryDAOImpl.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/crawler/JobHistoryDAOImpl.java b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/crawler/JobHistoryDAOImpl.java
index 3b303fd..cfd5994 100644
--- a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/crawler/JobHistoryDAOImpl.java
+++ b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/crawler/JobHistoryDAOImpl.java
@@ -33,7 +33,7 @@ import java.util.Arrays;
 import java.util.Comparator;
 import java.util.List;
 import org.apache.eagle.jpm.util.HDFSUtil;
-import org.apache.eagle.jpm.mr.history.common.JHFConfigManager.JobHistoryEndpointConfig;
+import org.apache.eagle.jpm.mr.history.MRHistoryJobConfig.JobHistoryEndpointConfig;
 
 public class JobHistoryDAOImpl extends AbstractJobHistoryDAO {
     private static final Logger LOG = LoggerFactory.getLogger(JobHistoryDAOImpl.class);

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/0bde482b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/HistoryJobEntityCreationListener.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/HistoryJobEntityCreationListener.java b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/HistoryJobEntityCreationListener.java
index bdaedd4..892c2ea 100644
--- a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/HistoryJobEntityCreationListener.java
+++ b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/HistoryJobEntityCreationListener.java
@@ -22,16 +22,18 @@ package org.apache.eagle.jpm.mr.history.parser;
 import org.apache.eagle.jpm.mr.historyentity.JobBaseAPIEntity;
 
 /**
- * generalizing this listener would decouple entity creation and entity handling, also will help unit testing
- * @author yonzhang
+ * generalizing this listener would decouple entity creation and entity handling, also will help unit testing.
  *
+ * @author yonzhang
  */
 public interface HistoryJobEntityCreationListener {
     /**
      * job entity created event
+     *
      * @param entity
      */
     void jobEntityCreated(JobBaseAPIEntity entity) throws Exception;
+
     /**
      * for streaming processing, flush would help commit the last several entities
      */

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/0bde482b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/HistoryJobEntityLifecycleListener.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/HistoryJobEntityLifecycleListener.java b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/HistoryJobEntityLifecycleListener.java
index ae6b5c9..a803c6d 100644
--- a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/HistoryJobEntityLifecycleListener.java
+++ b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/HistoryJobEntityLifecycleListener.java
@@ -22,13 +22,12 @@ import org.apache.eagle.jpm.mr.historyentity.JobBaseAPIEntity;
 
 public interface HistoryJobEntityLifecycleListener extends HistoryJobEntityCreationListener {
     /**
-     * job entity created event
-     * @param entity
+     * job entity created event.
      */
     void jobEntityCreated(JobBaseAPIEntity entity) throws Exception;
 
     /**
-     * Job finished
+     * Job finished.
      */
     void jobFinish();
 }

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/0bde482b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/ImportException.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/ImportException.java b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/ImportException.java
index d454c31..652eaf8 100644
--- a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/ImportException.java
+++ b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/ImportException.java
@@ -18,8 +18,6 @@
 
 package org.apache.eagle.jpm.mr.history.parser;
 
-/**
- */
 public class ImportException extends RuntimeException {
     private static final long serialVersionUID = -706778307046285820L;
 

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/0bde482b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JHFEventReaderBase.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JHFEventReaderBase.java b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JHFEventReaderBase.java
index 9992690..82e305a 100644
--- a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JHFEventReaderBase.java
+++ b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JHFEventReaderBase.java
@@ -6,9 +6,9 @@
  * to you under the Apache License, Version 2.0 (the
  * "License"); you may not use this file except in compliance
  * with the License.  You may obtain a copy of the License at
- * 
+ *
  *     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.
@@ -18,14 +18,13 @@
 
 package org.apache.eagle.jpm.mr.history.parser;
 
-import org.apache.eagle.jpm.mr.historyentity.JobConfig;
 import org.apache.eagle.jpm.mr.history.crawler.JobHistoryContentFilter;
+import org.apache.eagle.jpm.mr.history.parser.JHFMRVer1Parser.Keys;
 import org.apache.eagle.jpm.mr.historyentity.*;
 import org.apache.eagle.jpm.util.Constants;
 import org.apache.eagle.jpm.util.JobNameNormalization;
 import org.apache.eagle.jpm.util.MRJobTagName;
 import org.apache.eagle.jpm.util.jobcounter.JobCounters;
-import org.apache.eagle.jpm.mr.history.parser.JHFMRVer1Parser.Keys;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.mapreduce.jobhistory.EventType;
 import org.slf4j.Logger;
@@ -75,10 +74,18 @@ public abstract class JHFEventReaderBase extends JobEntityCreationPublisher impl
     private long sumReduceTaskDuration;
 
     public Constants.JobType fetchJobType(Configuration config) {
-        if (config.get(Constants.JobConfiguration.CASCADING_JOB) != null) { return Constants.JobType.CASCADING; }
-        if (config.get(Constants.JobConfiguration.HIVE_JOB) != null) { return Constants.JobType.HIVE; }
-        if (config.get(Constants.JobConfiguration.PIG_JOB) != null) { return Constants.JobType.PIG; }
-        if (config.get(Constants.JobConfiguration.SCOOBI_JOB) != null) {return Constants.JobType.SCOOBI; }
+        if (config.get(Constants.JobConfiguration.CASCADING_JOB) != null) {
+            return Constants.JobType.CASCADING;
+        }
+        if (config.get(Constants.JobConfiguration.HIVE_JOB) != null) {
+            return Constants.JobType.HIVE;
+        }
+        if (config.get(Constants.JobConfiguration.PIG_JOB) != null) {
+            return Constants.JobType.PIG;
+        }
+        if (config.get(Constants.JobConfiguration.SCOOBI_JOB) != null) {
+            return Constants.JobType.SCOOBI;
+        }
         return Constants.JobType.NOTAVALIABLE;
     }
 
@@ -86,6 +93,7 @@ public abstract class JHFEventReaderBase extends JobEntityCreationPublisher impl
      * baseTags stores the basic tag name values which might be used for persisting various entities
      * baseTags includes: cluster, datacenter and jobName
      * baseTags are used for all job/task related entities
+     *
      * @param baseTags
      */
     public JHFEventReaderBase(Map<String, String> baseTags, Configuration configuration, JobHistoryContentFilter filter) {
@@ -120,7 +128,7 @@ public abstract class JHFEventReaderBase extends JobEntityCreationPublisher impl
         this.sumReduceTaskDuration = 0l;
     }
 
-    public void register(HistoryJobEntityLifecycleListener lifecycleListener){
+    public void register(HistoryJobEntityLifecycleListener lifecycleListener) {
         this.jobEntityLifecycleListeners.add(lifecycleListener);
     }
 
@@ -132,7 +140,7 @@ public abstract class JHFEventReaderBase extends JobEntityCreationPublisher impl
         }
         try {
             flush();
-        } catch(Exception ex) {
+        } catch (Exception ex) {
             throw new IOException(ex);
         }
     }
@@ -146,8 +154,8 @@ public abstract class JHFEventReaderBase extends JobEntityCreationPublisher impl
     }
 
     /**
-       * @param id
-       */
+     * @param id
+     */
     private void setJobID(String id) {
         this.m_jobId = id;
     }
@@ -157,128 +165,128 @@ public abstract class JHFEventReaderBase extends JobEntityCreationPublisher impl
     }
 
     protected void handleJob(EventType eventType, Map<Keys, String> values, Object totalCounters) throws Exception {
-       String id = values.get(Keys.JOBID);
-
-       if (m_jobId == null) {
-           setJobID(id);
-       } else if (!m_jobId.equals(id)) {
-           String msg = "Current job ID '" + id + "' does not match previously stored value '" + m_jobId + "'";
-           LOG.error(msg);
-           throw new ImportException(msg);
-       }
-
-       if (values.get(Keys.SUBMIT_TIME) != null) {  // job submitted
-           m_jobSubmitEventEntity.setTimestamp(Long.valueOf(values.get(Keys.SUBMIT_TIME)));
-           m_user = values.get(Keys.USER);
-           m_queueName = values.get(Keys.JOB_QUEUE);
-           m_jobName = values.get(Keys.JOBNAME);
-
-           // If given job name then use it as norm job name, otherwise use eagle JobNameNormalization rule to generate.
-           String jobDefId = null;
-           if(configuration != null ) {
-               jobDefId = configuration.get(m_filter.getJobNameKey());
-           }
-
-           if(jobDefId == null) {
-               m_jobDefId = JobNameNormalization.getInstance().normalize(m_jobName);
-           } else {
-               LOG.debug("Got JobDefId from job configuration for " + id + ": " + jobDefId);
-               m_jobDefId = jobDefId;
-           }
-
-           LOG.info("JobDefId of " + id + ": " + m_jobDefId);
-
-           m_jobSubmitEventEntity.getTags().put(MRJobTagName.USER.toString(), m_user);
-           m_jobSubmitEventEntity.getTags().put(MRJobTagName.JOB_ID.toString(), m_jobId);
-           m_jobSubmitEventEntity.getTags().put(MRJobTagName.JOB_STATUS.toString(), EagleJobStatus.SUBMITTED.name());
-           m_jobSubmitEventEntity.getTags().put(MRJobTagName.JOB_NAME.toString(), m_jobName);
-           m_jobSubmitEventEntity.getTags().put(MRJobTagName.JOD_DEF_ID.toString(), m_jobDefId);
-           m_jobExecutionEntity.getTags().put(MRJobTagName.JOB_TYPE.toString(),this.m_jobType);
-           entityCreated(m_jobSubmitEventEntity);
-       } else if(values.get(Keys.LAUNCH_TIME) != null) {  // job launched
-           m_jobLaunchEventEntity.setTimestamp(Long.valueOf(values.get(Keys.LAUNCH_TIME)));
-           m_jobLauchTime = m_jobLaunchEventEntity.getTimestamp();
-           m_jobLaunchEventEntity.getTags().put(MRJobTagName.USER.toString(), m_user);
-           m_jobLaunchEventEntity.getTags().put(MRJobTagName.JOB_ID.toString(), m_jobId);
-           m_jobLaunchEventEntity.getTags().put(MRJobTagName.JOB_STATUS.toString(), EagleJobStatus.LAUNCHED.name());
-           m_jobLaunchEventEntity.getTags().put(MRJobTagName.JOB_NAME.toString(), m_jobName);
-           m_jobLaunchEventEntity.getTags().put(MRJobTagName.JOD_DEF_ID.toString(), m_jobDefId);
-           m_jobLaunchEventEntity.getTags().put(MRJobTagName.JOB_TYPE.toString(),this.m_jobType);
-           m_numTotalMaps = Integer.valueOf(values.get(Keys.TOTAL_MAPS));
-           m_numTotalReduces = Integer.valueOf(values.get(Keys.TOTAL_REDUCES));
-           entityCreated(m_jobLaunchEventEntity);
-       } else if(values.get(Keys.FINISH_TIME) != null) {  // job finished
-           m_jobFinishEventEntity.setTimestamp(Long.valueOf(values.get(Keys.FINISH_TIME)));
-           m_jobFinishEventEntity.getTags().put(MRJobTagName.USER.toString(), m_user);
-           m_jobFinishEventEntity.getTags().put(MRJobTagName.JOB_ID.toString(), m_jobId);
-           m_jobFinishEventEntity.getTags().put(MRJobTagName.JOB_STATUS.toString(), values.get(Keys.JOB_STATUS));
-           m_jobFinishEventEntity.getTags().put(MRJobTagName.JOB_NAME.toString(), m_jobName);
-           m_jobFinishEventEntity.getTags().put(MRJobTagName.JOD_DEF_ID.toString(), m_jobDefId);
-           m_jobFinishEventEntity.getTags().put(MRJobTagName.JOB_TYPE.toString(),this.m_jobType);
-           entityCreated(m_jobFinishEventEntity);
-
-           // populate jobExecutionEntity entity
-           m_jobExecutionEntity.getTags().put(MRJobTagName.USER.toString(), m_user);
-           m_jobExecutionEntity.getTags().put(MRJobTagName.JOB_ID.toString(), m_jobId);
-           m_jobExecutionEntity.getTags().put(MRJobTagName.JOB_NAME.toString(), m_jobName);
-           m_jobExecutionEntity.getTags().put(MRJobTagName.JOD_DEF_ID.toString(), m_jobDefId);
-           m_jobExecutionEntity.getTags().put(MRJobTagName.JOB_QUEUE.toString(), m_queueName);
-           m_jobExecutionEntity.getTags().put(MRJobTagName.JOB_TYPE.toString(),this.m_jobType);
-
-           m_jobExecutionEntity.setCurrentState(values.get(Keys.JOB_STATUS));
-           m_jobExecutionEntity.setStartTime(m_jobLaunchEventEntity.getTimestamp());
-           m_jobExecutionEntity.setEndTime(m_jobFinishEventEntity.getTimestamp());
-           m_jobExecutionEntity.setDurationTime(m_jobExecutionEntity.getEndTime() - m_jobExecutionEntity.getStartTime());
-           m_jobExecutionEntity.setTimestamp(m_jobLaunchEventEntity.getTimestamp());
-           m_jobExecutionEntity.setSubmissionTime(m_jobSubmitEventEntity.getTimestamp());
-           if (values.get(Keys.FAILED_MAPS) != null) {
-               // for Artemis
-               m_jobExecutionEntity.setNumFailedMaps(Integer.valueOf(values.get(Keys.FAILED_MAPS)));
-           }
-           if (values.get(Keys.FAILED_REDUCES) != null) {
-               // for Artemis
-               m_jobExecutionEntity.setNumFailedReduces(Integer.valueOf(values.get(Keys.FAILED_REDUCES)));
-           }
-           m_jobExecutionEntity.setNumFinishedMaps(Integer.valueOf(values.get(Keys.FINISHED_MAPS)));
-           m_jobExecutionEntity.setNumFinishedReduces(Integer.valueOf(values.get(Keys.FINISHED_REDUCES)));
-           m_jobExecutionEntity.setNumTotalMaps(m_numTotalMaps);
-           m_jobExecutionEntity.setNumTotalReduces(m_numTotalReduces);
-           if (values.get(Keys.COUNTERS) != null || totalCounters != null) {
-               JobCounters jobCounters = parseCounters(totalCounters);
-               m_jobExecutionEntity.setJobCounters(jobCounters);
-               if (jobCounters.getCounters().containsKey(Constants.JOB_COUNTER)) {
-                   Map<String, Long> counters = jobCounters.getCounters().get(Constants.JOB_COUNTER);
-                   if (counters.containsKey(Constants.JobCounter.DATA_LOCAL_MAPS.toString())) {
-                       m_jobExecutionEntity.setDataLocalMaps(counters.get(Constants.JobCounter.DATA_LOCAL_MAPS.toString()).intValue());
-                   }
-
-                   if (counters.containsKey(Constants.JobCounter.RACK_LOCAL_MAPS.toString())) {
-                       m_jobExecutionEntity.setRackLocalMaps(counters.get(Constants.JobCounter.RACK_LOCAL_MAPS.toString()).intValue());
-                   }
-
-                   if (counters.containsKey(Constants.JobCounter.TOTAL_LAUNCHED_MAPS.toString())) {
-                       m_jobExecutionEntity.setTotalLaunchedMaps(counters.get(Constants.JobCounter.TOTAL_LAUNCHED_MAPS.toString()).intValue());
-                   }
-               }
-
-               if (m_jobExecutionEntity.getTotalLaunchedMaps() > 0) {
-                   m_jobExecutionEntity.setDataLocalMapsPercentage(m_jobExecutionEntity.getDataLocalMaps() * 1.0 / m_jobExecutionEntity.getTotalLaunchedMaps());
-                   m_jobExecutionEntity.setRackLocalMapsPercentage(m_jobExecutionEntity.getRackLocalMaps() * 1.0 / m_jobExecutionEntity.getTotalLaunchedMaps());
-               }
-           }
-           m_jobExecutionEntity.setAvgMapTaskDuration(this.sumMapTaskDuration * 1.0 / m_numTotalMaps);
-           if (m_numTotalReduces == 0) {
-               m_jobExecutionEntity.setMaxReduceTaskDuration(0);
-               m_jobExecutionEntity.setAvgReduceTaskDuration(0);
-           } else {
-               m_jobExecutionEntity.setAvgReduceTaskDuration(this.sumReduceTaskDuration * 1.0 / m_numTotalReduces);
-           }
-           entityCreated(m_jobExecutionEntity);
-       }
+        String id = values.get(Keys.JOBID);
+
+        if (m_jobId == null) {
+            setJobID(id);
+        } else if (!m_jobId.equals(id)) {
+            String msg = "Current job ID '" + id + "' does not match previously stored value '" + m_jobId + "'";
+            LOG.error(msg);
+            throw new ImportException(msg);
+        }
+
+        if (values.get(Keys.SUBMIT_TIME) != null) {  // job submitted
+            m_jobSubmitEventEntity.setTimestamp(Long.valueOf(values.get(Keys.SUBMIT_TIME)));
+            m_user = values.get(Keys.USER);
+            m_queueName = values.get(Keys.JOB_QUEUE);
+            m_jobName = values.get(Keys.JOBNAME);
+
+            // If given job name then use it as norm job name, otherwise use eagle JobNameNormalization rule to generate.
+            String jobDefId = null;
+            if (configuration != null) {
+                jobDefId = configuration.get(m_filter.getJobNameKey());
+            }
+
+            if (jobDefId == null) {
+                m_jobDefId = JobNameNormalization.getInstance().normalize(m_jobName);
+            } else {
+                LOG.debug("Got JobDefId from job configuration for " + id + ": " + jobDefId);
+                m_jobDefId = jobDefId;
+            }
+
+            LOG.info("JobDefId of " + id + ": " + m_jobDefId);
+
+            m_jobSubmitEventEntity.getTags().put(MRJobTagName.USER.toString(), m_user);
+            m_jobSubmitEventEntity.getTags().put(MRJobTagName.JOB_ID.toString(), m_jobId);
+            m_jobSubmitEventEntity.getTags().put(MRJobTagName.JOB_STATUS.toString(), EagleJobStatus.SUBMITTED.name());
+            m_jobSubmitEventEntity.getTags().put(MRJobTagName.JOB_NAME.toString(), m_jobName);
+            m_jobSubmitEventEntity.getTags().put(MRJobTagName.JOD_DEF_ID.toString(), m_jobDefId);
+            m_jobExecutionEntity.getTags().put(MRJobTagName.JOB_TYPE.toString(), this.m_jobType);
+            entityCreated(m_jobSubmitEventEntity);
+        } else if (values.get(Keys.LAUNCH_TIME) != null) {  // job launched
+            m_jobLaunchEventEntity.setTimestamp(Long.valueOf(values.get(Keys.LAUNCH_TIME)));
+            m_jobLauchTime = m_jobLaunchEventEntity.getTimestamp();
+            m_jobLaunchEventEntity.getTags().put(MRJobTagName.USER.toString(), m_user);
+            m_jobLaunchEventEntity.getTags().put(MRJobTagName.JOB_ID.toString(), m_jobId);
+            m_jobLaunchEventEntity.getTags().put(MRJobTagName.JOB_STATUS.toString(), EagleJobStatus.LAUNCHED.name());
+            m_jobLaunchEventEntity.getTags().put(MRJobTagName.JOB_NAME.toString(), m_jobName);
+            m_jobLaunchEventEntity.getTags().put(MRJobTagName.JOD_DEF_ID.toString(), m_jobDefId);
+            m_jobLaunchEventEntity.getTags().put(MRJobTagName.JOB_TYPE.toString(), this.m_jobType);
+            m_numTotalMaps = Integer.valueOf(values.get(Keys.TOTAL_MAPS));
+            m_numTotalReduces = Integer.valueOf(values.get(Keys.TOTAL_REDUCES));
+            entityCreated(m_jobLaunchEventEntity);
+        } else if (values.get(Keys.FINISH_TIME) != null) {  // job finished
+            m_jobFinishEventEntity.setTimestamp(Long.valueOf(values.get(Keys.FINISH_TIME)));
+            m_jobFinishEventEntity.getTags().put(MRJobTagName.USER.toString(), m_user);
+            m_jobFinishEventEntity.getTags().put(MRJobTagName.JOB_ID.toString(), m_jobId);
+            m_jobFinishEventEntity.getTags().put(MRJobTagName.JOB_STATUS.toString(), values.get(Keys.JOB_STATUS));
+            m_jobFinishEventEntity.getTags().put(MRJobTagName.JOB_NAME.toString(), m_jobName);
+            m_jobFinishEventEntity.getTags().put(MRJobTagName.JOD_DEF_ID.toString(), m_jobDefId);
+            m_jobFinishEventEntity.getTags().put(MRJobTagName.JOB_TYPE.toString(), this.m_jobType);
+            entityCreated(m_jobFinishEventEntity);
+
+            // populate jobExecutionEntity entity
+            m_jobExecutionEntity.getTags().put(MRJobTagName.USER.toString(), m_user);
+            m_jobExecutionEntity.getTags().put(MRJobTagName.JOB_ID.toString(), m_jobId);
+            m_jobExecutionEntity.getTags().put(MRJobTagName.JOB_NAME.toString(), m_jobName);
+            m_jobExecutionEntity.getTags().put(MRJobTagName.JOD_DEF_ID.toString(), m_jobDefId);
+            m_jobExecutionEntity.getTags().put(MRJobTagName.JOB_QUEUE.toString(), m_queueName);
+            m_jobExecutionEntity.getTags().put(MRJobTagName.JOB_TYPE.toString(), this.m_jobType);
+
+            m_jobExecutionEntity.setCurrentState(values.get(Keys.JOB_STATUS));
+            m_jobExecutionEntity.setStartTime(m_jobLaunchEventEntity.getTimestamp());
+            m_jobExecutionEntity.setEndTime(m_jobFinishEventEntity.getTimestamp());
+            m_jobExecutionEntity.setDurationTime(m_jobExecutionEntity.getEndTime() - m_jobExecutionEntity.getStartTime());
+            m_jobExecutionEntity.setTimestamp(m_jobLaunchEventEntity.getTimestamp());
+            m_jobExecutionEntity.setSubmissionTime(m_jobSubmitEventEntity.getTimestamp());
+            if (values.get(Keys.FAILED_MAPS) != null) {
+                // for Artemis
+                m_jobExecutionEntity.setNumFailedMaps(Integer.valueOf(values.get(Keys.FAILED_MAPS)));
+            }
+            if (values.get(Keys.FAILED_REDUCES) != null) {
+                // for Artemis
+                m_jobExecutionEntity.setNumFailedReduces(Integer.valueOf(values.get(Keys.FAILED_REDUCES)));
+            }
+            m_jobExecutionEntity.setNumFinishedMaps(Integer.valueOf(values.get(Keys.FINISHED_MAPS)));
+            m_jobExecutionEntity.setNumFinishedReduces(Integer.valueOf(values.get(Keys.FINISHED_REDUCES)));
+            m_jobExecutionEntity.setNumTotalMaps(m_numTotalMaps);
+            m_jobExecutionEntity.setNumTotalReduces(m_numTotalReduces);
+            if (values.get(Keys.COUNTERS) != null || totalCounters != null) {
+                JobCounters jobCounters = parseCounters(totalCounters);
+                m_jobExecutionEntity.setJobCounters(jobCounters);
+                if (jobCounters.getCounters().containsKey(Constants.JOB_COUNTER)) {
+                    Map<String, Long> counters = jobCounters.getCounters().get(Constants.JOB_COUNTER);
+                    if (counters.containsKey(Constants.JobCounter.DATA_LOCAL_MAPS.toString())) {
+                        m_jobExecutionEntity.setDataLocalMaps(counters.get(Constants.JobCounter.DATA_LOCAL_MAPS.toString()).intValue());
+                    }
+
+                    if (counters.containsKey(Constants.JobCounter.RACK_LOCAL_MAPS.toString())) {
+                        m_jobExecutionEntity.setRackLocalMaps(counters.get(Constants.JobCounter.RACK_LOCAL_MAPS.toString()).intValue());
+                    }
+
+                    if (counters.containsKey(Constants.JobCounter.TOTAL_LAUNCHED_MAPS.toString())) {
+                        m_jobExecutionEntity.setTotalLaunchedMaps(counters.get(Constants.JobCounter.TOTAL_LAUNCHED_MAPS.toString()).intValue());
+                    }
+                }
+
+                if (m_jobExecutionEntity.getTotalLaunchedMaps() > 0) {
+                    m_jobExecutionEntity.setDataLocalMapsPercentage(m_jobExecutionEntity.getDataLocalMaps() * 1.0 / m_jobExecutionEntity.getTotalLaunchedMaps());
+                    m_jobExecutionEntity.setRackLocalMapsPercentage(m_jobExecutionEntity.getRackLocalMaps() * 1.0 / m_jobExecutionEntity.getTotalLaunchedMaps());
+                }
+            }
+            m_jobExecutionEntity.setAvgMapTaskDuration(this.sumMapTaskDuration * 1.0 / m_numTotalMaps);
+            if (m_numTotalReduces == 0) {
+                m_jobExecutionEntity.setMaxReduceTaskDuration(0);
+                m_jobExecutionEntity.setAvgReduceTaskDuration(0);
+            } else {
+                m_jobExecutionEntity.setAvgReduceTaskDuration(this.sumReduceTaskDuration * 1.0 / m_numTotalReduces);
+            }
+            entityCreated(m_jobExecutionEntity);
+        }
     }
 
     private void entityCreated(JobBaseAPIEntity entity) throws Exception {
-        for (HistoryJobEntityLifecycleListener lifecycleListener: this.jobEntityLifecycleListeners) {
+        for (HistoryJobEntityLifecycleListener lifecycleListener : this.jobEntityLifecycleListeners) {
             lifecycleListener.jobEntityCreated(entity);
         }
 
@@ -295,11 +303,12 @@ public abstract class JHFEventReaderBase extends JobEntityCreationPublisher impl
     protected abstract JobCounters parseCounters(Object value) throws IOException;
 
     /**
-      * for one task ID, it has several sequential task events, i.e.
-      * task_start -> task_attempt_start -> task_attempt_finish -> task_attempt_start -> task_attempt_finish -> ... -> task_end
-      * @param values
-      * @throws IOException
-      */
+     * for one task ID, it has several sequential task events, i.e.
+     * task_start -> task_attempt_start -> task_attempt_finish -> task_attempt_start -> task_attempt_finish -> ... -> task_end
+     *
+     * @param values
+     * @throws IOException
+     */
     @SuppressWarnings("serial")
     protected void handleTask(RecordTypes recType, EventType eventType, final Map<Keys, String> values, Object counters) throws Exception {
         String taskAttemptID = values.get(Keys.TASK_ATTEMPT_ID);
@@ -308,7 +317,7 @@ public abstract class JHFEventReaderBase extends JobEntityCreationPublisher impl
         final String taskType = values.get(Keys.TASK_TYPE);
         final String taskID = values.get(Keys.TASKID);
 
-        Map<String, String> taskBaseTags = new HashMap<String, String>(){{
+        Map<String, String> taskBaseTags = new HashMap<String, String>() {{
             put(MRJobTagName.TASK_TYPE.toString(), taskType);
             put(MRJobTagName.USER.toString(), m_user);
             //put(MRJobTagName.JOB_NAME.toString(), _jobName);
@@ -402,11 +411,12 @@ public abstract class JHFEventReaderBase extends JobEntityCreationPublisher impl
         Map<String, String> prop = new TreeMap<>();
 
         if (m_filter.acceptJobConfFile()) {
-            Iterator<Map.Entry<String, String> > iter = configuration.iterator();
+            Iterator<Map.Entry<String, String>> iter = configuration.iterator();
             while (iter.hasNext()) {
                 String key = iter.next().getKey();
-                if (included(key) && !excluded(key))
+                if (included(key) && !excluded(key)) {
                     prop.put(key, configuration.get(key));
+                }
             }
         }
 
@@ -442,15 +452,17 @@ public abstract class JHFEventReaderBase extends JobEntityCreationPublisher impl
                     break;
                 }
             }
-            if (!matched)
+            if (!matched) {
                 return false;
+            }
         }
         return true;
     }
 
     private boolean included(String key) {
-        if (m_filter.getJobConfKeyInclusionPatterns() == null)
+        if (m_filter.getJobConfKeyInclusionPatterns() == null) {
             return true;
+        }
         for (Pattern p : m_filter.getJobConfKeyInclusionPatterns()) {
             Matcher m = p.matcher(key);
             if (m.matches()) {
@@ -462,13 +474,15 @@ public abstract class JHFEventReaderBase extends JobEntityCreationPublisher impl
     }
 
     private boolean excluded(String key) {
-        if (m_filter.getJobConfKeyExclusionPatterns() == null)
+        if (m_filter.getJobConfKeyExclusionPatterns() == null) {
             return false;
+        }
         for (Pattern p : m_filter.getJobConfKeyExclusionPatterns()) {
             Matcher m = p.matcher(key);
-            if (m.matches())
+            if (m.matches()) {
                 return true;
+            }
         }
         return false;
     }
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/0bde482b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JHFMRVer1EventReader.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JHFMRVer1EventReader.java b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JHFMRVer1EventReader.java
index 654f63f..6932dad 100644
--- a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JHFMRVer1EventReader.java
+++ b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JHFMRVer1EventReader.java
@@ -19,9 +19,9 @@
 package org.apache.eagle.jpm.mr.history.parser;
 
 import org.apache.eagle.jpm.mr.history.crawler.JobHistoryContentFilter;
+import org.apache.eagle.jpm.mr.history.parser.JHFMRVer1Parser.Keys;
 import org.apache.eagle.jpm.mr.historyentity.JobExecutionAPIEntity;
 import org.apache.eagle.jpm.util.jobcounter.JobCounters;
-import org.apache.eagle.jpm.mr.history.parser.JHFMRVer1Parser.Keys;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.mapreduce.Counter;
 import org.apache.hadoop.mapreduce.CounterGroup;
@@ -37,8 +37,8 @@ import java.util.Map;
 
 /**
  * Listener holds all informations related to one whole job history file, so it's stateful and does not support multithreading.
- * @author yonzhang
  *
+ * @author yonzhang
  */
 public class JHFMRVer1EventReader extends JHFEventReaderBase implements JHFMRVer1PerLineListener {
     private static final Logger logger = LoggerFactory.getLogger(JHFMRVer1EventReader.class);
@@ -47,6 +47,7 @@ public class JHFMRVer1EventReader extends JHFEventReaderBase implements JHFMRVer
      * baseTags stores the basic tag name values which might be used for persisting various entities
      * baseTags includes: cluster, datacenter and jobName
      * baseTags are used for all job/task related entities
+     *
      * @param baseTags
      */
     public JHFMRVer1EventReader(Map<String, String> baseTags, Configuration configuration, JobHistoryContentFilter filter) {
@@ -55,7 +56,7 @@ public class JHFMRVer1EventReader extends JHFEventReaderBase implements JHFMRVer
 
     @Override
     public void handle(RecordTypes recType, Map<Keys, String> values)
-          throws Exception {
+        throws Exception {
         switch (recType) {
             case Job:
                 handleJob(null, values, values.get(Keys.COUNTERS));
@@ -76,11 +77,12 @@ public class JHFMRVer1EventReader extends JHFEventReaderBase implements JHFMRVer
                 ;
         }
     }
-     
+
     private void ensureRackHostnameAfterAttemptFinish(Map<Keys, String> values) {
         // only care about attempt finish
-        if (values.get(Keys.FINISH_TIME) == null)
+        if (values.get(Keys.FINISH_TIME) == null) {
             return;
+        }
         String hostname = null;
         String rack = null;
         // we get rack/hostname based on task's status
@@ -92,24 +94,24 @@ public class JHFMRVer1EventReader extends JHFEventReaderBase implements JHFMRVer
             hostname = tmp[tmp.length - 1];
             rack = tmp[tmp.length - 2];
             m_host2RackMapping.put(hostname, rack);
-        } else if(values.get(Keys.TASK_STATUS).equals(EagleTaskStatus.KILLED.name()) || values.get(Keys.TASK_STATUS).equals(EagleTaskStatus.FAILED.name())) {
+        } else if (values.get(Keys.TASK_STATUS).equals(EagleTaskStatus.KILLED.name()) || values.get(Keys.TASK_STATUS).equals(EagleTaskStatus.FAILED.name())) {
             hostname = values.get(Keys.HOSTNAME);
             // make every effort to get RACK information
             hostname = (hostname == null) ? "" : hostname;
             rack = m_host2RackMapping.get(hostname);
         }
-          
+
         values.put(Keys.HOSTNAME, hostname);
         values.put(Keys.RACK, rack);
     }
-    
+
     @Override
     protected JobCounters parseCounters(Object value) throws IOException {
         JobCounters jc = new JobCounters();
         Map<String, Map<String, Long>> groups = new HashMap<String, Map<String, Long>>();
-        Counters counters = new Counters(); 
+        Counters counters = new Counters();
         try {
-            CountersStrings.parseEscapedCompactString((String)value, counters);
+            CountersStrings.parseEscapedCompactString((String) value, counters);
         } catch (Exception ex) {
             logger.error("can not parse job history", ex);
             throw new IOException(ex);
@@ -118,7 +120,7 @@ public class JHFMRVer1EventReader extends JHFEventReaderBase implements JHFMRVer
         while (it.hasNext()) {
             CounterGroup cg = it.next();
 
-           // hardcoded to exclude business level counters
+            // hardcoded to exclude business level counters
             if (!cg.getName().equals("org.apache.hadoop.mapreduce.FileSystemCounter")
                 && !cg.getName().equals("org.apache.hadoop.mapreduce.TaskCounter")
                 && !cg.getName().equals("org.apache.hadoop.mapreduce.JobCounter")
@@ -128,7 +130,9 @@ public class JHFMRVer1EventReader extends JHFEventReaderBase implements JHFMRVer
                 && !cg.getName().equals("org.apache.hadoop.mapred.Task$Counter")                   // for artemis
                 && !cg.getName().equals("org.apache.hadoop.mapreduce.lib.input.FileInputFormat$Counter")  // for artemis
                 && !cg.getName().equals("org.apache.hadoop.mapreduce.lib.input.FileOutputFormat$Counter")
-            ) continue;
+                ) {
+                continue;
+            }
 
             groups.put(cg.getName(), new HashMap<String, Long>());
             Map<String, Long> counterValues = groups.get(cg.getName());
@@ -143,7 +147,7 @@ public class JHFMRVer1EventReader extends JHFEventReaderBase implements JHFMRVer
         jc.setCounters(groups);
         return jc;
     }
-    
+
     public JobExecutionAPIEntity jobExecution() {
         return m_jobExecutionEntity;
     }

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/0bde482b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JHFMRVer1Parser.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JHFMRVer1Parser.java b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JHFMRVer1Parser.java
index bb08ef0..ab59a41 100644
--- a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JHFMRVer1Parser.java
+++ b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JHFMRVer1Parser.java
@@ -44,16 +44,18 @@ public class JHFMRVer1Parser implements JHFParserBase {
     static final String MAX_COUNTER_COUNT = "10000";
 
     private JHFMRVer1EventReader m_reader;
-    public JHFMRVer1Parser(JHFMRVer1EventReader reader){
+
+    public JHFMRVer1Parser(JHFMRVer1EventReader reader) {
         this.m_reader = reader;
     }
 
     /**
-      * Parses history file and invokes Listener.handle() for
-      * each line of history. It can be used for looking through history
-      * files for specific items without having to keep whole history in memory.
-      * @throws IOException
-      */
+     * Parses history file and invokes Listener.handle() for
+     * each line of history. It can be used for looking through history
+     * files for specific items without having to keep whole history in memory.
+     *
+     * @throws IOException
+     */
     @Override
     public void parse(InputStream in) throws Exception, ParseException {
         // set enough counter number as user may build more counters
@@ -68,7 +70,7 @@ public class JHFMRVer1Parser implements JHFParserBase {
 
             // Check if the file is empty
             if (line == null) {
-              return;
+                return;
             }
 
             // Get the information required for further processing
@@ -80,17 +82,17 @@ public class JHFMRVer1Parser implements JHFParserBase {
             do {
                 buf.append(line);
                 if (!line.trim().endsWith(lineDelim) || line.trim().endsWith(escapedLineDelim)) {
-                  buf.append("\n");
-                  continue;
+                    buf.append("\n");
+                    continue;
                 }
                 parseLine(buf.toString(), m_reader, isEscaped);
                 buf = new StringBuffer();
-            } while ((line = reader.readLine())!= null);
+            } while ((line = reader.readLine()) != null);
 
             // flush to tell listener that we have finished parsing
             logger.info("finish parsing job history file and close");
             m_reader.close();
-        } catch(Exception ex) {
+        } catch (Exception ex) {
             logger.error("can not parse correctly ", ex);
             throw ex;
         } finally {
@@ -104,17 +106,17 @@ public class JHFMRVer1Parser implements JHFParserBase {
         // extract the record type
         int idx = line.indexOf(' ');
         String recType = line.substring(0, idx);
-        String data = line.substring(idx+1, line.length());
+        String data = line.substring(idx + 1, line.length());
 
         Matcher matcher = pattern.matcher(data);
-        Map<Keys,String> parseBuffer = new HashMap<Keys, String>();
+        Map<Keys, String> parseBuffer = new HashMap<Keys, String>();
 
-        while(matcher.find()) {
+        while (matcher.find()) {
             String tuple = matcher.group(0);
-            String []parts = StringUtils.split(tuple, StringUtils.ESCAPE_CHAR, '=');
-            String value = parts[1].substring(1, parts[1].length() -1);
+            String[] parts = StringUtils.split(tuple, StringUtils.ESCAPE_CHAR, '=');
+            String value = parts[1].substring(1, parts[1].length() - 1);
             if (isEscaped) {
-              value = StringUtils.unEscapeString(value, StringUtils.ESCAPE_CHAR, charsToEscape);
+                value = StringUtils.unEscapeString(value, StringUtils.ESCAPE_CHAR, charsToEscape);
             }
             parseBuffer.put(Keys.valueOf(parts[0]), value);
         }
@@ -131,141 +133,154 @@ public class JHFMRVer1Parser implements JHFParserBase {
         }
 
         parseBuffer.clear();
-      }
-
-      /**
-       * Manages job-history's meta information such as version etc.
-       * Helps in logging version information to the job-history and recover
-       * version information from the history.
-       */
-      static class MetaInfoManager implements JHFMRVer1PerLineListener {
-          private long version = 0L;
-          private KeyValuePair pairs = new KeyValuePair();
-
-          public void close() {
-          }
-          // Extract the version of the history that was used to write the history
-          public MetaInfoManager(String line) throws Exception, ParseException {
-              if (null != line) {
-                  // Parse the line
-                  parseLine(line, this, false);
-              }
-          }
-
-          // Get the line delimiter
-          char getLineDelim() {
-              if (version == 0) {
-                  return '"';
-              } else {
-                  return LINE_DELIMITER_CHAR;
-              }
-          }
-
-          // Checks if the values are escaped or not
-          boolean isValueEscaped() {
-              // Note that the values are not escaped in version 0
-              return version != 0;
-          }
-
-          public void handle(RecordTypes recType, Map<Keys, String> values) throws IOException {
+    }
+
+    /**
+     * Manages job-history's meta information such as version etc.
+     * Helps in logging version information to the job-history and recover
+     * version information from the history.
+     */
+    static class MetaInfoManager implements JHFMRVer1PerLineListener {
+        private long version = 0L;
+        private KeyValuePair pairs = new KeyValuePair();
+
+        public void close() {
+        }
+
+        // Extract the version of the history that was used to write the history
+        public MetaInfoManager(String line) throws Exception, ParseException {
+            if (null != line) {
+                // Parse the line
+                parseLine(line, this, false);
+            }
+        }
+
+        // Get the line delimiter
+        char getLineDelim() {
+            if (version == 0) {
+                return '"';
+            } else {
+                return LINE_DELIMITER_CHAR;
+            }
+        }
+
+        // Checks if the values are escaped or not
+        boolean isValueEscaped() {
+            // Note that the values are not escaped in version 0
+            return version != 0;
+        }
+
+        public void handle(RecordTypes recType, Map<Keys, String> values) throws IOException {
             // Check if the record is of type META
-              if (RecordTypes.Meta == recType) {
-                  pairs.handle(values);
-                  version = pairs.getLong(Keys.VERSION); // defaults to 0
-              }
-          }
-      }
-      
-      /**
-       * Base class contais utility stuff to manage types key value pairs with enums. 
-       */
-      static class KeyValuePair {
-          private Map<Keys, String> values = new HashMap<Keys, String>(); 
-
-          /**
-           * Get 'String' value for given key. Most of the places use Strings as 
-           * values so the default get' method returns 'String'.  This method never returns 
-           * null to ease on GUIs. if no value is found it returns empty string ""
-           * @param k 
-           * @return if null it returns empty string - "" 
-           */
-          public String get(Keys k) {
-              String s = values.get(k);
-              return s == null ? "" : s;
-          }
-          /**
-           * Convert value from history to int and return. 
-           * if no value is found it returns 0.
-           * @param k key 
-           */
-          public int getInt(Keys k) {
-              String s = values.get(k);
-              if (null != s){
-                  return Integer.parseInt(s);
-              }
-              return 0;
-          }
-          /**
-           * Convert value from history to int and return. 
-           * if no value is found it returns 0.
-           * @param k
-           */
-          public long getLong(Keys k) {
-              String s = values.get(k);
-              if (null != s){
-                  return Long.parseLong(s);
-              }
-              return 0;
-          }
-          /**
-           * Set value for the key. 
-           * @param k
-           * @param s
-           */
-          public void set(Keys k, String s) {
-              values.put(k, s);
-          }
-          /**
-           * Adds all values in the Map argument to its own values. 
-           * @param m
-           */
-          public void set(Map<Keys, String> m) {
-              values.putAll(m);
-          }
-          /**
-           * Reads values back from the history, input is same Map as passed to Listener by parseHistory().  
-           * @param values
-           */
-          public synchronized void handle(Map<Keys, String> values) {
-              set(values);
-          }
-          /**
-           * Returns Map containing all key-values. 
-           */
-          public Map<Keys, String> getValues() {
-              return values;
-          }
-      }
-      
-      /**
-       * Job history files contain key="value" pairs, where keys belong to this enum. 
-       * It acts as a global namespace for all keys. 
-       */
-      public static enum Keys { 
-          JOBTRACKERID,
-          START_TIME, FINISH_TIME, JOBID, JOBNAME, USER, JOBCONF, SUBMIT_TIME, 
-          LAUNCH_TIME, TOTAL_MAPS, TOTAL_REDUCES, FAILED_MAPS, FAILED_REDUCES, 
-          FINISHED_MAPS, FINISHED_REDUCES, JOB_STATUS, TASKID, HOSTNAME, TASK_TYPE, 
-          ERROR, TASK_ATTEMPT_ID, TASK_STATUS, COPY_PHASE, SORT_PHASE, REDUCE_PHASE, 
-          SHUFFLE_FINISHED, SORT_FINISHED, COUNTERS, SPLITS, JOB_PRIORITY, HTTP_PORT, 
-          TRACKER_NAME, STATE_STRING, VERSION, MAP_COUNTERS, REDUCE_COUNTERS,
-          VIEW_JOB, MODIFY_JOB, JOB_QUEUE, RACK,
-
-          UBERISED,SPLIT_LOCATIONS,FAILED_DUE_TO_ATTEMPT,MAP_FINISH_TIME,PORT,RACK_NAME,
-
-          //For Artemis
-          WORKFLOW_ID,WORKFLOW_NAME,WORKFLOW_NODE_NAME,WORKFLOW_ADJACENCIES,WORKFLOW_TAGS,
-          SHUFFLE_PORT,LOCALITY,AVATAAR,FAIL_REASON
-      }
+            if (RecordTypes.Meta == recType) {
+                pairs.handle(values);
+                version = pairs.getLong(Keys.VERSION); // defaults to 0
+            }
+        }
+    }
+
+    /**
+     * Base class contais utility stuff to manage types key value pairs with enums.
+     */
+    static class KeyValuePair {
+        private Map<Keys, String> values = new HashMap<Keys, String>();
+
+        /**
+         * Get 'String' value for given key. Most of the places use Strings as
+         * values so the default get' method returns 'String'.  This method never returns
+         * null to ease on GUIs. if no value is found it returns empty string ""
+         *
+         * @param k
+         * @return if null it returns empty string - ""
+         */
+        public String get(Keys k) {
+            String s = values.get(k);
+            return s == null ? "" : s;
+        }
+
+        /**
+         * Convert value from history to int and return.
+         * if no value is found it returns 0.
+         *
+         * @param k key
+         */
+        public int getInt(Keys k) {
+            String s = values.get(k);
+            if (null != s) {
+                return Integer.parseInt(s);
+            }
+            return 0;
+        }
+
+        /**
+         * Convert value from history to int and return.
+         * if no value is found it returns 0.
+         *
+         * @param k
+         */
+        public long getLong(Keys k) {
+            String s = values.get(k);
+            if (null != s) {
+                return Long.parseLong(s);
+            }
+            return 0;
+        }
+
+        /**
+         * Set value for the key.
+         *
+         * @param k
+         * @param s
+         */
+        public void set(Keys k, String s) {
+            values.put(k, s);
+        }
+
+        /**
+         * Adds all values in the Map argument to its own values.
+         *
+         * @param m
+         */
+        public void set(Map<Keys, String> m) {
+            values.putAll(m);
+        }
+
+        /**
+         * Reads values back from the history, input is same Map as passed to Listener by parseHistory().
+         *
+         * @param values
+         */
+        public synchronized void handle(Map<Keys, String> values) {
+            set(values);
+        }
+
+        /**
+         * Returns Map containing all key-values.
+         */
+        public Map<Keys, String> getValues() {
+            return values;
+        }
+    }
+
+    /**
+     * Job history files contain key="value" pairs, where keys belong to this enum.
+     * It acts as a global namespace for all keys.
+     */
+    public static enum Keys {
+        JOBTRACKERID,
+        START_TIME, FINISH_TIME, JOBID, JOBNAME, USER, JOBCONF, SUBMIT_TIME,
+        LAUNCH_TIME, TOTAL_MAPS, TOTAL_REDUCES, FAILED_MAPS, FAILED_REDUCES,
+        FINISHED_MAPS, FINISHED_REDUCES, JOB_STATUS, TASKID, HOSTNAME, TASK_TYPE,
+        ERROR, TASK_ATTEMPT_ID, TASK_STATUS, COPY_PHASE, SORT_PHASE, REDUCE_PHASE,
+        SHUFFLE_FINISHED, SORT_FINISHED, COUNTERS, SPLITS, JOB_PRIORITY, HTTP_PORT,
+        TRACKER_NAME, STATE_STRING, VERSION, MAP_COUNTERS, REDUCE_COUNTERS,
+        VIEW_JOB, MODIFY_JOB, JOB_QUEUE, RACK,
+
+        UBERISED, SPLIT_LOCATIONS, FAILED_DUE_TO_ATTEMPT, MAP_FINISH_TIME, PORT, RACK_NAME,
+
+        //For Artemis
+        WORKFLOW_ID, WORKFLOW_NAME, WORKFLOW_NODE_NAME, WORKFLOW_ADJACENCIES, WORKFLOW_TAGS,
+        SHUFFLE_PORT, LOCALITY, AVATAAR, FAIL_REASON
+    }
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/0bde482b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JHFMRVer1PerLineListener.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JHFMRVer1PerLineListener.java b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JHFMRVer1PerLineListener.java
index 1c096fc..5d48d5d 100644
--- a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JHFMRVer1PerLineListener.java
+++ b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/parser/JHFMRVer1PerLineListener.java
@@ -24,15 +24,15 @@ import java.io.IOException;
 import java.util.Map;
 
 /**
- * Callback interface for reading back log events from JobHistory. This interface 
- * should be implemented and passed to JobHistory.parseHistory() 
- *
+ * Callback interface for reading back log events from JobHistory. This interface
+ * should be implemented and passed to JobHistory.parseHistory()
  */
-public interface JHFMRVer1PerLineListener{
+public interface JHFMRVer1PerLineListener {
     /**
-     * Callback method for history parser. 
-     * @param recType type of record, which is the first entry in the line. 
-     * @param values a map of key-value pairs as thry appear in history.
+     * Callback method for history parser.
+     *
+     * @param recType type of record, which is the first entry in the line.
+     * @param values  a map of key-value pairs as thry appear in history.
      * @throws IOException
      */
     void handle(RecordTypes recType, Map<Keys, String> values) throws Exception;


[45/52] [abbrv] incubator-eagle git commit: [EAGLE-520] Fix and decouple co-processor from eagle aggreation query service

Posted by yo...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/21187b55/eagle-server/src/main/java/org/apache/eagle/server/ServerApplication.java
----------------------------------------------------------------------
diff --git a/eagle-server/src/main/java/org/apache/eagle/server/ServerApplication.java b/eagle-server/src/main/java/org/apache/eagle/server/ServerApplication.java
index 6299e28..e495e77 100644
--- a/eagle-server/src/main/java/org/apache/eagle/server/ServerApplication.java
+++ b/eagle-server/src/main/java/org/apache/eagle/server/ServerApplication.java
@@ -16,6 +16,12 @@
  */
 package org.apache.eagle.server;
 
+import org.apache.eagle.alert.coordinator.CoordinatorListener;
+import org.apache.eagle.alert.resource.SimpleCORSFiler;
+import org.apache.eagle.log.base.taggedlog.EntityJsonModule;
+import org.apache.eagle.log.base.taggedlog.TaggedLogAPIEntity;
+import org.apache.eagle.server.module.GuideBundleLoader;
+
 import com.sun.jersey.api.core.PackagesResourceConfig;
 import io.dropwizard.Application;
 import io.dropwizard.assets.AssetsBundle;
@@ -23,18 +29,15 @@ import io.dropwizard.setup.Bootstrap;
 import io.dropwizard.setup.Environment;
 import io.swagger.jaxrs.config.BeanConfig;
 import io.swagger.jaxrs.listing.ApiListingResource;
-import org.apache.eagle.alert.coordinator.CoordinatorListener;
-import org.apache.eagle.alert.resource.SimpleCORSFiler;
-import org.apache.eagle.server.module.GuideBundleLoader;
 
-import javax.servlet.DispatcherType;
 import java.util.EnumSet;
+import javax.servlet.DispatcherType;
 
 class ServerApplication extends Application<ServerConfig> {
     @Override
     public void initialize(Bootstrap<ServerConfig> bootstrap) {
         bootstrap.addBundle(GuideBundleLoader.load());
-        bootstrap.addBundle(new AssetsBundle("/assets","/","index.html","/"));
+        bootstrap.addBundle(new AssetsBundle("/assets", "/", "index.html", "/"));
     }
 
     @Override
@@ -47,6 +50,8 @@ class ServerApplication extends Application<ServerConfig> {
         environment.getApplicationContext().setContextPath(ServerConfig.getContextPath());
         environment.jersey().register(RESTExceptionMapper.class);
         environment.jersey().setUrlPattern(ServerConfig.getApiBasePath());
+        environment.getObjectMapper().setFilters(TaggedLogAPIEntity.getFilterProvider());
+        environment.getObjectMapper().registerModule(new EntityJsonModule());
 
         // Automatically scan all REST resources
         new PackagesResourceConfig(ServerConfig.getResourcePackage()).getClasses().forEach(environment.jersey()::register);
@@ -65,9 +70,9 @@ class ServerApplication extends Application<ServerConfig> {
 
         // Simple CORS filter
         environment.servlets().addFilter(SimpleCORSFiler.class.getName(), new SimpleCORSFiler())
-                .addMappingForUrlPatterns(EnumSet.of(DispatcherType.REQUEST), true, "/*");
+            .addMappingForUrlPatterns(EnumSet.of(DispatcherType.REQUEST), true, "/*");
 
         // context listener
         environment.servlets().addServletListeners(new CoordinatorListener());
     }
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/21187b55/eagle-server/src/main/java/org/apache/eagle/server/ServerConfig.java
----------------------------------------------------------------------
diff --git a/eagle-server/src/main/java/org/apache/eagle/server/ServerConfig.java b/eagle-server/src/main/java/org/apache/eagle/server/ServerConfig.java
index 3bbe335..58442d9 100644
--- a/eagle-server/src/main/java/org/apache/eagle/server/ServerConfig.java
+++ b/eagle-server/src/main/java/org/apache/eagle/server/ServerConfig.java
@@ -21,42 +21,43 @@ import com.typesafe.config.ConfigFactory;
 import io.dropwizard.Configuration;
 
 public class ServerConfig extends Configuration {
-    private final static String SERVER_NAME = "Apache Eagle";
-    private final static String SERVER_VERSION = "0.5.0-incubating";
-    private final static String API_BASE_PATH = "/rest/*";
-    private final static String CONTEXT_PATH="/";
-    private final static String RESOURCE_PACKAGE = "org.apache.eagle";
-    private final static String LICENSE = "Apache License (Version 2.0)";
-    private final static String LICENSE_URL = "http://www.apache.org/licenses/LICENSE-2.0";
-
-    public Config getConfig(){
+    private static final String SERVER_NAME = "Apache Eagle";
+    private static final String SERVER_VERSION = "0.5.0-incubating";
+    private static final String API_BASE_PATH = "/rest/*";
+    private static final String CONTEXT_PATH = "/";
+    private static final String RESOURCE_PACKAGE = "org.apache.eagle";
+    private static final String LICENSE = "Apache License (Version 2.0)";
+    private static final String LICENSE_URL = "http://www.apache.org/licenses/LICENSE-2.0";
+
+    public Config getConfig() {
         return ConfigFactory.load();
     }
 
-    static String getServerName(){
+    static String getServerName() {
         return SERVER_NAME;
     }
 
-    static String getServerVersion(){
+    static String getServerVersion() {
         return SERVER_VERSION;
     }
 
-    static String getApiBasePath(){
+    static String getApiBasePath() {
         return API_BASE_PATH;
     }
-    static String getResourcePackage(){
+
+    static String getResourcePackage() {
         return RESOURCE_PACKAGE;
     }
 
-    static String getContextPath(){
+    static String getContextPath() {
         return CONTEXT_PATH;
     }
 
-    public static String getLicense(){
+    public static String getLicense() {
         return LICENSE;
     }
 
-    static String getLicenseUrl(){
+    static String getLicenseUrl() {
         return LICENSE_URL;
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/21187b55/eagle-server/src/main/java/org/apache/eagle/server/ServerMain.java
----------------------------------------------------------------------
diff --git a/eagle-server/src/main/java/org/apache/eagle/server/ServerMain.java b/eagle-server/src/main/java/org/apache/eagle/server/ServerMain.java
index 968a93c..a944265 100644
--- a/eagle-server/src/main/java/org/apache/eagle/server/ServerMain.java
+++ b/eagle-server/src/main/java/org/apache/eagle/server/ServerMain.java
@@ -19,8 +19,8 @@ package org.apache.eagle.server;
 public class ServerMain {
     public static void main(String[] args) {
         System.out.println(
-            "Welcome to Apache Eagle (Incubating)\n"+
-            "---------------"
+            "Welcome to Apache Eagle (Incubating)\n"
+                + "---------------"
         );
         System.out.println(
             "Starting Eagle Server ..."
@@ -28,7 +28,7 @@ public class ServerMain {
         try {
             new ServerApplication().run(args);
         } catch (Exception e) {
-            System.err.println("Got exception: "+e.getMessage());
+            System.err.println("Got exception: " + e.getMessage());
             e.printStackTrace();
             System.exit(1);
         }

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/21187b55/eagle-server/src/main/java/org/apache/eagle/server/module/GuideBundleLoader.java
----------------------------------------------------------------------
diff --git a/eagle-server/src/main/java/org/apache/eagle/server/module/GuideBundleLoader.java b/eagle-server/src/main/java/org/apache/eagle/server/module/GuideBundleLoader.java
index d4ecaf7..51f23b0 100644
--- a/eagle-server/src/main/java/org/apache/eagle/server/module/GuideBundleLoader.java
+++ b/eagle-server/src/main/java/org/apache/eagle/server/module/GuideBundleLoader.java
@@ -16,10 +16,6 @@
  */
 package org.apache.eagle.server.module;
 
-import com.google.inject.Module;
-import com.hubspot.dropwizard.guice.GuiceBundle;
-import com.typesafe.config.Config;
-import com.typesafe.config.ConfigFactory;
 import org.apache.eagle.app.module.ApplicationExtensionLoader;
 import org.apache.eagle.app.service.ApplicationProviderService;
 import org.apache.eagle.app.service.impl.ApplicationProviderServiceImpl;
@@ -28,15 +24,20 @@ import org.apache.eagle.common.module.ModuleRegistry;
 import org.apache.eagle.metadata.persistence.MetadataStore;
 import org.apache.eagle.metadata.persistence.MetadataStoreModuleFactory;
 import org.apache.eagle.server.ServerConfig;
+
+import com.google.inject.Module;
+import com.hubspot.dropwizard.guice.GuiceBundle;
+import com.typesafe.config.Config;
+import com.typesafe.config.ConfigFactory;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import java.util.List;
 
 public class GuideBundleLoader {
-    private final static Logger LOGGER = LoggerFactory.getLogger(GuideBundleLoader.class);
+    private static final  Logger LOGGER = LoggerFactory.getLogger(GuideBundleLoader.class);
 
-    public static GuiceBundle<ServerConfig> load(List<Module> modules){
+    public static GuiceBundle<ServerConfig> load(List<Module> modules) {
         /*
            We use tow injectors, one is Dropwizard injector, the other injector is to instantiate ApplicationProvider and
            load sub modules from applications
@@ -56,27 +57,29 @@ public class GuideBundleLoader {
         List<Module> metadataExtensions = metadataStoreModule.getModules(registry);
         int extensionNum = 0;
         GuiceBundle.Builder<ServerConfig> builder = GuiceBundle.newBuilder();
-        if(metadataExtensions!=null){
+        if (metadataExtensions != null) {
             extensionNum = metadataExtensions.size();
             metadataExtensions.forEach(builder::addModule);
         }
-        LOGGER.warn("Loaded {} modules (scope: metadataStore)",extensionNum);
+        LOGGER.warn("Loaded {} modules (scope: metadataStore)", extensionNum);
 
         List<Module> globalExtensions = registry.getModules(GlobalScope.class);
         extensionNum = 0;
-        if(globalExtensions!=null){
+        if (globalExtensions != null) {
             extensionNum = globalExtensions.size();
             globalExtensions.forEach(builder::addModule);
         }
-        LOGGER.warn("Loaded {} modules (scope: global)",extensionNum);
+        LOGGER.warn("Loaded {} modules (scope: global)", extensionNum);
 
-        if(modules!=null) modules.forEach(builder::addModule);
+        if (modules != null) {
+            modules.forEach(builder::addModule);
+        }
         return builder.addModule(serveBaseModule)
-                .setConfigClass(ServerConfig.class)
-                .build();
+            .setConfigClass(ServerConfig.class)
+            .build();
     }
 
-    public static GuiceBundle<ServerConfig> load(){
+    public static GuiceBundle<ServerConfig> load() {
         return load(null);
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/21187b55/eagle-server/src/main/java/org/apache/eagle/server/module/ServerModule.java
----------------------------------------------------------------------
diff --git a/eagle-server/src/main/java/org/apache/eagle/server/module/ServerModule.java b/eagle-server/src/main/java/org/apache/eagle/server/module/ServerModule.java
index 47e34b3..476096c 100644
--- a/eagle-server/src/main/java/org/apache/eagle/server/module/ServerModule.java
+++ b/eagle-server/src/main/java/org/apache/eagle/server/module/ServerModule.java
@@ -16,15 +16,16 @@
  */
 package org.apache.eagle.server.module;
 
-import com.google.inject.AbstractModule;
 import org.apache.eagle.app.module.ApplicationGuiceModule;
 import org.apache.eagle.app.service.ApplicationProviderService;
 import org.apache.eagle.common.module.CommonGuiceModule;
 import org.apache.eagle.metadata.persistence.MetadataStoreModuleFactory;
+import com.google.inject.AbstractModule;
 
 public class ServerModule extends AbstractModule {
     private ApplicationProviderService appProviderInst;
-    public ServerModule(ApplicationProviderService appProviderInst){
+
+    public ServerModule(ApplicationProviderService appProviderInst) {
         this.appProviderInst = appProviderInst;
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/21187b55/eagle-server/src/main/resources/application.conf
----------------------------------------------------------------------
diff --git a/eagle-server/src/main/resources/application.conf b/eagle-server/src/main/resources/application.conf
index 6db7707..2c076af 100644
--- a/eagle-server/src/main/resources/application.conf
+++ b/eagle-server/src/main/resources/application.conf
@@ -13,51 +13,81 @@
 # See the License for the specific language governing permissions and
 # limitations under the License.
 
-{
-	"coordinator" : {
-		"policiesPerBolt" : 5,
-		"boltParallelism" : 5,
-		"policyDefaultParallelism" : 5,
-		"boltLoadUpbound": 0.8,
-		"topologyLoadUpbound" : 0.8,
-		"numOfAlertBoltsPerTopology" : 5,
-		"zkConfig" : {
-			"zkQuorum" : "server.eagle.apache.org:2181",
-			"zkRoot" : "/alert",
-			"zkSessionTimeoutMs" : 10000,
-			"connectionTimeoutMs" : 10000,
-			"zkRetryTimes" : 3,
-			"zkRetryInterval" : 3000
-		},
-		"metadataService" : {
-			"host" : "localhost",
-			"port" : 9090,
-			"context" : "/rest"
-		},
-		"metadataDynamicCheck" : {
-			"initDelayMillis" : 1000,
-			"delayMillis" : 30000
-		}
-	},
-	"metadata":{
-		"store": "org.apache.eagle.metadata.service.memory.MemoryMetadataStore"
-		"jdbc":{
-      "username": "root"
-      "password": ""
-      "driverClassName":"com.mysql.jdbc.Driver"
-			"url":"jdbc:mysql://server.eagle.apache.org:3306/eagle"
-		}
-	},
-	"application":{
-		"sink":{
-			"type": "org.apache.eagle.app.sink.KafkaStreamSink"
-		},
-		"storm": {
-			"nimbusHost": "server.eagle.apache.org"
-			"nimbusThriftPort": 6627
-		},
-		"provider" : {
-//			"dir" : "/tmp/eagle"
-		}
-	}
+# ---------------------------------------------
+# Eagle REST Web Service Configuration
+# ---------------------------------------------
+service {
+  host = "localhost"
+  port = 9090
+  context = "/rest"
 }
+
+# ---------------------------------------------
+# Eagle Deep Storage Configuration
+# ---------------------------------------------
+storage {
+  storage-type = "hbase"
+  table-name-prefixed-with-environment = false
+  coprocessor-enabled = false
+  hbase-zookeeper-quorum = "localhost"
+  hbase-zookeeper-property-clientPort = 2181
+  zookeeper-znode-parent = "/hbase-unsecure"
+}
+
+
+# ---------------------------------------------
+# Eagle Metadata Store Configuration
+# ---------------------------------------------
+metadata {
+  store = org.apache.eagle.metadata.service.memory.MemoryMetadataStore
+  jdbc {
+    username = "root"
+    password = ""
+    driverClassName = com.mysql.jdbc.Driver
+    url = "jdbc:mysql://server.eagle.apache.org:3306/eagle"
+  }
+}
+
+# ---------------------------------------------
+# Eagle Application Configuration
+# ---------------------------------------------
+application {
+  sink {
+    type = org.apache.eagle.app.sink.KafkaStreamSink
+  }
+  storm {
+    nimbusHost = "server.eagle.apache.org"
+    nimbusThriftPort = 6627
+  }
+}
+
+# ---------------------------------------------
+# Eagle Alert Engine Configuration
+# ---------------------------------------------
+
+# Coordinator Configuration
+coordinator {
+  policiesPerBolt = 5
+  boltParallelism = 5
+  policyDefaultParallelism = 5
+  boltLoadUpbound = 0.8
+  topologyLoadUpbound = 0.8
+  numOfAlertBoltsPerTopology = 5
+  zkConfig {
+    zkQuorum = "server.eagle.apache.org:2181"
+    zkRoot = "/alert"
+    zkSessionTimeoutMs = 10000
+    connectionTimeoutMs = 10000
+    zkRetryTimes = 3
+    zkRetryInterval = 3000
+  }
+  metadataService {
+    host = "localhost",
+    port = 9090,
+    context = "/rest"
+  }
+  metadataDynamicCheck {
+    initDelayMillis = 1000
+    delayMillis = 30000
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/21187b55/eagle-server/src/main/resources/configuration.yml
----------------------------------------------------------------------
diff --git a/eagle-server/src/main/resources/configuration.yml b/eagle-server/src/main/resources/configuration.yml
index c671ade..6dc864d 100644
--- a/eagle-server/src/main/resources/configuration.yml
+++ b/eagle-server/src/main/resources/configuration.yml
@@ -12,10 +12,11 @@
 # 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.
+
 server:
   applicationConnectors:
     - type: http
       port: 9090
   adminConnectors:
     - type: http
-      port: 9091
+      port: 9091
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/21187b55/eagle-server/src/main/resources/log4j.properties
----------------------------------------------------------------------
diff --git a/eagle-server/src/main/resources/log4j.properties b/eagle-server/src/main/resources/log4j.properties
index d59ded6..9c6875d 100644
--- a/eagle-server/src/main/resources/log4j.properties
+++ b/eagle-server/src/main/resources/log4j.properties
@@ -12,9 +12,7 @@
 # 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.
-
 log4j.rootLogger=INFO, stdout
-
 # standard output
 log4j.appender.stdout=org.apache.log4j.ConsoleAppender
 log4j.appender.stdout.layout=org.apache.log4j.PatternLayout

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/21187b55/eagle-server/src/main/resources/providers-disabled.xml
----------------------------------------------------------------------
diff --git a/eagle-server/src/main/resources/providers-disabled.xml b/eagle-server/src/main/resources/providers-disabled.xml
deleted file mode 100644
index b434715..0000000
--- a/eagle-server/src/main/resources/providers-disabled.xml
+++ /dev/null
@@ -1,27 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
-  ~ Licensed to the Apache Software Foundation (ASF) under one or more
-  ~ contributor license agreements.  See the NOTICE file distributed with
-  ~ this work for additional information regarding copyright ownership.
-  ~ The ASF licenses this file to You under the Apache License, Version 2.0
-  ~ (the "License"); you may not use this file except in compliance with
-  ~ the License.  You may obtain a copy of the License at
-  ~
-  ~    http://www.apache.org/licenses/LICENSE-2.0
-  ~
-  ~ Unless required by applicable law or agreed to in writing, software
-  ~ distributed under the License is distributed on an "AS IS" BASIS,
-  ~ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-  ~ See the License for the specific language governing permissions and
-  ~ limitations under the License.
-  -->
-<providers>
-    <provider>
-        <jarPath>target/apache-eagle-example-app.jar</jarPath>
-        <className>org.apache.eagle.app.example.ExampleApplicationProvider</className>
-    </provider>
-    <provider>
-        <jarPath>target/apache-eagle-example-app.jar</jarPath>
-        <className>org.apache.eagle.app.example.ExampleApplicationProvider2</className>
-    </provider>
-</providers>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/21187b55/eagle-server/src/test/java/org/apache/eagle/server/GuideBundleLoaderTest.java
----------------------------------------------------------------------
diff --git a/eagle-server/src/test/java/org/apache/eagle/server/GuideBundleLoaderTest.java b/eagle-server/src/test/java/org/apache/eagle/server/GuideBundleLoaderTest.java
index 4316dfd..b9b6fcf 100644
--- a/eagle-server/src/test/java/org/apache/eagle/server/GuideBundleLoaderTest.java
+++ b/eagle-server/src/test/java/org/apache/eagle/server/GuideBundleLoaderTest.java
@@ -22,7 +22,7 @@ import org.junit.Test;
 
 public class GuideBundleLoaderTest {
     @Test
-    public void testLoad(){
+    public void testLoad() {
         GuiceBundle<ServerConfig> bundle = GuideBundleLoader.load();
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/21187b55/eagle-server/src/test/java/org/apache/eagle/server/ServerApplicationTest.java
----------------------------------------------------------------------
diff --git a/eagle-server/src/test/java/org/apache/eagle/server/ServerApplicationTest.java b/eagle-server/src/test/java/org/apache/eagle/server/ServerApplicationTest.java
index 823981f..6b299e5 100644
--- a/eagle-server/src/test/java/org/apache/eagle/server/ServerApplicationTest.java
+++ b/eagle-server/src/test/java/org/apache/eagle/server/ServerApplicationTest.java
@@ -21,6 +21,6 @@ import org.junit.Test;
 public class ServerApplicationTest {
     @Test
     public void testServerMain() throws Exception {
-        ServerMain.main(new String[]{"server", "src/test/resources/configuration.yml"});
+        ServerMain.main(new String[] {"server", "src/test/resources/configuration.yml"});
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/21187b55/eagle-webservice/src/main/resources/application-derby.conf
----------------------------------------------------------------------
diff --git a/eagle-webservice/src/main/resources/application-derby.conf b/eagle-webservice/src/main/resources/application-derby.conf
index c922a7e..fa72ec0 100644
--- a/eagle-webservice/src/main/resources/application-derby.conf
+++ b/eagle-webservice/src/main/resources/application-derby.conf
@@ -13,8 +13,7 @@
 # See the License for the specific language governing permissions and
 # limitations under the License.
 
-eagle {
-	service {
+storage {
 		storage-type="jdbc"
 		storage-adapter="derby"
 		storage-username="eagle"
@@ -24,7 +23,6 @@ eagle {
 		storage-connection-props="encoding=UTF-8"
 		storage-driver-class="org.apache.derby.jdbc.EmbeddedDriver"
 		storage-connection-max=8
-	}
 }
 
 

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/21187b55/eagle-webservice/src/main/resources/application-hbase.conf
----------------------------------------------------------------------
diff --git a/eagle-webservice/src/main/resources/application-hbase.conf b/eagle-webservice/src/main/resources/application-hbase.conf
index 680be0e..12ade6a 100644
--- a/eagle-webservice/src/main/resources/application-hbase.conf
+++ b/eagle-webservice/src/main/resources/application-hbase.conf
@@ -13,13 +13,11 @@
 # See the License for the specific language governing permissions and
 # limitations under the License.
 
-eagle{
-	service{
+storage {
 		storage-type="hbase"
 		hbase-zookeeper-quorum="localhost"
 		hbase-zookeeper-property-clientPort=2181
 		zookeeper-znode-parent="/hbase",
 		springActiveProfile="sandbox"
 		audit-enabled=true
-	}
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/21187b55/eagle-webservice/src/main/resources/application-mysql.conf
----------------------------------------------------------------------
diff --git a/eagle-webservice/src/main/resources/application-mysql.conf b/eagle-webservice/src/main/resources/application-mysql.conf
index 61a899f..b14a125 100644
--- a/eagle-webservice/src/main/resources/application-mysql.conf
+++ b/eagle-webservice/src/main/resources/application-mysql.conf
@@ -13,8 +13,7 @@
 # See the License for the specific language governing permissions and
 # limitations under the License.
 
-eagle {
-	service {
+storage {
 		storage-type="jdbc"
 		storage-adapter="mysql"
 		storage-username="eagle"
@@ -24,5 +23,4 @@ eagle {
 		storage-connection-props="encoding=UTF-8"
 		storage-driver-class="com.mysql.jdbc.Driver"
 		storage-connection-max=8
-	}
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/21187b55/eagle-webservice/src/main/resources/application.conf
----------------------------------------------------------------------
diff --git a/eagle-webservice/src/main/resources/application.conf b/eagle-webservice/src/main/resources/application.conf
index fabe83f..c1ca0e5 100644
--- a/eagle-webservice/src/main/resources/application.conf
+++ b/eagle-webservice/src/main/resources/application.conf
@@ -13,47 +13,46 @@
 # See the License for the specific language governing permissions and
 # limitations under the License.
 
-{
-	eagle {
-		service {
-			storage-type = "hbase"
-			hbase-zookeeper-quorum = "sandbox.hortonworks.com"
-			hbase-zookeeper-property-clientPort = 2181
-			zookeeper-znode-parent = "/hbase-unsecure",
-			springActiveProfile = "sandbox"
-			audit-enabled = true
-		}
-	}
-	"coordinator" : {
-		"policiesPerBolt" : 5,
-		"boltParallelism" : 5,
-		"policyDefaultParallelism" : 5,
-		"boltLoadUpbound": 0.8,
-		"topologyLoadUpbound" : 0.8,
-		"numOfAlertBoltsPerTopology" : 5,
-		"zkConfig" : {
-			"zkQuorum" : "sandbox.hortonworks.com:2181",
-			"zkRoot" : "/alert",
-			"zkSessionTimeoutMs" : 10000,
-			"connectionTimeoutMs" : 10000,
-			"zkRetryTimes" : 3,
-			"zkRetryInterval" : 3000
-		},
-		"metadataService" : {
-			"host" : "localhost",
-			"port" : 58080,
-			"context" : "/rest"
-		},
-		"metadataDynamicCheck" : {
-			"initDelayMillis" : 1000,
-			"delayMillis" : 30000
-		}
-	}
-	"datastore": {
-		"metadataDao": "org.apache.eagle.alert.metadata.impl.InMemMetadataDaoImpl",
-		"connection": "localhost:27017"
+storage {
+	storage-type = "hbase"
+	hbase-zookeeper-quorum = "sandbox.hortonworks.com"
+	hbase-zookeeper-property-clientPort = 2181
+	zookeeper-znode-parent = "/hbase-unsecure",
+	springActiveProfile = "sandbox"
+	audit-enabled = true
+}
+
+coordinator {
+	"policiesPerBolt" : 5,
+	"boltParallelism" : 5,
+	"policyDefaultParallelism" : 5,
+	"boltLoadUpbound": 0.8,
+	"topologyLoadUpbound" : 0.8,
+	"numOfAlertBoltsPerTopology" : 5,
+	"zkConfig" : {
+		"zkQuorum" : "sandbox.hortonworks.com:2181",
+		"zkRoot" : "/alert",
+		"zkSessionTimeoutMs" : 10000,
+		"connectionTimeoutMs" : 10000,
+		"zkRetryTimes" : 3,
+		"zkRetryInterval" : 3000
+	},
+	"metadataService" : {
+		"host" : "localhost",
+		"port" : 58080,
+		"context" : "/rest"
+	},
+	"metadataDynamicCheck" : {
+		"initDelayMillis" : 1000,
+		"delayMillis" : 30000
 	}
 }
 
+datastore {
+	"metadataDao": "org.apache.eagle.alert.metadata.impl.InMemMetadataDaoImpl",
+	"connection": "localhost:27017"
+}
+
+
 
 

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/21187b55/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 2a5b88a..0b9eeac 100755
--- a/pom.xml
+++ b/pom.xml
@@ -250,7 +250,6 @@
 
         <!-- Query -->
         <tomcat.version>7.0.42</tomcat.version>
-        <protobuf-java.version>2.6.1</protobuf-java.version>
         <antlr4.version>4.5</antlr4.version>
         <groovy.version>2.3.4</groovy.version>
         <parsii.version>1.3</parsii.version>
@@ -261,7 +260,7 @@
         <mysql-connector-java.version>5.1.34</mysql-connector-java.version>
         <ddlutils.version>1.0</ddlutils.version>
         <torque.version>4.0</torque.version>
-        <!--<derby.version>10.1.3.1</derby.version>-->
+        <derby.version>10.1.3.1</derby.version>
         <derby.version>10.10.2.0</derby.version>
 
         <!-- REST Service-->
@@ -503,14 +502,14 @@
                 <scope>compile</scope>
             </dependency>
             <dependency>
-                <groupId>org.codehaus.jackson</groupId>
-                <artifactId>jackson-mapper-asl</artifactId>
-                <version>${codehaus-jackson.version}</version>
+                <groupId>com.fasterxml.jackson.jaxrs</groupId>
+                <artifactId>jackson-jaxrs-json-provider</artifactId>
+                <version>${fasterxml-jackson.version}</version>
             </dependency>
             <dependency>
-                <groupId>org.codehaus.jackson</groupId>
-                <artifactId>jackson-core-asl</artifactId>
-                <version>${codehaus-jackson.version}</version>
+                <groupId>com.fasterxml.jackson.jaxrs</groupId>
+                <artifactId>jackson-jaxrs-base</artifactId>
+                <version>${fasterxml-jackson.version}</version>
             </dependency>
             <dependency>
                 <groupId>com.google.protobuf</groupId>
@@ -522,16 +521,16 @@
                 <artifactId>json</artifactId>
                 <version>${org.json.version}</version>
             </dependency>
-            <dependency>
-                <groupId>org.codehaus.jackson</groupId>
-                <artifactId>jackson-jaxrs</artifactId>
-                <version>${codehaus-jackson.version}</version>
-            </dependency>
-            <dependency>
-                <groupId>org.codehaus.jackson</groupId>
-                <artifactId>jackson-xc</artifactId>
-                <version>${codehaus-jackson.version}</version>
-            </dependency>
+            <!--<dependency>-->
+                <!--<groupId>org.codehaus.jackson</groupId>-->
+                <!--<artifactId>jackson-jaxrs</artifactId>-->
+                <!--<version>${codehaus-jackson.version}</version>-->
+            <!--</dependency>-->
+            <!--<dependency>-->
+                <!--<groupId>org.codehaus.jackson</groupId>-->
+                <!--<artifactId>jackson-xc</artifactId>-->
+                <!--<version>${codehaus-jackson.version}</version>-->
+            <!--</dependency>-->
             <dependency>
                 <groupId>com.google.code.gson</groupId>
                 <artifactId>gson</artifactId>


[31/52] [abbrv] incubator-eagle git commit: [EAGLE-502] Always retry to parse spark history job when failure happens.

Posted by yo...@apache.org.
[EAGLE-502] Always retry to parse spark history job when failure happens.

Author: pkuwm <ih...@gmail.com>

Closes #390 from pkuwm/EAGLE-502.


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

Branch: refs/heads/master
Commit: 3f7004f1cc939ba2e633e38248c322d2216e48a9
Parents: e778775
Author: pkuwm <ih...@gmail.com>
Authored: Fri Sep 2 10:51:31 2016 +0800
Committer: Qingwen Zhao <qi...@gmail.com>
Committed: Fri Sep 2 10:51:31 2016 +0800

----------------------------------------------------------------------
 .../jpm/spark/crawl/JHFSparkEventReader.java    |  6 +--
 .../status/JobHistoryZKStateManager.java        | 26 ++++--------
 .../history/storm/SparkHistoryJobSpout.java     | 44 ++++++--------------
 .../src/main/resources/application.conf         |  2 +-
 .../jpm/spark/running/SparkRunningJobApp.java   |  2 +-
 .../storm/SparkRunningJobFetchSpout.java        |  4 +-
 .../running/storm/SparkRunningJobParseBolt.java |  1 +
 .../jpm/util/resourcefetch/ResourceFetcher.java |  2 +-
 .../SparkHistoryServerResourceFetcher.java      |  6 +--
 .../hbase/HbaseMetadataBrowseWebResource.java   | 16 ++++---
 10 files changed, 41 insertions(+), 68 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/3f7004f1/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/spark/crawl/JHFSparkEventReader.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/spark/crawl/JHFSparkEventReader.java b/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/spark/crawl/JHFSparkEventReader.java
index fe02da5..6c68b48 100644
--- a/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/spark/crawl/JHFSparkEventReader.java
+++ b/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/spark/crawl/JHFSparkEventReader.java
@@ -17,14 +17,14 @@
 
 package org.apache.eagle.jpm.spark.crawl;
 
+import com.typesafe.config.Config;
+import com.typesafe.config.ConfigFactory;
 import org.apache.commons.lang.ArrayUtils;
 import org.apache.eagle.jpm.spark.entity.*;
 import org.apache.eagle.jpm.util.*;
 import org.apache.eagle.log.base.taggedlog.TaggedLogAPIEntity;
 import org.apache.eagle.service.client.impl.EagleServiceBaseClient;
 import org.apache.eagle.service.client.impl.EagleServiceClientImpl;
-import com.typesafe.config.Config;
-import com.typesafe.config.ConfigFactory;
 import org.json.simple.JSONArray;
 import org.json.simple.JSONObject;
 import org.slf4j.Logger;
@@ -353,7 +353,7 @@ public class JHFSparkEventReader {
         stage.setCompleteTime(completeTime);
         this.lastEventTime = completeTime;
 
-        if (stageInfo.containsKey("Failure Reason")) {
+        if (stageInfo != null && stageInfo.containsKey("Failure Reason")) {
             stage.setStatus(SparkEntityConstant.SparkStageStatus.FAILED.toString());
         } else {
             stage.setStatus(SparkEntityConstant.SparkStageStatus.COMPLETE.toString());

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/3f7004f1/eagle-jpm/eagle-jpm-spark-history/src/main/java/org/apache/eagle/jpm/spark/history/status/JobHistoryZKStateManager.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-spark-history/src/main/java/org/apache/eagle/jpm/spark/history/status/JobHistoryZKStateManager.java b/eagle-jpm/eagle-jpm-spark-history/src/main/java/org/apache/eagle/jpm/spark/history/status/JobHistoryZKStateManager.java
index 7a95e56..9fafc1f 100644
--- a/eagle-jpm/eagle-jpm-spark-history/src/main/java/org/apache/eagle/jpm/spark/history/status/JobHistoryZKStateManager.java
+++ b/eagle-jpm/eagle-jpm-spark-history/src/main/java/org/apache/eagle/jpm/spark/history/status/JobHistoryZKStateManager.java
@@ -35,10 +35,11 @@ import java.util.Iterator;
 import java.util.List;
 
 public class JobHistoryZKStateManager {
-    public static final Logger LOG = LoggerFactory.getLogger(JobHistoryZKStateManager.class);
+    private final static Logger LOG = LoggerFactory.getLogger(JobHistoryZKStateManager.class);
+
+    private final static String START_TIMESTAMP = "lastAppTime";
     private String zkRoot;
     private CuratorFramework curator;
-    private static String START_TIMESTAMP = "lastAppTime";
 
     private CuratorFramework newCurator(SparkHistoryJobAppConfig config) throws Exception {
         return CuratorFrameworkFactory.newClient(
@@ -72,7 +73,7 @@ public class JobHistoryZKStateManager {
         InterProcessLock lock = new InterProcessReadWriteLock(curator,jobPath).writeLock();
         try {
             lock.acquire();
-            Iterator<String> iter =  curator.getChildren().forPath(jobPath).iterator();
+            Iterator<String> iter = curator.getChildren().forPath(jobPath).iterator();
             while (iter.hasNext()) {
                 String appId = iter.next();
                 String path = jobPath + "/" + appId;
@@ -104,9 +105,7 @@ public class JobHistoryZKStateManager {
         InterProcessLock lock = new InterProcessReadWriteLock(curator,jobPath).writeLock();
         try {
             lock.acquire();
-            Iterator<String> iter =  curator.getChildren().forPath(jobPath).iterator();
-            while (iter.hasNext()) {
-                String appId = iter.next();
+            (curator.getChildren().forPath(jobPath)).forEach(appId -> {
                 String path = jobPath + "/" + appId;
                 try {
                     if (curator.checkExists().forPath(path) != null) {
@@ -119,7 +118,7 @@ public class JobHistoryZKStateManager {
                     LOG.error("fail to read unprocessed job", e);
                     throw new RuntimeException(e);
                 }
-            }
+            });
 
         } catch (Exception e) {
             LOG.error("fail to read unprocessed jobs", e);
@@ -174,10 +173,7 @@ public class JobHistoryZKStateManager {
     public boolean hasApplication(String appId) {
         String path = zkRoot + "/jobs/" + appId;
         try {
-            if (curator.checkExists().forPath(path) != null) {
-                return true;
-            }
-            return false;
+            return curator.checkExists().forPath(path) != null;
         } catch (Exception e) {
             LOG.error("fail to check whether application exists", e);
             throw new RuntimeException(e);
@@ -191,7 +187,7 @@ public class JobHistoryZKStateManager {
                 curator.delete().deletingChildrenIfNeeded().forPath(path);
             }
 
-            name = name.replace("/","_");
+            name = name.replace("/", "_");
             if (name.length() > 50) {
                 name = name.substring(0, 50);
             }
@@ -226,7 +222,6 @@ public class JobHistoryZKStateManager {
     }
 
     public void updateApplicationStatus(String appId, Enum<ZKStateConstant.AppStatus> status) {
-
         String path = zkRoot + "/jobs/" + appId ;
         InterProcessLock lock = new InterProcessReadWriteLock(curator,zkRoot + "/jobs").readLock();
         try {
@@ -238,8 +233,7 @@ public class JobHistoryZKStateManager {
                     curator.setData().forPath(path, status.toString().getBytes("UTF-8"));
                 }
             } else {
-                String errorMsg = String.format("fail to update for application with path %s", path);
-                LOG.error(errorMsg);
+                LOG.error("Failed to update for application with path: " + path);
             }
         } catch (Exception e) {
             LOG.error("fail to update application status", e);
@@ -252,8 +246,6 @@ public class JobHistoryZKStateManager {
             } catch (Exception e) {
                 LOG.error("fail to release lock",e);
             }
-
         }
-
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/3f7004f1/eagle-jpm/eagle-jpm-spark-history/src/main/java/org/apache/eagle/jpm/spark/history/storm/SparkHistoryJobSpout.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-spark-history/src/main/java/org/apache/eagle/jpm/spark/history/storm/SparkHistoryJobSpout.java b/eagle-jpm/eagle-jpm-spark-history/src/main/java/org/apache/eagle/jpm/spark/history/storm/SparkHistoryJobSpout.java
index db60744..5602b4c 100644
--- a/eagle-jpm/eagle-jpm-spark-history/src/main/java/org/apache/eagle/jpm/spark/history/storm/SparkHistoryJobSpout.java
+++ b/eagle-jpm/eagle-jpm-spark-history/src/main/java/org/apache/eagle/jpm/spark/history/storm/SparkHistoryJobSpout.java
@@ -19,6 +19,13 @@
 
 package org.apache.eagle.jpm.spark.history.storm;
 
+import backtype.storm.spout.SpoutOutputCollector;
+import backtype.storm.task.TopologyContext;
+import backtype.storm.topology.OutputFieldsDeclarer;
+import backtype.storm.topology.base.BaseRichSpout;
+import backtype.storm.tuple.Fields;
+import backtype.storm.tuple.Values;
+import backtype.storm.utils.Utils;
 import org.apache.eagle.jpm.spark.history.SparkHistoryJobAppConfig;
 import org.apache.eagle.jpm.spark.history.status.JobHistoryZKStateManager;
 import org.apache.eagle.jpm.spark.history.status.ZKStateConstant;
@@ -26,16 +33,10 @@ import org.apache.eagle.jpm.util.Constants;
 import org.apache.eagle.jpm.util.resourcefetch.RMResourceFetcher;
 import org.apache.eagle.jpm.util.resourcefetch.ResourceFetcher;
 import org.apache.eagle.jpm.util.resourcefetch.model.AppInfo;
-import backtype.storm.spout.SpoutOutputCollector;
-import backtype.storm.task.TopologyContext;
-import backtype.storm.topology.OutputFieldsDeclarer;
-import backtype.storm.topology.base.BaseRichSpout;
-import backtype.storm.tuple.Fields;
-import backtype.storm.tuple.Values;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+
 import java.util.Calendar;
-import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 
@@ -46,9 +47,6 @@ public class SparkHistoryJobSpout extends BaseRichSpout {
     private SparkHistoryJobAppConfig config;
     private ResourceFetcher rmFetch;
     private long lastFinishAppTime = 0;
-    private Map<String, Integer> failTimes;
-
-    private static final int FAIL_MAX_TIMES = 5;
 
     public SparkHistoryJobSpout(SparkHistoryJobAppConfig config) {
         this.config = config;
@@ -57,7 +55,6 @@ public class SparkHistoryJobSpout extends BaseRichSpout {
     @Override
     public void open(Map map, TopologyContext topologyContext, SpoutOutputCollector spoutOutputCollector) {
         rmFetch = new RMResourceFetcher(config.jobHistoryConfig.rms);
-        this.failTimes = new HashMap<>();
         this.collector = spoutOutputCollector;
         this.zkState = new JobHistoryZKStateManager(config);
         this.lastFinishAppTime = zkState.readLastFinishedTimestamp();
@@ -75,7 +72,6 @@ public class SparkHistoryJobSpout extends BaseRichSpout {
             LOG.info("Last finished time = {}", calendar.getTime());
             if (fetchTime - this.lastFinishAppTime > this.config.stormConfig.spoutCrawlInterval) {
                 List<AppInfo> appInfos = rmFetch.getResource(Constants.ResourceType.COMPLETE_SPARK_JOB, Long.toString(lastFinishAppTime));
-                //List<AppInfo> appInfos = (null != apps ? (List<AppInfo>)apps.get(0):new ArrayList<AppInfo>());
                 if (appInfos != null) {
                     LOG.info("Get " + appInfos.size() + " from yarn resource manager.");
                     for (AppInfo app : appInfos) {
@@ -120,29 +116,15 @@ public class SparkHistoryJobSpout extends BaseRichSpout {
 
     @Override
     public void fail(Object msgId) {
-        String appId = (String) msgId;
-        int failTimes = 0;
-        if (this.failTimes.containsKey(appId)) {
-            failTimes = this.failTimes.get(appId);
-        }
-        failTimes++;
-        if (failTimes >= FAIL_MAX_TIMES) {
-            this.failTimes.remove(appId);
-            zkState.updateApplicationStatus(appId, ZKStateConstant.AppStatus.FINISHED);
-            LOG.error(String.format("Application %s has failed for over %s times, drop it.", appId, FAIL_MAX_TIMES));
-        } else {
-            this.failTimes.put(appId, failTimes);
-            collector.emit(new Values(appId), appId);
-            zkState.updateApplicationStatus(appId, ZKStateConstant.AppStatus.SENT_FOR_PARSE);
-        }
+        // Sleep 3 seconds and retry.
+        Utils.sleep(3000);
+
+        collector.emit(new Values(msgId), msgId);
+        zkState.updateApplicationStatus((String)msgId, ZKStateConstant.AppStatus.SENT_FOR_PARSE);
     }
 
     @Override
     public void ack(Object msgId) {
-        String appId = (String) msgId;
-        if (this.failTimes.containsKey(appId)) {
-            this.failTimes.remove(appId);
-        }
 
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/3f7004f1/eagle-jpm/eagle-jpm-spark-history/src/main/resources/application.conf
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-spark-history/src/main/resources/application.conf b/eagle-jpm/eagle-jpm-spark-history/src/main/resources/application.conf
index 289c6f7..483e2e9 100644
--- a/eagle-jpm/eagle-jpm-spark-history/src/main/resources/application.conf
+++ b/eagle-jpm/eagle-jpm-spark-history/src/main/resources/application.conf
@@ -17,7 +17,7 @@
 {
   "basic":{
     "cluster":"sandbox",
-    "datacenter":"sandbox",
+    "dataCenter":"sandbox",
     jobConf.additional.info: ""
   },
   "eagleProps":{

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/3f7004f1/eagle-jpm/eagle-jpm-spark-running/src/main/java/org/apache/eagle/jpm/spark/running/SparkRunningJobApp.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-spark-running/src/main/java/org/apache/eagle/jpm/spark/running/SparkRunningJobApp.java b/eagle-jpm/eagle-jpm-spark-running/src/main/java/org/apache/eagle/jpm/spark/running/SparkRunningJobApp.java
index 5e21406..2ee2a04 100644
--- a/eagle-jpm/eagle-jpm-spark-running/src/main/java/org/apache/eagle/jpm/spark/running/SparkRunningJobApp.java
+++ b/eagle-jpm/eagle-jpm-spark-running/src/main/java/org/apache/eagle/jpm/spark/running/SparkRunningJobApp.java
@@ -65,4 +65,4 @@ public class SparkRunningJobApp extends StormApplication {
 
         return topologyBuilder.createTopology();
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/3f7004f1/eagle-jpm/eagle-jpm-spark-running/src/main/java/org/apache/eagle/jpm/spark/running/storm/SparkRunningJobFetchSpout.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-spark-running/src/main/java/org/apache/eagle/jpm/spark/running/storm/SparkRunningJobFetchSpout.java b/eagle-jpm/eagle-jpm-spark-running/src/main/java/org/apache/eagle/jpm/spark/running/storm/SparkRunningJobFetchSpout.java
index 7162bac..76c7815 100644
--- a/eagle-jpm/eagle-jpm-spark-running/src/main/java/org/apache/eagle/jpm/spark/running/storm/SparkRunningJobFetchSpout.java
+++ b/eagle-jpm/eagle-jpm-spark-running/src/main/java/org/apache/eagle/jpm/spark/running/storm/SparkRunningJobFetchSpout.java
@@ -67,6 +67,7 @@ public class SparkRunningJobFetchSpout extends BaseRichSpout {
         this.sparkRunningJobManager = new SparkRunningJobManager(zkStateConfig);
     }
 
+    @SuppressWarnings("unchecked")
     @Override
     public void nextTuple() {
         LOG.info("Start to fetch spark running jobs");
@@ -154,8 +155,7 @@ public class SparkRunningJobFetchSpout extends BaseRichSpout {
         //content of path /apps/spark/running/yarnAppId/appId is SparkAppEntity(current attempt)
         //as we know, a yarn application may contains many spark applications
         //so, the returned results is a Map, key is yarn appId
-        Map<String, Map<String, SparkAppEntity>> result = this.sparkRunningJobManager.recover();
-        return result;
+        return this.sparkRunningJobManager.recover();
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/3f7004f1/eagle-jpm/eagle-jpm-spark-running/src/main/java/org/apache/eagle/jpm/spark/running/storm/SparkRunningJobParseBolt.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-spark-running/src/main/java/org/apache/eagle/jpm/spark/running/storm/SparkRunningJobParseBolt.java b/eagle-jpm/eagle-jpm-spark-running/src/main/java/org/apache/eagle/jpm/spark/running/storm/SparkRunningJobParseBolt.java
index a497e29..9c0ffef 100644
--- a/eagle-jpm/eagle-jpm-spark-running/src/main/java/org/apache/eagle/jpm/spark/running/storm/SparkRunningJobParseBolt.java
+++ b/eagle-jpm/eagle-jpm-spark-running/src/main/java/org/apache/eagle/jpm/spark/running/storm/SparkRunningJobParseBolt.java
@@ -71,6 +71,7 @@ public class SparkRunningJobParseBolt extends BaseRichBolt {
         this.resourceFetcher = new RMResourceFetcher(endpointConfig.rmUrls);
     }
 
+    @SuppressWarnings("unchecked")
     @Override
     public void execute(Tuple tuple) {
         AppInfo appInfo = (AppInfo)tuple.getValue(1);

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/3f7004f1/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/ResourceFetcher.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/ResourceFetcher.java b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/ResourceFetcher.java
index f920ddb..4999315 100644
--- a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/ResourceFetcher.java
+++ b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/ResourceFetcher.java
@@ -22,6 +22,6 @@ import java.util.List;
 
 public interface ResourceFetcher<T> {
     //continue to refactor later
-    List<T> getResource(Constants.ResourceType resoureType, Object... parameter) throws Exception;
+    List<T> getResource(Constants.ResourceType resourceType, Object... parameter) throws Exception;
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/3f7004f1/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/SparkHistoryServerResourceFetcher.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/SparkHistoryServerResourceFetcher.java b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/SparkHistoryServerResourceFetcher.java
index ce2d9b8..d9390c1 100644
--- a/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/SparkHistoryServerResourceFetcher.java
+++ b/eagle-jpm/eagle-jpm-util/src/main/java/org/apache/eagle/jpm/util/resourcefetch/SparkHistoryServerResourceFetcher.java
@@ -77,12 +77,12 @@ public class SparkHistoryServerResourceFetcher implements ResourceFetcher<SparkA
         }
     }
 
-    public List<SparkApplication> getResource(Constants.ResourceType resoureType, Object... parameter) throws Exception {
-        switch (resoureType) {
+    public List<SparkApplication> getResource(Constants.ResourceType resourceType, Object... parameter) throws Exception {
+        switch (resourceType) {
             case SPARK_JOB_DETAIL:
                 return doFetchSparkApplicationDetail((String) parameter[0]);
             default:
-                throw new Exception("Not support resourceType :" + resoureType);
+                throw new Exception("Not support resourceType :" + resourceType);
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/3f7004f1/eagle-security/eagle-security-hbase-web/src/main/java/org/apache/eagle/service/security/hbase/HbaseMetadataBrowseWebResource.java
----------------------------------------------------------------------
diff --git a/eagle-security/eagle-security-hbase-web/src/main/java/org/apache/eagle/service/security/hbase/HbaseMetadataBrowseWebResource.java b/eagle-security/eagle-security-hbase-web/src/main/java/org/apache/eagle/service/security/hbase/HbaseMetadataBrowseWebResource.java
index 68ea552..69be2e5 100644
--- a/eagle-security/eagle-security-hbase-web/src/main/java/org/apache/eagle/service/security/hbase/HbaseMetadataBrowseWebResource.java
+++ b/eagle-security/eagle-security-hbase-web/src/main/java/org/apache/eagle/service/security/hbase/HbaseMetadataBrowseWebResource.java
@@ -73,7 +73,7 @@ public class HbaseMetadataBrowseWebResource {
     private Configuration convert(Map<String, Object> originalConfig) throws Exception {
         Configuration config = new Configuration();
         for (Map.Entry<String, Object> entry : originalConfig.entrySet()) {
-            config.set(entry.getKey().toString(), entry.getValue().toString());
+            config.set(entry.getKey(), entry.getValue().toString());
         }
         return config;
     }
@@ -125,12 +125,11 @@ public class HbaseMetadataBrowseWebResource {
             response.setException(EagleExceptionWrapper.wrap(ex));
         }
         if(tables != null) {
-            String resource = null;
             for (String table : tables) {
-                resource = String.format("%s:%s", namespace, table);
+                String resource = String.format("%s:%s", namespace, table);
                 Set<String> childSensitiveTypes = new HashSet<>();
-                String senstiveType = checkSensitivity(site, resource, childSensitiveTypes);
-                values.add(new HbaseResourceEntity(resource, namespace, table, null, senstiveType, childSensitiveTypes));
+                String sensitiveType = checkSensitivity(site, resource, childSensitiveTypes);
+                values.add(new HbaseResourceEntity(resource, namespace, table, null, sensitiveType, childSensitiveTypes));
             }
         }
         response.setObj(values);
@@ -157,12 +156,11 @@ public class HbaseMetadataBrowseWebResource {
             response.setException(EagleExceptionWrapper.wrap(ex));
         }
         if(columns != null) {
-            String resource = null;
             for (String col : columns) {
-                resource = String.format("%s:%s:%s", namespace, table, col);
+                String resource = String.format("%s:%s:%s", namespace, table, col);
                 Set<String> childSensitiveTypes = new HashSet<>();
-                String senstiveType = checkSensitivity(site, resource, childSensitiveTypes);
-                values.add(new HbaseResourceEntity(resource, namespace, table, col, senstiveType, childSensitiveTypes));
+                String sensitiveType = checkSensitivity(site, resource, childSensitiveTypes);
+                values.add(new HbaseResourceEntity(resource, namespace, table, col, sensitiveType, childSensitiveTypes));
             }
         }
         response.setObj(values);


[48/52] [abbrv] incubator-eagle git commit: [EAGLE-520] Fix and decouple co-processor from eagle aggreation query service

Posted by yo...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/21187b55/eagle-core/eagle-query/eagle-storage-hbase/src/main/java/org/apache/eagle/storage/hbase/query/coprocessor/generated/AggregateProtos.java
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-query/eagle-storage-hbase/src/main/java/org/apache/eagle/storage/hbase/query/coprocessor/generated/AggregateProtos.java b/eagle-core/eagle-query/eagle-storage-hbase/src/main/java/org/apache/eagle/storage/hbase/query/coprocessor/generated/AggregateProtos.java
index 2aee11b..369ffa7 100644
--- a/eagle-core/eagle-query/eagle-storage-hbase/src/main/java/org/apache/eagle/storage/hbase/query/coprocessor/generated/AggregateProtos.java
+++ b/eagle-core/eagle-query/eagle-storage-hbase/src/main/java/org/apache/eagle/storage/hbase/query/coprocessor/generated/AggregateProtos.java
@@ -1,4214 +1,4236 @@
-/*
- * 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.
- */
-// Generated by the protocol buffer compiler.  DO NOT EDIT!
-// source: AggregateProtocol.proto
-
-package org.apache.eagle.storage.hbase.query.coprocessor.generated;
-
-import com.google.protobuf.AbstractMessage;
-
-public final class AggregateProtos {
-  private AggregateProtos() {}
-  public static void registerAllExtensions(
-      com.google.protobuf.ExtensionRegistry registry) {
-  }
-  public interface EntityDefinitionOrBuilder extends
-      // @@protoc_insertion_point(interface_extends:EntityDefinition)
-      com.google.protobuf.MessageOrBuilder {
-
-    /**
-     * <code>required bytes byte_array = 1;</code>
-     */
-    boolean hasByteArray();
-    /**
-     * <code>required bytes byte_array = 1;</code>
-     */
-    com.google.protobuf.ByteString getByteArray();
-  }
-  /**
-   * Protobuf type {@code EntityDefinition}
-   */
-  public static final class EntityDefinition extends
-      com.google.protobuf.GeneratedMessage implements
-      // @@protoc_insertion_point(message_implements:EntityDefinition)
-      EntityDefinitionOrBuilder {
-    // Use EntityDefinition.newBuilder() to construct.
-    private EntityDefinition(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
-      super(builder);
-      this.unknownFields = builder.getUnknownFields();
-    }
-    private EntityDefinition(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
-
-    private static final EntityDefinition defaultInstance;
-    public static EntityDefinition getDefaultInstance() {
-      return defaultInstance;
-    }
-
-    public EntityDefinition getDefaultInstanceForType() {
-      return defaultInstance;
-    }
-
-    private final com.google.protobuf.UnknownFieldSet unknownFields;
-    @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
-        getUnknownFields() {
-      return this.unknownFields;
-    }
-    private EntityDefinition(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
-      initFields();
-      int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
-      try {
-        boolean done = false;
-        while (!done) {
-          int tag = input.readTag();
-          switch (tag) {
-            case 0:
-              done = true;
-              break;
-            default: {
-              if (!parseUnknownField(input, unknownFields,
-                                     extensionRegistry, tag)) {
-                done = true;
-              }
-              break;
-            }
-            case 10: {
-              bitField0_ |= 0x00000001;
-              byteArray_ = input.readBytes();
-              break;
-            }
-          }
-        }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
-        throw e.setUnfinishedMessage(this);
-      } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
-            e.getMessage()).setUnfinishedMessage(this);
-      } finally {
-        this.unknownFields = unknownFields.build();
-        makeExtensionsImmutable();
-      }
-    }
-    public static final com.google.protobuf.Descriptors.Descriptor
-        getDescriptor() {
-      return AggregateProtos.internal_static_EntityDefinition_descriptor;
-    }
-
-    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
-        internalGetFieldAccessorTable() {
-      return AggregateProtos.internal_static_EntityDefinition_fieldAccessorTable
-          .ensureFieldAccessorsInitialized(
-              AggregateProtos.EntityDefinition.class, AggregateProtos.EntityDefinition.Builder.class);
-    }
-
-    public static com.google.protobuf.Parser<EntityDefinition> PARSER =
-        new com.google.protobuf.AbstractParser<EntityDefinition>() {
-      public EntityDefinition parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
-        return new EntityDefinition(input, extensionRegistry);
-      }
-    };
-
-    @java.lang.Override
-    public com.google.protobuf.Parser<EntityDefinition> getParserForType() {
-      return PARSER;
-    }
-
-    private int bitField0_;
-    public static final int BYTE_ARRAY_FIELD_NUMBER = 1;
-    private com.google.protobuf.ByteString byteArray_;
-    /**
-     * <code>required bytes byte_array = 1;</code>
-     */
-    public boolean hasByteArray() {
-      return ((bitField0_ & 0x00000001) == 0x00000001);
-    }
-    /**
-     * <code>required bytes byte_array = 1;</code>
-     */
-    public com.google.protobuf.ByteString getByteArray() {
-      return byteArray_;
-    }
-
-    private void initFields() {
-      byteArray_ = com.google.protobuf.ByteString.EMPTY;
-    }
-    private byte memoizedIsInitialized = -1;
-    public final boolean isInitialized() {
-      byte isInitialized = memoizedIsInitialized;
-      if (isInitialized == 1) return true;
-      if (isInitialized == 0) return false;
-
-      if (!hasByteArray()) {
-        memoizedIsInitialized = 0;
-        return false;
-      }
-      memoizedIsInitialized = 1;
-      return true;
-    }
-
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
-                        throws java.io.IOException {
-      getSerializedSize();
-      if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        output.writeBytes(1, byteArray_);
-      }
-      getUnknownFields().writeTo(output);
-    }
-
-    private int memoizedSerializedSize = -1;
-    public int getSerializedSize() {
-      int size = memoizedSerializedSize;
-      if (size != -1) return size;
-
-      size = 0;
-      if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
-          .computeBytesSize(1, byteArray_);
-      }
-      size += getUnknownFields().getSerializedSize();
-      memoizedSerializedSize = size;
-      return size;
-    }
-
-    private static final long serialVersionUID = 0L;
-    @java.lang.Override
-    protected java.lang.Object writeReplace()
-        throws java.io.ObjectStreamException {
-      return super.writeReplace();
-    }
-
-    @java.lang.Override
-    public boolean equals(final java.lang.Object obj) {
-      if (obj == this) {
-       return true;
-      }
-      if (!(obj instanceof AggregateProtos.EntityDefinition)) {
-        return super.equals(obj);
-      }
-      AggregateProtos.EntityDefinition other = (AggregateProtos.EntityDefinition) obj;
-
-      boolean result = true;
-      result = result && (hasByteArray() == other.hasByteArray());
-      if (hasByteArray()) {
-        result = result && getByteArray()
-            .equals(other.getByteArray());
-      }
-      result = result &&
-          getUnknownFields().equals(other.getUnknownFields());
-      return result;
-    }
-
-    @java.lang.Override
-    public int hashCode() {
-      if (memoizedHashCode != 0) {
-        return memoizedHashCode;
-      }
-      int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
-      if (hasByteArray()) {
-        hash = (37 * hash) + BYTE_ARRAY_FIELD_NUMBER;
-        hash = (53 * hash) + getByteArray().hashCode();
-      }
-      hash = (29 * hash) + getUnknownFields().hashCode();
-      memoizedHashCode = hash;
-      return hash;
-    }
-
-    public static AggregateProtos.EntityDefinition parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
-      return PARSER.parseFrom(data);
-    }
-    public static AggregateProtos.EntityDefinition parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
-      return PARSER.parseFrom(data, extensionRegistry);
-    }
-    public static AggregateProtos.EntityDefinition parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
-      return PARSER.parseFrom(data);
-    }
-    public static AggregateProtos.EntityDefinition parseFrom(
-        byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
-      return PARSER.parseFrom(data, extensionRegistry);
-    }
-    public static AggregateProtos.EntityDefinition parseFrom(java.io.InputStream input)
-        throws java.io.IOException {
-      return PARSER.parseFrom(input);
-    }
-    public static AggregateProtos.EntityDefinition parseFrom(
-        java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws java.io.IOException {
-      return PARSER.parseFrom(input, extensionRegistry);
-    }
-    public static AggregateProtos.EntityDefinition parseDelimitedFrom(java.io.InputStream input)
-        throws java.io.IOException {
-      return PARSER.parseDelimitedFrom(input);
-    }
-    public static AggregateProtos.EntityDefinition parseDelimitedFrom(
-        java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws java.io.IOException {
-      return PARSER.parseDelimitedFrom(input, extensionRegistry);
-    }
-    public static AggregateProtos.EntityDefinition parseFrom(
-        com.google.protobuf.CodedInputStream input)
-        throws java.io.IOException {
-      return PARSER.parseFrom(input);
-    }
-    public static AggregateProtos.EntityDefinition parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws java.io.IOException {
-      return PARSER.parseFrom(input, extensionRegistry);
-    }
-
-    public static Builder newBuilder() { return Builder.create(); }
-    public Builder newBuilderForType() { return newBuilder(); }
-    public static Builder newBuilder(AggregateProtos.EntityDefinition prototype) {
-      return newBuilder().mergeFrom(prototype);
-    }
-    public Builder toBuilder() { return newBuilder(this); }
-
-    @java.lang.Override
-    protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessage.BuilderParent parent) {
-      Builder builder = new Builder(parent);
-      return builder;
-    }
-    /**
-     * Protobuf type {@code EntityDefinition}
-     */
-    public static final class Builder extends
-        com.google.protobuf.GeneratedMessage.Builder<Builder> implements
-        // @@protoc_insertion_point(builder_implements:EntityDefinition)
-        AggregateProtos.EntityDefinitionOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
-          getDescriptor() {
-        return AggregateProtos.internal_static_EntityDefinition_descriptor;
-      }
-
-      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
-          internalGetFieldAccessorTable() {
-        return AggregateProtos.internal_static_EntityDefinition_fieldAccessorTable
-            .ensureFieldAccessorsInitialized(
-                AggregateProtos.EntityDefinition.class, AggregateProtos.EntityDefinition.Builder.class);
-      }
-
-      // Construct using eagle.storage.hbase.query.coprocessor.generated.AggregateProtos.EntityDefinition.newBuilder()
-      private Builder() {
-        maybeForceBuilderInitialization();
-      }
-
-      private Builder(
-          com.google.protobuf.GeneratedMessage.BuilderParent parent) {
-        super(parent);
-        maybeForceBuilderInitialization();
-      }
-      private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
-        }
-      }
-      private static Builder create() {
-        return new Builder();
-      }
-
-      public Builder clear() {
-        super.clear();
-        byteArray_ = com.google.protobuf.ByteString.EMPTY;
-        bitField0_ = (bitField0_ & ~0x00000001);
-        return this;
-      }
-
-      public Builder clone() {
-        return create().mergeFrom(buildPartial());
-      }
-
-      public com.google.protobuf.Descriptors.Descriptor
-          getDescriptorForType() {
-        return AggregateProtos.internal_static_EntityDefinition_descriptor;
-      }
-
-      public AggregateProtos.EntityDefinition getDefaultInstanceForType() {
-        return AggregateProtos.EntityDefinition.getDefaultInstance();
-      }
-
-      public AggregateProtos.EntityDefinition build() {
-        AggregateProtos.EntityDefinition result = buildPartial();
-        if (!result.isInitialized()) {
-          throw AbstractMessage.Builder.newUninitializedMessageException(result);
-        }
-        return result;
-      }
-
-      public AggregateProtos.EntityDefinition buildPartial() {
-        AggregateProtos.EntityDefinition result = new AggregateProtos.EntityDefinition(this);
-        int from_bitField0_ = bitField0_;
-        int to_bitField0_ = 0;
-        if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
-          to_bitField0_ |= 0x00000001;
-        }
-        result.byteArray_ = byteArray_;
-        result.bitField0_ = to_bitField0_;
-        onBuilt();
-        return result;
-      }
-
-      public Builder mergeFrom(com.google.protobuf.Message other) {
-        if (other instanceof AggregateProtos.EntityDefinition) {
-          return mergeFrom((AggregateProtos.EntityDefinition)other);
-        } else {
-          super.mergeFrom(other);
-          return this;
-        }
-      }
-
-      public Builder mergeFrom(AggregateProtos.EntityDefinition other) {
-        if (other == AggregateProtos.EntityDefinition.getDefaultInstance()) return this;
-        if (other.hasByteArray()) {
-          setByteArray(other.getByteArray());
-        }
-        this.mergeUnknownFields(other.getUnknownFields());
-        return this;
-      }
-
-      public final boolean isInitialized() {
-        if (!hasByteArray()) {
-          
-          return false;
-        }
-        return true;
-      }
-
-      public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws java.io.IOException {
-        AggregateProtos.EntityDefinition parsedMessage = null;
-        try {
-          parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
-          parsedMessage = (AggregateProtos.EntityDefinition) e.getUnfinishedMessage();
-          throw e;
-        } finally {
-          if (parsedMessage != null) {
-            mergeFrom(parsedMessage);
-          }
-        }
-        return this;
-      }
-      private int bitField0_;
-
-      private com.google.protobuf.ByteString byteArray_ = com.google.protobuf.ByteString.EMPTY;
-      /**
-       * <code>required bytes byte_array = 1;</code>
-       */
-      public boolean hasByteArray() {
-        return ((bitField0_ & 0x00000001) == 0x00000001);
-      }
-      /**
-       * <code>required bytes byte_array = 1;</code>
-       */
-      public com.google.protobuf.ByteString getByteArray() {
-        return byteArray_;
-      }
-      /**
-       * <code>required bytes byte_array = 1;</code>
-       */
-      public Builder setByteArray(com.google.protobuf.ByteString value) {
-        if (value == null) {
-    throw new NullPointerException();
-  }
-  bitField0_ |= 0x00000001;
-        byteArray_ = value;
-        onChanged();
-        return this;
-      }
-      /**
-       * <code>required bytes byte_array = 1;</code>
-       */
-      public Builder clearByteArray() {
-        bitField0_ = (bitField0_ & ~0x00000001);
-        byteArray_ = getDefaultInstance().getByteArray();
-        onChanged();
-        return this;
-      }
-
-      // @@protoc_insertion_point(builder_scope:EntityDefinition)
-    }
-
-    static {
-      defaultInstance = new EntityDefinition(true);
-      defaultInstance.initFields();
-    }
-
-    // @@protoc_insertion_point(class_scope:EntityDefinition)
-  }
-
-  public interface AggregateResultOrBuilder extends
-      // @@protoc_insertion_point(interface_extends:AggregateResult)
-      com.google.protobuf.MessageOrBuilder {
-
-    /**
-     * <code>required bytes byte_array = 1;</code>
-     */
-    boolean hasByteArray();
-    /**
-     * <code>required bytes byte_array = 1;</code>
-     */
-    com.google.protobuf.ByteString getByteArray();
-  }
-  /**
-   * Protobuf type {@code AggregateResult}
-   */
-  public static final class AggregateResult extends
-      com.google.protobuf.GeneratedMessage implements
-      // @@protoc_insertion_point(message_implements:AggregateResult)
-      AggregateResultOrBuilder {
-    // Use AggregateResult.newBuilder() to construct.
-    private AggregateResult(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
-      super(builder);
-      this.unknownFields = builder.getUnknownFields();
-    }
-    private AggregateResult(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
-
-    private static final AggregateResult defaultInstance;
-    public static AggregateResult getDefaultInstance() {
-      return defaultInstance;
-    }
-
-    public AggregateResult getDefaultInstanceForType() {
-      return defaultInstance;
-    }
-
-    private final com.google.protobuf.UnknownFieldSet unknownFields;
-    @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
-        getUnknownFields() {
-      return this.unknownFields;
-    }
-    private AggregateResult(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
-      initFields();
-      int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
-      try {
-        boolean done = false;
-        while (!done) {
-          int tag = input.readTag();
-          switch (tag) {
-            case 0:
-              done = true;
-              break;
-            default: {
-              if (!parseUnknownField(input, unknownFields,
-                                     extensionRegistry, tag)) {
-                done = true;
-              }
-              break;
-            }
-            case 10: {
-              bitField0_ |= 0x00000001;
-              byteArray_ = input.readBytes();
-              break;
-            }
-          }
-        }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
-        throw e.setUnfinishedMessage(this);
-      } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
-            e.getMessage()).setUnfinishedMessage(this);
-      } finally {
-        this.unknownFields = unknownFields.build();
-        makeExtensionsImmutable();
-      }
-    }
-    public static final com.google.protobuf.Descriptors.Descriptor
-        getDescriptor() {
-      return AggregateProtos.internal_static_AggregateResult_descriptor;
-    }
-
-    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
-        internalGetFieldAccessorTable() {
-      return AggregateProtos.internal_static_AggregateResult_fieldAccessorTable
-          .ensureFieldAccessorsInitialized(
-              AggregateProtos.AggregateResult.class, AggregateProtos.AggregateResult.Builder.class);
-    }
-
-    public static com.google.protobuf.Parser<AggregateResult> PARSER =
-        new com.google.protobuf.AbstractParser<AggregateResult>() {
-      public AggregateResult parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
-        return new AggregateResult(input, extensionRegistry);
-      }
-    };
-
-    @java.lang.Override
-    public com.google.protobuf.Parser<AggregateResult> getParserForType() {
-      return PARSER;
-    }
-
-    private int bitField0_;
-    public static final int BYTE_ARRAY_FIELD_NUMBER = 1;
-    private com.google.protobuf.ByteString byteArray_;
-    /**
-     * <code>required bytes byte_array = 1;</code>
-     */
-    public boolean hasByteArray() {
-      return ((bitField0_ & 0x00000001) == 0x00000001);
-    }
-    /**
-     * <code>required bytes byte_array = 1;</code>
-     */
-    public com.google.protobuf.ByteString getByteArray() {
-      return byteArray_;
-    }
-
-    private void initFields() {
-      byteArray_ = com.google.protobuf.ByteString.EMPTY;
-    }
-    private byte memoizedIsInitialized = -1;
-    public final boolean isInitialized() {
-      byte isInitialized = memoizedIsInitialized;
-      if (isInitialized == 1) return true;
-      if (isInitialized == 0) return false;
-
-      if (!hasByteArray()) {
-        memoizedIsInitialized = 0;
-        return false;
-      }
-      memoizedIsInitialized = 1;
-      return true;
-    }
-
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
-                        throws java.io.IOException {
-      getSerializedSize();
-      if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        output.writeBytes(1, byteArray_);
-      }
-      getUnknownFields().writeTo(output);
-    }
-
-    private int memoizedSerializedSize = -1;
-    public int getSerializedSize() {
-      int size = memoizedSerializedSize;
-      if (size != -1) return size;
-
-      size = 0;
-      if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
-          .computeBytesSize(1, byteArray_);
-      }
-      size += getUnknownFields().getSerializedSize();
-      memoizedSerializedSize = size;
-      return size;
-    }
-
-    private static final long serialVersionUID = 0L;
-    @java.lang.Override
-    protected java.lang.Object writeReplace()
-        throws java.io.ObjectStreamException {
-      return super.writeReplace();
-    }
-
-    @java.lang.Override
-    public boolean equals(final java.lang.Object obj) {
-      if (obj == this) {
-       return true;
-      }
-      if (!(obj instanceof AggregateProtos.AggregateResult)) {
-        return super.equals(obj);
-      }
-      AggregateProtos.AggregateResult other = (AggregateProtos.AggregateResult) obj;
-
-      boolean result = true;
-      result = result && (hasByteArray() == other.hasByteArray());
-      if (hasByteArray()) {
-        result = result && getByteArray()
-            .equals(other.getByteArray());
-      }
-      result = result &&
-          getUnknownFields().equals(other.getUnknownFields());
-      return result;
-    }
-
-    @java.lang.Override
-    public int hashCode() {
-      if (memoizedHashCode != 0) {
-        return memoizedHashCode;
-      }
-      int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
-      if (hasByteArray()) {
-        hash = (37 * hash) + BYTE_ARRAY_FIELD_NUMBER;
-        hash = (53 * hash) + getByteArray().hashCode();
-      }
-      hash = (29 * hash) + getUnknownFields().hashCode();
-      memoizedHashCode = hash;
-      return hash;
-    }
-
-    public static AggregateProtos.AggregateResult parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
-      return PARSER.parseFrom(data);
-    }
-    public static AggregateProtos.AggregateResult parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
-      return PARSER.parseFrom(data, extensionRegistry);
-    }
-    public static AggregateProtos.AggregateResult parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
-      return PARSER.parseFrom(data);
-    }
-    public static AggregateProtos.AggregateResult parseFrom(
-        byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
-      return PARSER.parseFrom(data, extensionRegistry);
-    }
-    public static AggregateProtos.AggregateResult parseFrom(java.io.InputStream input)
-        throws java.io.IOException {
-      return PARSER.parseFrom(input);
-    }
-    public static AggregateProtos.AggregateResult parseFrom(
-        java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws java.io.IOException {
-      return PARSER.parseFrom(input, extensionRegistry);
-    }
-    public static AggregateProtos.AggregateResult parseDelimitedFrom(java.io.InputStream input)
-        throws java.io.IOException {
-      return PARSER.parseDelimitedFrom(input);
-    }
-    public static AggregateProtos.AggregateResult parseDelimitedFrom(
-        java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws java.io.IOException {
-      return PARSER.parseDelimitedFrom(input, extensionRegistry);
-    }
-    public static AggregateProtos.AggregateResult parseFrom(
-        com.google.protobuf.CodedInputStream input)
-        throws java.io.IOException {
-      return PARSER.parseFrom(input);
-    }
-    public static AggregateProtos.AggregateResult parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws java.io.IOException {
-      return PARSER.parseFrom(input, extensionRegistry);
-    }
-
-    public static Builder newBuilder() { return Builder.create(); }
-    public Builder newBuilderForType() { return newBuilder(); }
-    public static Builder newBuilder(AggregateProtos.AggregateResult prototype) {
-      return newBuilder().mergeFrom(prototype);
-    }
-    public Builder toBuilder() { return newBuilder(this); }
-
-    @java.lang.Override
-    protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessage.BuilderParent parent) {
-      Builder builder = new Builder(parent);
-      return builder;
-    }
-    /**
-     * Protobuf type {@code AggregateResult}
-     */
-    public static final class Builder extends
-        com.google.protobuf.GeneratedMessage.Builder<Builder> implements
-        // @@protoc_insertion_point(builder_implements:AggregateResult)
-        AggregateProtos.AggregateResultOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
-          getDescriptor() {
-        return AggregateProtos.internal_static_AggregateResult_descriptor;
-      }
-
-      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
-          internalGetFieldAccessorTable() {
-        return AggregateProtos.internal_static_AggregateResult_fieldAccessorTable
-            .ensureFieldAccessorsInitialized(
-                AggregateProtos.AggregateResult.class, AggregateProtos.AggregateResult.Builder.class);
-      }
-
-      // Construct using eagle.storage.hbase.query.coprocessor.generated.AggregateProtos.AggregateResult.newBuilder()
-      private Builder() {
-        maybeForceBuilderInitialization();
-      }
-
-      private Builder(
-          com.google.protobuf.GeneratedMessage.BuilderParent parent) {
-        super(parent);
-        maybeForceBuilderInitialization();
-      }
-      private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
-        }
-      }
-      private static Builder create() {
-        return new Builder();
-      }
-
-      public Builder clear() {
-        super.clear();
-        byteArray_ = com.google.protobuf.ByteString.EMPTY;
-        bitField0_ = (bitField0_ & ~0x00000001);
-        return this;
-      }
-
-      public Builder clone() {
-        return create().mergeFrom(buildPartial());
-      }
-
-      public com.google.protobuf.Descriptors.Descriptor
-          getDescriptorForType() {
-        return AggregateProtos.internal_static_AggregateResult_descriptor;
-      }
-
-      public AggregateProtos.AggregateResult getDefaultInstanceForType() {
-        return AggregateProtos.AggregateResult.getDefaultInstance();
-      }
-
-      public AggregateProtos.AggregateResult build() {
-        AggregateProtos.AggregateResult result = buildPartial();
-        if (!result.isInitialized()) {
-          throw AbstractMessage.Builder.newUninitializedMessageException(result);
-        }
-        return result;
-      }
-
-      public AggregateProtos.AggregateResult buildPartial() {
-        AggregateProtos.AggregateResult result = new AggregateProtos.AggregateResult(this);
-        int from_bitField0_ = bitField0_;
-        int to_bitField0_ = 0;
-        if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
-          to_bitField0_ |= 0x00000001;
-        }
-        result.byteArray_ = byteArray_;
-        result.bitField0_ = to_bitField0_;
-        onBuilt();
-        return result;
-      }
-
-      public Builder mergeFrom(com.google.protobuf.Message other) {
-        if (other instanceof AggregateProtos.AggregateResult) {
-          return mergeFrom((AggregateProtos.AggregateResult)other);
-        } else {
-          super.mergeFrom(other);
-          return this;
-        }
-      }
-
-      public Builder mergeFrom(AggregateProtos.AggregateResult other) {
-        if (other == AggregateProtos.AggregateResult.getDefaultInstance()) return this;
-        if (other.hasByteArray()) {
-          setByteArray(other.getByteArray());
-        }
-        this.mergeUnknownFields(other.getUnknownFields());
-        return this;
-      }
-
-      public final boolean isInitialized() {
-        if (!hasByteArray()) {
-          
-          return false;
-        }
-        return true;
-      }
-
-      public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws java.io.IOException {
-        AggregateProtos.AggregateResult parsedMessage = null;
-        try {
-          parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
-          parsedMessage = (AggregateProtos.AggregateResult) e.getUnfinishedMessage();
-          throw e;
-        } finally {
-          if (parsedMessage != null) {
-            mergeFrom(parsedMessage);
-          }
-        }
-        return this;
-      }
-      private int bitField0_;
-
-      private com.google.protobuf.ByteString byteArray_ = com.google.protobuf.ByteString.EMPTY;
-      /**
-       * <code>required bytes byte_array = 1;</code>
-       */
-      public boolean hasByteArray() {
-        return ((bitField0_ & 0x00000001) == 0x00000001);
-      }
-      /**
-       * <code>required bytes byte_array = 1;</code>
-       */
-      public com.google.protobuf.ByteString getByteArray() {
-        return byteArray_;
-      }
-      /**
-       * <code>required bytes byte_array = 1;</code>
-       */
-      public Builder setByteArray(com.google.protobuf.ByteString value) {
-        if (value == null) {
-    throw new NullPointerException();
-  }
-  bitField0_ |= 0x00000001;
-        byteArray_ = value;
-        onChanged();
-        return this;
-      }
-      /**
-       * <code>required bytes byte_array = 1;</code>
-       */
-      public Builder clearByteArray() {
-        bitField0_ = (bitField0_ & ~0x00000001);
-        byteArray_ = getDefaultInstance().getByteArray();
-        onChanged();
-        return this;
-      }
-
-      // @@protoc_insertion_point(builder_scope:AggregateResult)
-    }
-
-    static {
-      defaultInstance = new AggregateResult(true);
-      defaultInstance.initFields();
-    }
-
-    // @@protoc_insertion_point(class_scope:AggregateResult)
-  }
-
-  public interface AggregateRequestOrBuilder extends
-      // @@protoc_insertion_point(interface_extends:AggregateRequest)
-      com.google.protobuf.MessageOrBuilder {
-
-    /**
-     * <code>required .EntityDefinition entity_definition = 1;</code>
-     */
-    boolean hasEntityDefinition();
-    /**
-     * <code>required .EntityDefinition entity_definition = 1;</code>
-     */
-    AggregateProtos.EntityDefinition getEntityDefinition();
-    /**
-     * <code>required .EntityDefinition entity_definition = 1;</code>
-     */
-    AggregateProtos.EntityDefinitionOrBuilder getEntityDefinitionOrBuilder();
-
-    /**
-     * <code>required .Scan scan = 2;</code>
-     */
-    boolean hasScan();
-    /**
-     * <code>required .Scan scan = 2;</code>
-     */
-    org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Scan getScan();
-    /**
-     * <code>required .Scan scan = 2;</code>
-     */
-    org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanOrBuilder getScanOrBuilder();
-
-    /**
-     * <code>repeated string groupby_fields = 3;</code>
-     */
-    com.google.protobuf.ProtocolStringList
-        getGroupbyFieldsList();
-    /**
-     * <code>repeated string groupby_fields = 3;</code>
-     */
-    int getGroupbyFieldsCount();
-    /**
-     * <code>repeated string groupby_fields = 3;</code>
-     */
-    java.lang.String getGroupbyFields(int index);
-    /**
-     * <code>repeated string groupby_fields = 3;</code>
-     */
-    com.google.protobuf.ByteString
-        getGroupbyFieldsBytes(int index);
-
-    /**
-     * <code>repeated bytes aggregate_func_types = 4;</code>
-     */
-    java.util.List<com.google.protobuf.ByteString> getAggregateFuncTypesList();
-    /**
-     * <code>repeated bytes aggregate_func_types = 4;</code>
-     */
-    int getAggregateFuncTypesCount();
-    /**
-     * <code>repeated bytes aggregate_func_types = 4;</code>
-     */
-    com.google.protobuf.ByteString getAggregateFuncTypes(int index);
-
-    /**
-     * <code>repeated string aggregated_fields = 5;</code>
-     */
-    com.google.protobuf.ProtocolStringList
-        getAggregatedFieldsList();
-    /**
-     * <code>repeated string aggregated_fields = 5;</code>
-     */
-    int getAggregatedFieldsCount();
-    /**
-     * <code>repeated string aggregated_fields = 5;</code>
-     */
-    java.lang.String getAggregatedFields(int index);
-    /**
-     * <code>repeated string aggregated_fields = 5;</code>
-     */
-    com.google.protobuf.ByteString
-        getAggregatedFieldsBytes(int index);
-  }
-  /**
-   * Protobuf type {@code AggregateRequest}
-   */
-  public static final class AggregateRequest extends
-      com.google.protobuf.GeneratedMessage implements
-      // @@protoc_insertion_point(message_implements:AggregateRequest)
-      AggregateRequestOrBuilder {
-    // Use AggregateRequest.newBuilder() to construct.
-    private AggregateRequest(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
-      super(builder);
-      this.unknownFields = builder.getUnknownFields();
-    }
-    private AggregateRequest(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
-
-    private static final AggregateRequest defaultInstance;
-    public static AggregateRequest getDefaultInstance() {
-      return defaultInstance;
-    }
-
-    public AggregateRequest getDefaultInstanceForType() {
-      return defaultInstance;
-    }
-
-    private final com.google.protobuf.UnknownFieldSet unknownFields;
-    @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
-        getUnknownFields() {
-      return this.unknownFields;
-    }
-    private AggregateRequest(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
-      initFields();
-      int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
-      try {
-        boolean done = false;
-        while (!done) {
-          int tag = input.readTag();
-          switch (tag) {
-            case 0:
-              done = true;
-              break;
-            default: {
-              if (!parseUnknownField(input, unknownFields,
-                                     extensionRegistry, tag)) {
-                done = true;
-              }
-              break;
-            }
-            case 10: {
-              AggregateProtos.EntityDefinition.Builder subBuilder = null;
-              if (((bitField0_ & 0x00000001) == 0x00000001)) {
-                subBuilder = entityDefinition_.toBuilder();
-              }
-              entityDefinition_ = input.readMessage(AggregateProtos.EntityDefinition.PARSER, extensionRegistry);
-              if (subBuilder != null) {
-                subBuilder.mergeFrom(entityDefinition_);
-                entityDefinition_ = subBuilder.buildPartial();
-              }
-              bitField0_ |= 0x00000001;
-              break;
-            }
-            case 18: {
-              org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Scan.Builder subBuilder = null;
-              if (((bitField0_ & 0x00000002) == 0x00000002)) {
-                subBuilder = scan_.toBuilder();
-              }
-              scan_ = input.readMessage(org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Scan.PARSER, extensionRegistry);
-              if (subBuilder != null) {
-                subBuilder.mergeFrom(scan_);
-                scan_ = subBuilder.buildPartial();
-              }
-              bitField0_ |= 0x00000002;
-              break;
-            }
-            case 26: {
-              com.google.protobuf.ByteString bs = input.readBytes();
-              if (!((mutable_bitField0_ & 0x00000004) == 0x00000004)) {
-                groupbyFields_ = new com.google.protobuf.LazyStringArrayList();
-                mutable_bitField0_ |= 0x00000004;
-              }
-              groupbyFields_.add(bs);
-              break;
-            }
-            case 34: {
-              if (!((mutable_bitField0_ & 0x00000008) == 0x00000008)) {
-                aggregateFuncTypes_ = new java.util.ArrayList<com.google.protobuf.ByteString>();
-                mutable_bitField0_ |= 0x00000008;
-              }
-              aggregateFuncTypes_.add(input.readBytes());
-              break;
-            }
-            case 42: {
-              com.google.protobuf.ByteString bs = input.readBytes();
-              if (!((mutable_bitField0_ & 0x00000010) == 0x00000010)) {
-                aggregatedFields_ = new com.google.protobuf.LazyStringArrayList();
-                mutable_bitField0_ |= 0x00000010;
-              }
-              aggregatedFields_.add(bs);
-              break;
-            }
-          }
-        }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
-        throw e.setUnfinishedMessage(this);
-      } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
-            e.getMessage()).setUnfinishedMessage(this);
-      } finally {
-        if (((mutable_bitField0_ & 0x00000004) == 0x00000004)) {
-          groupbyFields_ = groupbyFields_.getUnmodifiableView();
-        }
-        if (((mutable_bitField0_ & 0x00000008) == 0x00000008)) {
-          aggregateFuncTypes_ = java.util.Collections.unmodifiableList(aggregateFuncTypes_);
-        }
-        if (((mutable_bitField0_ & 0x00000010) == 0x00000010)) {
-          aggregatedFields_ = aggregatedFields_.getUnmodifiableView();
-        }
-        this.unknownFields = unknownFields.build();
-        makeExtensionsImmutable();
-      }
-    }
-    public static final com.google.protobuf.Descriptors.Descriptor
-        getDescriptor() {
-      return AggregateProtos.internal_static_AggregateRequest_descriptor;
-    }
-
-    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
-        internalGetFieldAccessorTable() {
-      return AggregateProtos.internal_static_AggregateRequest_fieldAccessorTable
-          .ensureFieldAccessorsInitialized(
-              AggregateProtos.AggregateRequest.class, AggregateProtos.AggregateRequest.Builder.class);
-    }
-
-    public static com.google.protobuf.Parser<AggregateRequest> PARSER =
-        new com.google.protobuf.AbstractParser<AggregateRequest>() {
-      public AggregateRequest parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
-        return new AggregateRequest(input, extensionRegistry);
-      }
-    };
-
-    @java.lang.Override
-    public com.google.protobuf.Parser<AggregateRequest> getParserForType() {
-      return PARSER;
-    }
-
-    private int bitField0_;
-    public static final int ENTITY_DEFINITION_FIELD_NUMBER = 1;
-    private AggregateProtos.EntityDefinition entityDefinition_;
-    /**
-     * <code>required .EntityDefinition entity_definition = 1;</code>
-     */
-    public boolean hasEntityDefinition() {
-      return ((bitField0_ & 0x00000001) == 0x00000001);
-    }
-    /**
-     * <code>required .EntityDefinition entity_definition = 1;</code>
-     */
-    public AggregateProtos.EntityDefinition getEntityDefinition() {
-      return entityDefinition_;
-    }
-    /**
-     * <code>required .EntityDefinition entity_definition = 1;</code>
-     */
-    public AggregateProtos.EntityDefinitionOrBuilder getEntityDefinitionOrBuilder() {
-      return entityDefinition_;
-    }
-
-    public static final int SCAN_FIELD_NUMBER = 2;
-    private org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Scan scan_;
-    /**
-     * <code>required .Scan scan = 2;</code>
-     */
-    public boolean hasScan() {
-      return ((bitField0_ & 0x00000002) == 0x00000002);
-    }
-    /**
-     * <code>required .Scan scan = 2;</code>
-     */
-    public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Scan getScan() {
-      return scan_;
-    }
-    /**
-     * <code>required .Scan scan = 2;</code>
-     */
-    public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanOrBuilder getScanOrBuilder() {
-      return scan_;
-    }
-
-    public static final int GROUPBY_FIELDS_FIELD_NUMBER = 3;
-    private com.google.protobuf.LazyStringList groupbyFields_;
-    /**
-     * <code>repeated string groupby_fields = 3;</code>
-     */
-    public com.google.protobuf.ProtocolStringList
-        getGroupbyFieldsList() {
-      return groupbyFields_;
-    }
-    /**
-     * <code>repeated string groupby_fields = 3;</code>
-     */
-    public int getGroupbyFieldsCount() {
-      return groupbyFields_.size();
-    }
-    /**
-     * <code>repeated string groupby_fields = 3;</code>
-     */
-    public java.lang.String getGroupbyFields(int index) {
-      return groupbyFields_.get(index);
-    }
-    /**
-     * <code>repeated string groupby_fields = 3;</code>
-     */
-    public com.google.protobuf.ByteString
-        getGroupbyFieldsBytes(int index) {
-      return groupbyFields_.getByteString(index);
-    }
-
-    public static final int AGGREGATE_FUNC_TYPES_FIELD_NUMBER = 4;
-    private java.util.List<com.google.protobuf.ByteString> aggregateFuncTypes_;
-    /**
-     * <code>repeated bytes aggregate_func_types = 4;</code>
-     */
-    public java.util.List<com.google.protobuf.ByteString>
-        getAggregateFuncTypesList() {
-      return aggregateFuncTypes_;
-    }
-    /**
-     * <code>repeated bytes aggregate_func_types = 4;</code>
-     */
-    public int getAggregateFuncTypesCount() {
-      return aggregateFuncTypes_.size();
-    }
-    /**
-     * <code>repeated bytes aggregate_func_types = 4;</code>
-     */
-    public com.google.protobuf.ByteString getAggregateFuncTypes(int index) {
-      return aggregateFuncTypes_.get(index);
-    }
-
-    public static final int AGGREGATED_FIELDS_FIELD_NUMBER = 5;
-    private com.google.protobuf.LazyStringList aggregatedFields_;
-    /**
-     * <code>repeated string aggregated_fields = 5;</code>
-     */
-    public com.google.protobuf.ProtocolStringList
-        getAggregatedFieldsList() {
-      return aggregatedFields_;
-    }
-    /**
-     * <code>repeated string aggregated_fields = 5;</code>
-     */
-    public int getAggregatedFieldsCount() {
-      return aggregatedFields_.size();
-    }
-    /**
-     * <code>repeated string aggregated_fields = 5;</code>
-     */
-    public java.lang.String getAggregatedFields(int index) {
-      return aggregatedFields_.get(index);
-    }
-    /**
-     * <code>repeated string aggregated_fields = 5;</code>
-     */
-    public com.google.protobuf.ByteString
-        getAggregatedFieldsBytes(int index) {
-      return aggregatedFields_.getByteString(index);
-    }
-
-    private void initFields() {
-      entityDefinition_ = AggregateProtos.EntityDefinition.getDefaultInstance();
-      scan_ = org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Scan.getDefaultInstance();
-      groupbyFields_ = com.google.protobuf.LazyStringArrayList.EMPTY;
-      aggregateFuncTypes_ = java.util.Collections.emptyList();
-      aggregatedFields_ = com.google.protobuf.LazyStringArrayList.EMPTY;
-    }
-    private byte memoizedIsInitialized = -1;
-    public final boolean isInitialized() {
-      byte isInitialized = memoizedIsInitialized;
-      if (isInitialized == 1) return true;
-      if (isInitialized == 0) return false;
-
-      if (!hasEntityDefinition()) {
-        memoizedIsInitialized = 0;
-        return false;
-      }
-      if (!hasScan()) {
-        memoizedIsInitialized = 0;
-        return false;
-      }
-      if (!getEntityDefinition().isInitialized()) {
-        memoizedIsInitialized = 0;
-        return false;
-      }
-      if (!getScan().isInitialized()) {
-        memoizedIsInitialized = 0;
-        return false;
-      }
-      memoizedIsInitialized = 1;
-      return true;
-    }
-
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
-                        throws java.io.IOException {
-      getSerializedSize();
-      if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        output.writeMessage(1, entityDefinition_);
-      }
-      if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        output.writeMessage(2, scan_);
-      }
-      for (int i = 0; i < groupbyFields_.size(); i++) {
-        output.writeBytes(3, groupbyFields_.getByteString(i));
-      }
-      for (int i = 0; i < aggregateFuncTypes_.size(); i++) {
-        output.writeBytes(4, aggregateFuncTypes_.get(i));
-      }
-      for (int i = 0; i < aggregatedFields_.size(); i++) {
-        output.writeBytes(5, aggregatedFields_.getByteString(i));
-      }
-      getUnknownFields().writeTo(output);
-    }
-
-    private int memoizedSerializedSize = -1;
-    public int getSerializedSize() {
-      int size = memoizedSerializedSize;
-      if (size != -1) return size;
-
-      size = 0;
-      if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
-          .computeMessageSize(1, entityDefinition_);
-      }
-      if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        size += com.google.protobuf.CodedOutputStream
-          .computeMessageSize(2, scan_);
-      }
-      {
-        int dataSize = 0;
-        for (int i = 0; i < groupbyFields_.size(); i++) {
-          dataSize += com.google.protobuf.CodedOutputStream
-            .computeBytesSizeNoTag(groupbyFields_.getByteString(i));
-        }
-        size += dataSize;
-        size += 1 * getGroupbyFieldsList().size();
-      }
-      {
-        int dataSize = 0;
-        for (int i = 0; i < aggregateFuncTypes_.size(); i++) {
-          dataSize += com.google.protobuf.CodedOutputStream
-            .computeBytesSizeNoTag(aggregateFuncTypes_.get(i));
-        }
-        size += dataSize;
-        size += 1 * getAggregateFuncTypesList().size();
-      }
-      {
-        int dataSize = 0;
-        for (int i = 0; i < aggregatedFields_.size(); i++) {
-          dataSize += com.google.protobuf.CodedOutputStream
-            .computeBytesSizeNoTag(aggregatedFields_.getByteString(i));
-        }
-        size += dataSize;
-        size += 1 * getAggregatedFieldsList().size();
-      }
-      size += getUnknownFields().getSerializedSize();
-      memoizedSerializedSize = size;
-      return size;
-    }
-
-    private static final long serialVersionUID = 0L;
-    @java.lang.Override
-    protected java.lang.Object writeReplace()
-        throws java.io.ObjectStreamException {
-      return super.writeReplace();
-    }
-
-    @java.lang.Override
-    public boolean equals(final java.lang.Object obj) {
-      if (obj == this) {
-       return true;
-      }
-      if (!(obj instanceof AggregateProtos.AggregateRequest)) {
-        return super.equals(obj);
-      }
-      AggregateProtos.AggregateRequest other = (AggregateProtos.AggregateRequest) obj;
-
-      boolean result = true;
-      result = result && (hasEntityDefinition() == other.hasEntityDefinition());
-      if (hasEntityDefinition()) {
-        result = result && getEntityDefinition()
-            .equals(other.getEntityDefinition());
-      }
-      result = result && (hasScan() == other.hasScan());
-      if (hasScan()) {
-        result = result && getScan()
-            .equals(other.getScan());
-      }
-      result = result && getGroupbyFieldsList()
-          .equals(other.getGroupbyFieldsList());
-      result = result && getAggregateFuncTypesList()
-          .equals(other.getAggregateFuncTypesList());
-      result = result && getAggregatedFieldsList()
-          .equals(other.getAggregatedFieldsList());
-      result = result &&
-          getUnknownFields().equals(other.getUnknownFields());
-      return result;
-    }
-
-    @java.lang.Override
-    public int hashCode() {
-      if (memoizedHashCode != 0) {
-        return memoizedHashCode;
-      }
-      int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
-      if (hasEntityDefinition()) {
-        hash = (37 * hash) + ENTITY_DEFINITION_FIELD_NUMBER;
-        hash = (53 * hash) + getEntityDefinition().hashCode();
-      }
-      if (hasScan()) {
-        hash = (37 * hash) + SCAN_FIELD_NUMBER;
-        hash = (53 * hash) + getScan().hashCode();
-      }
-      if (getGroupbyFieldsCount() > 0) {
-        hash = (37 * hash) + GROUPBY_FIELDS_FIELD_NUMBER;
-        hash = (53 * hash) + getGroupbyFieldsList().hashCode();
-      }
-      if (getAggregateFuncTypesCount() > 0) {
-        hash = (37 * hash) + AGGREGATE_FUNC_TYPES_FIELD_NUMBER;
-        hash = (53 * hash) + getAggregateFuncTypesList().hashCode();
-      }
-      if (getAggregatedFieldsCount() > 0) {
-        hash = (37 * hash) + AGGREGATED_FIELDS_FIELD_NUMBER;
-        hash = (53 * hash) + getAggregatedFieldsList().hashCode();
-      }
-      hash = (29 * hash) + getUnknownFields().hashCode();
-      memoizedHashCode = hash;
-      return hash;
-    }
-
-    public static AggregateProtos.AggregateRequest parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
-      return PARSER.parseFrom(data);
-    }
-    public static AggregateProtos.AggregateRequest parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
-      return PARSER.parseFrom(data, extensionRegistry);
-    }
-    public static AggregateProtos.AggregateRequest parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
-      return PARSER.parseFrom(data);
-    }
-    public static AggregateProtos.AggregateRequest parseFrom(
-        byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
-      return PARSER.parseFrom(data, extensionRegistry);
-    }
-    public static AggregateProtos.AggregateRequest parseFrom(java.io.InputStream input)
-        throws java.io.IOException {
-      return PARSER.parseFrom(input);
-    }
-    public static AggregateProtos.AggregateRequest parseFrom(
-        java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws java.io.IOException {
-      return PARSER.parseFrom(input, extensionRegistry);
-    }
-    public static AggregateProtos.AggregateRequest parseDelimitedFrom(java.io.InputStream input)
-        throws java.io.IOException {
-      return PARSER.parseDelimitedFrom(input);
-    }
-    public static AggregateProtos.AggregateRequest parseDelimitedFrom(
-        java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws java.io.IOException {
-      return PARSER.parseDelimitedFrom(input, extensionRegistry);
-    }
-    public static AggregateProtos.AggregateRequest parseFrom(
-        com.google.protobuf.CodedInputStream input)
-        throws java.io.IOException {
-      return PARSER.parseFrom(input);
-    }
-    public static AggregateProtos.AggregateRequest parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws java.io.IOException {
-      return PARSER.parseFrom(input, extensionRegistry);
-    }
-
-    public static Builder newBuilder() { return Builder.create(); }
-    public Builder newBuilderForType() { return newBuilder(); }
-    public static Builder newBuilder(AggregateProtos.AggregateRequest prototype) {
-      return newBuilder().mergeFrom(prototype);
-    }
-    public Builder toBuilder() { return newBuilder(this); }
-
-    @java.lang.Override
-    protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessage.BuilderParent parent) {
-      Builder builder = new Builder(parent);
-      return builder;
-    }
-    /**
-     * Protobuf type {@code AggregateRequest}
-     */
-    public static final class Builder extends
-        com.google.protobuf.GeneratedMessage.Builder<Builder> implements
-        // @@protoc_insertion_point(builder_implements:AggregateRequest)
-        AggregateProtos.AggregateRequestOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
-          getDescriptor() {
-        return AggregateProtos.internal_static_AggregateRequest_descriptor;
-      }
-
-      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
-          internalGetFieldAccessorTable() {
-        return AggregateProtos.internal_static_AggregateRequest_fieldAccessorTable
-            .ensureFieldAccessorsInitialized(
-                AggregateProtos.AggregateRequest.class, AggregateProtos.AggregateRequest.Builder.class);
-      }
-
-      // Construct using eagle.storage.hbase.query.coprocessor.generated.AggregateProtos.AggregateRequest.newBuilder()
-      private Builder() {
-        maybeForceBuilderInitialization();
-      }
-
-      private Builder(
-          com.google.protobuf.GeneratedMessage.BuilderParent parent) {
-        super(parent);
-        maybeForceBuilderInitialization();
-      }
-      private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
-          getEntityDefinitionFieldBuilder();
-          getScanFieldBuilder();
-        }
-      }
-      private static Builder create() {
-        return new Builder();
-      }
-
-      public Builder clear() {
-        super.clear();
-        if (entityDefinitionBuilder_ == null) {
-          entityDefinition_ = AggregateProtos.EntityDefinition.getDefaultInstance();
-        } else {
-          entityDefinitionBuilder_.clear();
-        }
-        bitField0_ = (bitField0_ & ~0x00000001);
-        if (scanBuilder_ == null) {
-          scan_ = org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Scan.getDefaultInstance();
-        } else {
-          scanBuilder_.clear();
-        }
-        bitField0_ = (bitField0_ & ~0x00000002);
-        groupbyFields_ = com.google.protobuf.LazyStringArrayList.EMPTY;
-        bitField0_ = (bitField0_ & ~0x00000004);
-        aggregateFuncTypes_ = java.util.Collections.emptyList();
-        bitField0_ = (bitField0_ & ~0x00000008);
-        aggregatedFields_ = com.google.protobuf.LazyStringArrayList.EMPTY;
-        bitField0_ = (bitField0_ & ~0x00000010);
-        return this;
-      }
-
-      public Builder clone() {
-        return create().mergeFrom(buildPartial());
-      }
-
-      public com.google.protobuf.Descriptors.Descriptor
-          getDescriptorForType() {
-        return AggregateProtos.internal_static_AggregateRequest_descriptor;
-      }
-
-      public AggregateProtos.AggregateRequest getDefaultInstanceForType() {
-        return AggregateProtos.AggregateRequest.getDefaultInstance();
-      }
-
-      public AggregateProtos.AggregateRequest build() {
-        AggregateProtos.AggregateRequest result = buildPartial();
-        if (!result.isInitialized()) {
-          throw AbstractMessage.Builder.newUninitializedMessageException(result);
-        }
-        return result;
-      }
-
-      public AggregateProtos.AggregateRequest buildPartial() {
-        AggregateProtos.AggregateRequest result = new AggregateProtos.AggregateRequest(this);
-        int from_bitField0_ = bitField0_;
-        int to_bitField0_ = 0;
-        if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
-          to_bitField0_ |= 0x00000001;
-        }
-        if (entityDefinitionBuilder_ == null) {
-          result.entityDefinition_ = entityDefinition_;
-        } else {
-          result.entityDefinition_ = entityDefinitionBuilder_.build();
-        }
-        if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
-          to_bitField0_ |= 0x00000002;
-        }
-        if (scanBuilder_ == null) {
-          result.scan_ = scan_;
-        } else {
-          result.scan_ = scanBuilder_.build();
-        }
-        if (((bitField0_ & 0x00000004) == 0x00000004)) {
-          groupbyFields_ = groupbyFields_.getUnmodifiableView();
-          bitField0_ = (bitField0_ & ~0x00000004);
-        }
-        result.groupbyFields_ = groupbyFields_;
-        if (((bitField0_ & 0x00000008) == 0x00000008)) {
-          aggregateFuncTypes_ = java.util.Collections.unmodifiableList(aggregateFuncTypes_);
-          bitField0_ = (bitField0_ & ~0x00000008);
-        }
-        result.aggregateFuncTypes_ = aggregateFuncTypes_;
-        if (((bitField0_ & 0x00000010) == 0x00000010)) {
-          aggregatedFields_ = aggregatedFields_.getUnmodifiableView();
-          bitField0_ = (bitField0_ & ~0x00000010);
-        }
-        result.aggregatedFields_ = aggregatedFields_;
-        result.bitField0_ = to_bitField0_;
-        onBuilt();
-        return result;
-      }
-
-      public Builder mergeFrom(com.google.protobuf.Message other) {
-        if (other instanceof AggregateProtos.AggregateRequest) {
-          return mergeFrom((AggregateProtos.AggregateRequest)other);
-        } else {
-          super.mergeFrom(other);
-          return this;
-        }
-      }
-
-      public Builder mergeFrom(AggregateProtos.AggregateRequest other) {
-        if (other == AggregateProtos.AggregateRequest.getDefaultInstance()) return this;
-        if (other.hasEntityDefinition()) {
-          mergeEntityDefinition(other.getEntityDefinition());
-        }
-        if (other.hasScan()) {
-          mergeScan(other.getScan());
-        }
-        if (!other.groupbyFields_.isEmpty()) {
-          if (groupbyFields_.isEmpty()) {
-            groupbyFields_ = other.groupbyFields_;
-            bitField0_ = (bitField0_ & ~0x00000004);
-          } else {
-            ensureGroupbyFieldsIsMutable();
-            groupbyFields_.addAll(other.groupbyFields_);
-          }
-          onChanged();
-        }
-        if (!other.aggregateFuncTypes_.isEmpty()) {
-          if (aggregateFuncTypes_.isEmpty()) {
-            aggregateFuncTypes_ = other.aggregateFuncTypes_;
-            bitField0_ = (bitField0_ & ~0x00000008);
-          } else {
-            ensureAggregateFuncTypesIsMutable();
-            aggregateFuncTypes_.addAll(other.aggregateFuncTypes_);
-          }
-          onChanged();
-        }
-        if (!other.aggregatedFields_.isEmpty()) {
-          if (aggregatedFields_.isEmpty()) {
-            aggregatedFields_ = other.aggregatedFields_;
-            bitField0_ = (bitField0_ & ~0x00000010);
-          } else {
-            ensureAggregatedFieldsIsMutable();
-            aggregatedFields_.addAll(other.aggregatedFields_);
-          }
-          onChanged();
-        }
-        this.mergeUnknownFields(other.getUnknownFields());
-        return this;
-      }
-
-      public final boolean isInitialized() {
-        if (!hasEntityDefinition()) {
-          
-          return false;
-        }
-        if (!hasScan()) {
-          
-          return false;
-        }
-        if (!getEntityDefinition().isInitialized()) {
-          
-          return false;
-        }
-        if (!getScan().isInitialized()) {
-          
-          return false;
-        }
-        return true;
-      }
-
-      public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws java.io.IOException {
-        AggregateProtos.AggregateRequest parsedMessage = null;
-        try {
-          parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
-          parsedMessage = (AggregateProtos.AggregateRequest) e.getUnfinishedMessage();
-          throw e;
-        } finally {
-          if (parsedMessage != null) {
-            mergeFrom(parsedMessage);
-          }
-        }
-        return this;
-      }
-      private int bitField0_;
-
-      private AggregateProtos.EntityDefinition entityDefinition_ = AggregateProtos.EntityDefinition.getDefaultInstance();
-      private com.google.protobuf.SingleFieldBuilder<
-          AggregateProtos.EntityDefinition, AggregateProtos.EntityDefinition.Builder, AggregateProtos.EntityDefinitionOrBuilder> entityDefinitionBuilder_;
-      /**
-       * <code>required .EntityDefinition entity_definition = 1;</code>
-       */
-      public boolean hasEntityDefinition() {
-        return ((bitField0_ & 0x00000001) == 0x00000001);
-      }
-      /**
-       * <code>required .EntityDefinition entity_definition = 1;</code>
-       */
-      public AggregateProtos.EntityDefinition getEntityDefinition() {
-        if (entityDefinitionBuilder_ == null) {
-          return entityDefinition_;
-        } else {
-          return entityDefinitionBuilder_.getMessage();
-        }
-      }
-      /**
-       * <code>required .EntityDefinition entity_definition = 1;</code>
-       */
-      public Builder setEntityDefinition(AggregateProtos.EntityDefinition value) {
-        if (entityDefinitionBuilder_ == null) {
-          if (value == null) {
-            throw new NullPointerException();
-          }
-          entityDefinition_ = value;
-          onChanged();
-        } else {
-          entityDefinitionBuilder_.setMessage(value);
-        }
-        bitField0_ |= 0x00000001;
-        return this;
-      }
-      /**
-       * <code>required .EntityDefinition entity_definition = 1;</code>
-       */
-      public Builder setEntityDefinition(
-          AggregateProtos.EntityDefinition.Builder builderForValue) {
-        if (entityDefinitionBuilder_ == null) {
-          entityDefinition_ = builderForValue.build();
-          onChanged();
-        } else {
-          entityDefinitionBuilder_.setMessage(builderForValue.build());
-        }
-        bitField0_ |= 0x00000001;
-        return this;
-      }
-      /**
-       * <code>required .EntityDefinition entity_definition = 1;</code>
-       */
-      public Builder mergeEntityDefinition(AggregateProtos.EntityDefinition value) {
-        if (entityDefinitionBuilder_ == null) {
-          if (((bitField0_ & 0x00000001) == 0x00000001) &&
-              entityDefinition_ != AggregateProtos.EntityDefinition.getDefaultInstance()) {
-            entityDefinition_ =
-              AggregateProtos.EntityDefinition.newBuilder(entityDefinition_).mergeFrom(value).buildPartial();
-          } else {
-            entityDefinition_ = value;
-          }
-          onChanged();
-        } else {
-          entityDefinitionBuilder_.mergeFrom(value);
-        }
-        bitField0_ |= 0x00000001;
-        return this;
-      }
-      /**
-       * <code>required .EntityDefinition entity_definition = 1;</code>
-       */
-      public Builder clearEntityDefinition() {
-        if (entityDefinitionBuilder_ == null) {
-          entityDefinition_ = AggregateProtos.EntityDefinition.getDefaultInstance();
-          onChanged();
-        } else {
-          entityDefinitionBuilder_.clear();
-        }
-        bitField0_ = (bitField0_ & ~0x00000001);
-        return this;
-      }
-      /**
-       * <code>required .EntityDefinition entity_definition = 1;</code>
-       */
-      public AggregateProtos.EntityDefinition.Builder getEntityDefinitionBuilder() {
-        bitField0_ |= 0x00000001;
-        onChanged();
-        return getEntityDefinitionFieldBuilder().getBuilder();
-      }
-      /**
-       * <code>required .EntityDefinition entity_definition = 1;</code>
-       */
-      public AggregateProtos.EntityDefinitionOrBuilder getEntityDefinitionOrBuilder() {
-        if (entityDefinitionBuilder_ != null) {
-          return entityDefinitionBuilder_.getMessageOrBuilder();
-        } else {
-          return entityDefinition_;
-        }
-      }
-      /**
-       * <code>required .EntityDefinition entity_definition = 1;</code>
-       */
-      private com.google.protobuf.SingleFieldBuilder<
-          AggregateProtos.EntityDefinition, AggregateProtos.EntityDefinition.Builder, AggregateProtos.EntityDefinitionOrBuilder>
-          getEntityDefinitionFieldBuilder() {
-        if (entityDefinitionBuilder_ == null) {
-          entityDefinitionBuilder_ = new com.google.protobuf.SingleFieldBuilder<
-              AggregateProtos.EntityDefinition, AggregateProtos.EntityDefinition.Builder, AggregateProtos.EntityDefinitionOrBuilder>(
-                  getEntityDefinition(),
-                  getParentForChildren(),
-                  isClean());
-          entityDefinition_ = null;
-        }
-        return entityDefinitionBuilder_;
-      }
-
-      private org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Scan scan_ = org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Scan.getDefaultInstance();
-      private com.google.protobuf.SingleFieldBuilder<
-          org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Scan, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Scan.Builder, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanOrBuilder> scanBuilder_;
-      /**
-       * <code>required .Scan scan = 2;</code>
-       */
-      public boolean hasScan() {
-        return ((bitField0_ & 0x00000002) == 0x00000002);
-      }
-      /**
-       * <code>required .Scan scan = 2;</code>
-       */
-      public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Scan getScan() {
-        if (scanBuilder_ == null) {
-          return scan_;
-        } else {
-          return scanBuilder_.getMessage();
-        }
-      }
-      /**
-       * <code>required .Scan scan = 2;</code>
-       */
-      public Builder setScan(org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Scan value) {
-        if (scanBuilder_ == null) {
-          if (value == null) {
-            throw new NullPointerException();
-          }
-          scan_ = value;
-          onChanged();
-        } else {
-          scanBuilder_.setMessage(value);
-        }
-        bitField0_ |= 0x00000002;
-        return this;
-      }
-      /**
-       * <code>required .Scan scan = 2;</code>
-       */
-      public Builder setScan(
-          org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Scan.Builder builderForValue) {
-        if (scanBuilder_ == null) {
-          scan_ = builderForValue.build();
-          onChanged();
-        } else {
-          scanBuilder_.setMessage(builderForValue.build());
-        }
-        bitField0_ |= 0x00000002;
-        return this;
-      }
-      /**
-       * <code>required .Scan scan = 2;</code>
-       */
-      public Builder mergeScan(org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Scan value) {
-        if (scanBuilder_ == null) {
-          if (((bitField0_ & 0x00000002) == 0x00000002) &&
-              scan_ != org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Scan.getDefaultInstance()) {
-            scan_ =
-              org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Scan.newBuilder(scan_).mergeFrom(value).buildPartial();
-          } else {
-            scan_ = value;
-          }
-          onChanged();
-        } else {
-          scanBuilder_.mergeFrom(value);
-        }
-        bitField0_ |= 0x00000002;
-        return this;
-      }
-      /**
-       * <code>required .Scan scan = 2;</code>
-       */
-      public Builder clearScan() {
-        if (scanBuilder_ == null) {
-          scan_ = org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Scan.getDefaultInstance();
-          onChanged();
-        } else {
-          scanBuilder_.clear();
-        }
-        bitField0_ = (bitField0_ & ~0x00000002);
-        return this;
-      }
-      /**
-       * <code>required .Scan scan = 2;</code>
-       */
-      public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Scan.Builder getScanBuilder() {
-        bitField0_ |= 0x00000002;
-        onChanged();
-        return getScanFieldBuilder().getBuilder();
-      }
-      /**
-       * <code>required .Scan scan = 2;</code>
-       */
-      public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanOrBuilder getScanOrBuilder() {
-        if (scanBuilder_ != null) {
-          return scanBuilder_.getMessageOrBuilder();
-        } else {
-          return scan_;
-        }
-      }
-      /**
-       * <code>required .Scan scan = 2;</code>
-       */
-      private com.google.protobuf.SingleFieldBuilder<
-          org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Scan, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Scan.Builder, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanOrBuilder> 
-          getScanFieldBuilder() {
-        if (scanBuilder_ == null) {
-          scanBuilder_ = new com.google.protobuf.SingleFieldBuilder<
-              org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Scan, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Scan.Builder, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanOrBuilder>(
-                  getScan(),
-                  getParentForChildren(),
-                  isClean());
-          scan_ = null;
-        }
-        return scanBuilder_;
-      }
-
-      private com.google.protobuf.LazyStringList groupbyFields_ = com.google.protobuf.LazyStringArrayList.EMPTY;
-      private void ensureGroupbyFieldsIsMutable() {
-        if (!((bitField0_ & 0x00000004) == 0x00000004)) {
-          groupbyFields_ = new com.google.protobuf.LazyStringArrayList(groupbyFields_);
-          bitField0_ |= 0x00000004;
-         }
-      }
-      /**
-       * <code>repeated string groupby_fields = 3;</code>
-       */
-      public com.google.protobuf.ProtocolStringList
-          getGroupbyFieldsList() {
-        return groupbyFields_.getUnmodifiableView();
-      }
-      /**
-       * <code>repeated string groupby_fields = 3;</code>
-       */
-      public int getGroupbyFieldsCount() {
-        return groupbyFields_.size();
-      }
-      /**
-       * <code>repeated string groupby_fields = 3;</code>
-       */
-      public java.lang.String getGroupbyFields(int index) {
-        return groupbyFields_.get(index);
-      }
-      /**
-       * <code>repeated string groupby_fields = 3;</code>
-       */
-      public com.google.protobuf.ByteString
-          getGroupbyFieldsBytes(int index) {
-        return groupbyFields_.getByteString(index);
-      }
-      /**
-       * <code>repeated string groupby_fields = 3;</code>
-       */
-      public Builder setGroupbyFields(
-          int index, java.lang.String value) {
-        if (value == null) {
-    throw new NullPointerException();
-  }
-  ensureGroupbyFieldsIsMutable();
-        groupbyFields_.set(index, value);
-        onChanged();
-        return this;
-      }
-      /**
-       * <code>repeated string groupby_fields = 3;</code>
-       */
-      public Builder addGroupbyFields(
-          java.lang.String value) {
-        if (value == null) {
-    throw new NullPointerException();
-  }
-  ensureGroupbyFieldsIsMutable();
-        groupbyFields_.add(value);
-        onChanged();
-        return this;
-      }
-      /**
-       * <code>repeated string groupby_fields = 3;</code>
-       */
-      public Builder addAllGroupbyFields(
-          java.lang.Iterable<java.lang.String> values) {
-        ensureGroupbyFieldsIsMutable();
-        com.google.protobuf.AbstractMessageLite.Builder.addAll(
-            values, groupbyFields_);
-        onChanged();
-        return this;
-      }
-      /**
-       * <code>repeated string groupby_fields = 3;</code>
-       */
-      public Builder clearGroupbyFields() {
-        groupbyFields_ = com.google.protobuf.LazyStringArrayList.EMPTY;
-        bitField0_ = (bitField0_ & ~0x00000004);
-        onChanged();
-        return this;
-      }
-      /**
-       * <code>repeated string groupby_fields = 3;</code>
-       */
-      public Builder addGroupbyFieldsBytes(
-          com.google.protobuf.ByteString value) {
-        if (value == null) {
-    throw new NullPointerException();
-  }
-  ensureGroupbyFieldsIsMutable();
-        groupbyFields_.add(value);
-        onChanged();
-        return this;
-      }
-
-      private java.util.List<com.google.protobuf.ByteString> aggregateFuncTypes_ = java.util.Collections.emptyList();
-      private void ensureAggregateFuncTypesIsMutable() {
-        if (!((bitField0_ & 0x00000008) == 0x00000008)) {
-          aggregateFuncTypes_ = new java.util.ArrayList<com.google.protobuf.ByteString>(aggregateFuncTypes_);
-          bitField0_ |= 0x00000008;
-         }
-      }
-      /**
-       * <code>repeated bytes aggregate_func_types = 4;</code>
-       */
-      public java.util.List<com.google.protobuf.ByteString>
-          getAggregateFuncTypesList() {
-        return java.util.Collections.unmodifiableList(aggregateFuncTypes_);
-      }
-      /**
-       * <code>repeated bytes aggregate_func_types = 4;</code>
-       */
-      public int getAggregateFuncTypesCount() {
-        return aggregateFuncTypes_.size();
-      }
-      /**
-       * <code>repeated bytes aggregate_func_types = 4;</code>
-       */
-      public com.google.protobuf.ByteString getAggregateFuncTypes(int index) {
-        return aggregateFuncTypes_.get(index);
-      }
-      /**
-       * <code>repeated bytes aggregate_func_types = 4;</code>
-       */
-      public Builder setAggregateFuncTypes(
-          int index, com.google.protobuf.ByteString value) {
-        if (value == null) {
-    throw new NullPointerException();
-  }
-  ensureAggregateFuncTypesIsMutable();
-        aggregateFuncTypes_.set(index, value);
-        onChanged();
-        return this;
-      }
-      /**
-       * <code>repeated bytes aggregate_func_types = 4;</code>
-       */
-      public Builder addAggregateFuncTypes(com.google.protobuf.ByteString value) {
-        if (value == null) {
-    throw new NullPointerException();
-  }
-  ensureAggregateFuncTypesIsMutable();
-        aggregateFuncTypes_.add(value);
-        onChanged();
-        return this;
-      }
-      /**
-       * <code>repeated bytes aggregate_func_types = 4;</code>
-       */
-      public Builder addAllAggregateFuncTypes(
-          java.lang.Iterable<? extends com.google.protobuf.ByteString> values) {
-        ensureAggregateFuncTypesIsMutable();
-        com.google.protobuf.AbstractMessageLite.Builder.addAll(
-            values, aggregateFuncTypes_);
-        onChanged();
-        return this;
-      }
-      /**
-       * <code>repeated bytes aggregate_func_types = 4;</code>
-       */
-      public Builder clearAggregateFuncTypes() {
-        aggregateFuncTypes_ = java.util.Collections.emptyList();
-        bitField0_ = (bitField0_ & ~0x00000008);
-        onChanged();
-        return this;
-      }
-
-      private com.google.protobuf.LazyStringList aggregatedFields_ = com.google.protobuf.LazyStringArrayList.EMPTY;
-      private void ensureAggregatedFieldsIsMutable() {
-        if (!((bitField0_ & 0x00000010) == 0x00000010)) {
-          aggregatedFields_ = new com.google.protobuf.LazyStringArrayList(aggregatedFields_);
-          bitField0_ |= 0x00000010;
-         }
-      }
-      /**
-       * <code>repeated string aggregated_fields = 5;</code>
-       */
-      public com.google.protobuf.ProtocolStringList
-          getAggregatedFieldsList() {
-        return aggregatedFields_.getUnmodifiableView();
-      }
-      /**
-       * <code>repeated string aggregated_fields = 5;</code>
-       */
-      public int getAggregatedFieldsCount() {
-        return aggregatedFields_.size();
-      }
-      /**
-       * <code>repeated string aggregated_fields = 5;</code>
-       */
-      public java.lang.String getAggregatedFields(int index) {
-        return aggregatedFields_.get(index);
-      }
-      /**
-       * <code>repeated string aggregated_fields = 5;</code>
-       */
-      public com.google.protobuf.ByteString
-          getAggregatedFieldsBytes(int index) {
-        return aggregatedFields_.getByteString(index);
-      }
-      /**
-       * <code>repeated string aggregated_fields = 5;</code>
-       */
-      public Builder setAggregatedFields(
-          int index, java.lang.String value) {
-        if (value == null) {
-    throw new NullPointerException();
-  }
-  ensureAggregatedFieldsIsMutable();
-        aggregatedFields_.set(index, value);
-        onChanged();
-        return this;
-      }
-      /**
-       * <code>repeated string aggregated_fields = 5;</code>
-       */
-      public Builder addAggregatedFields(
-          java.lang.String value) {
-        if (value == null) {
-    throw new NullPointerException();
-  }
-  ensureAggregatedFieldsIsMutable();
-        aggregatedFields_.add(value);
-        onChanged();
-        return this;
-      }
-      /**
-       * <code>repeated string aggregated_fields = 5;</code>
-       */
-      public Builder addAllAggregatedFields(
-          java.lang.Iterable<java.lang.String> values) {
-        ensureAggregatedFieldsIsMutable();
-        com.google.protobuf.AbstractMessageLite.Builder.addAll(
-            values, aggregatedFields_);
-        onChanged();
-        return this;
-      }
-      /**
-       * <code>repeated string aggregated_fields = 5;</code>
-       */
-      public Builder clearAggregatedFields() {
-        aggregatedFields_ = com.google.protobuf.LazyStringArrayList.EMPTY;
-        bitField0_ = (bitField0_ & ~0x00000010);
-        onChanged();
-        return this;
-      }
-      /**
-       * <code>repeated string aggregated_fields = 5;</code>
-       */
-      public Builder addAggregatedFieldsBytes(
-          com.google.protobuf.ByteString value) {
-        if (value == null) {
-    throw new NullPointerException();
-  }
-  ensureAggregatedFieldsIsMutable();
-        aggregatedFields_.add(value);
-        onChanged();
-        return this;
-      }
-
-      // @@protoc_insertion_point(builder_scope:AggregateRequest)
-    }
-
-    static {
-      defaultInstance = new AggregateRequest(true);
-      defaultInstance.initFields();
-    }
-
-    // @@protoc_insertion_point(class_scope:AggregateRequest)
-  }
-
-  public interface TimeSeriesAggregateRequestOrBuilder extends
-      // @@protoc_insertion_point(interface_extends:TimeSeriesAggregateRequest)
-      com.google.protobuf.MessageOrBuilder {
-
-    /**
-     * <code>required .EntityDefinition entity_definition = 1;</code>
-     */
-    boolean hasEntityDefinition();
-    /**
-     * <code>required .EntityDefinition entity_definition = 1;</code>
-     */
-    AggregateProtos.EntityDefinition getEntityDefinition();
-    /**
-     * <code>required .EntityDefinition entity_definition = 1;</code>
-     */
-    AggregateProtos.EntityDefinitionOrBuilder getEntityDefinitionOrBuilder();
-
-    /**
-     * <code>required .Scan scan = 2;</code>
-     */
-    boolean hasScan();
-    /**
-     * <code>required .Scan scan = 2;</code>
-     */
-    org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Scan getScan();
-    /**
-     * <code>required .Scan scan = 2;</code>
-     */
-    org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanOrBuilder getScanOrBuilder();
-
-    /**
-     * <code>repeated string groupby_fields = 3;</code>
-     */
-    com.google.protobuf.ProtocolStringList
-        getGroupbyFieldsList();
-    /**
-     * <code>repeated string groupby_fields = 3;</code>
-     */
-    int getGroupbyFieldsCount();
-    /**
-     * <code>repeated string groupby_fields = 3;</code>
-     */
-    java.lang.String getGroupbyFields(int index);
-    /**
-     * <code>repeated string groupby_fields = 3;</code>
-     */
-    com.google.protobuf.ByteString
-        getGroupbyFieldsBytes(int index);
-
-    /**
-     * <code>repeated bytes aggregate_func_types = 4;</code>
-     */
-    java.util.List<com.google.protobuf.ByteString> getAggregateFuncTypesList();
-    /**
-     * <code>repeated bytes aggregate_func_types = 4;</code>
-     */
-    int getAggregateFuncTypesCount();
-    /**
-     * <code>repeated bytes aggregate_func_types = 4;</code>
-     */
-    com.google.protobuf.ByteString getAggregateFuncTypes(int index);
-
-    /**
-     * <code>repeated string aggregated_fields = 5;</code>
-     */
-    com.google.protobuf.ProtocolStringList
-        getAggregatedFieldsList();
-    /**
-     * <code>repeated string aggregated_fields = 5;</code>
-     */
-    int getAggregatedFieldsCount();
-    /**
-     * <code>repeated string aggregated_fields = 5;</code>
-     */
-    java.lang.String getAggregatedFields(int index);
-    /**
-     * <code>repeated string aggregated_fields = 5;</code>
-     */
-    com.google.protobuf.ByteString
-        getAggregatedFieldsBytes(int index);
-
-    /**
-     * <code>required int64 start_time = 6;</code>
-     */
-    boolean hasStartTime();
-    /**
-     * <code>required int64 start_time = 6;</code>
-     */
-    long getStartTime();
-
-    /**
-     * <code>required int64 end_time = 7;</code>
-     */
-    boolean hasEndTime();
-    /**
-     * <code>required int64 end_time = 7;</code>
-     */
-    long getEndTime();
-
-    /**
-     * <code>required int64 interval_min = 8;</code>
-     */
-    boolean hasIntervalMin();
-    /**
-     * <code>required int64 interval_min = 8;</code>
-     */
-    long getIntervalMin();
-  }
-  /**
-   * Protobuf type {@code TimeSeriesAggregateRequest}
-   */
-  public static final class TimeSeriesAggregateRequest extends
-      com.google.protobuf.GeneratedMessage implements
-      // @@protoc_insertion_point(message_implements:TimeSeriesAggregateRequest)
-      TimeSeriesAggregateRequestOrBuilder {
-    // Use TimeSeriesAggregateRequest.newBuilder() to construct.
-    private TimeSeriesAggregateRequest(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
-      super(builder);
-      this.unknownFields = builder.getUnknownFields();
-    }
-    private TimeSeriesAggregateRequest(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
-
-    private static final TimeSeriesAggregateRequest defaultInstance;
-    public static TimeSeriesAggregateRequest getDefaultInstance() {
-      return defaultInstance;
-    }
-
-    public TimeSeriesAggregateRequest getDefaultInstanceForType() {
-      return defaultInstance;
-    }
-
-    private final com.google.protobuf.UnknownFieldSet unknownFields;
-    @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
-        getUnknownFields() {
-      return this.unknownFields;
-    }
-    private TimeSeriesAggregateRequest(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
-      initFields();
-      int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
-      try {
-        boolean done = false;
-        while (!done) {
-          int tag = input.readTag();
-          switch (tag) {
-            case 0:
-              done = true;
-              break;
-            default: {
-              if (!parseUnknownField(input, unknownFields,
-                                     extensionRegistry, tag)) {
-                done = true;
-              }
-              break;
-            }
-            case 10: {
-              AggregateProtos.EntityDefinition.Builder subBuilder = null;
-              if (((bitField0_ & 0x00000001) == 0x00000001)) {
-                subBuilder = entityDefinition_.toBuilder();
-              }
-              entityDefinition_ = input.readMessage(AggregateProtos.EntityDefinition.PARSER, extensionRegistry);
-              if (subBuilder != null) {
-                subBuilder.mergeFrom(entityDefinition_);
-                entityDefinition_ = subBuilder.buildPartial();
-              }
-              bitField0_ |= 0x00000001;
-              break;
-            }
-            case 18: {
-              org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Scan.Builder subBuilder = null;
-              if (((bitField0_ & 0x00000002) == 0x00000002)) {
-                subBuilder = scan_.toBuilder();
-              }
-              scan_ = input.readMessage(org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Scan.PARSER, extensionRegistry);
-              if (subBuilder != null) {
-                subBuilder.mergeFrom(scan_);
-                scan_ = subBuilder.buildPartial();
-              }
-              bitField0_ |= 0x00000002;
-              break;
-            }
-            case 26: {
-              com.google.protobuf.ByteString bs = input.readBytes();
-              if (!((mutable_bitField0_ & 0x00000004) == 0x00000004)) {
-                groupbyFields_ = new com.google.protobuf.LazyStringArrayList();
-                mutable_bitField0_ |= 0x00000004;
-              }
-              groupbyFields_.add(bs);
-              break;
-            }
-            case 34: {
-              if (!((mutable_bitField0_ & 0x00000008) == 0x00000008)) {
-                aggregateFuncTypes_ = new java.util.ArrayList<com.google.protobuf.ByteString>();
-                mutable_bitField0_ |= 0x00000008;
-              }
-              aggregateFuncTypes_.add(input.readBytes());
-              break;
-            }
- 

<TRUNCATED>