You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@eagle.apache.org by qi...@apache.org on 2016/08/19 09:50:26 UTC

[1/4] incubator-eagle git commit: [EAGLE-467] Job list apis for querying jobs regardless of the status [Forced Update!]

Repository: incubator-eagle
Updated Branches:
  refs/heads/develop a20656b5e -> d6ec142d3 (forced update)


http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/d6ec142d/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 9f993a6..2accad8 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
@@ -20,12 +20,13 @@ 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.entities.JobConfig;
-import org.apache.eagle.jpm.mr.running.entities.JobExecutionAPIEntity;
-import org.apache.eagle.jpm.mr.running.entities.TaskAttemptExecutionAPIEntity;
-import org.apache.eagle.jpm.mr.running.entities.TaskExecutionAPIEntity;
 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;
+import org.apache.eagle.jpm.mr.runningentity.TaskAttemptExecutionAPIEntity;
+import org.apache.eagle.jpm.mr.runningentity.TaskExecutionAPIEntity;
 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;
@@ -118,7 +119,7 @@ public class MRJobParser implements Runnable {
 
     private void finishMRJob(String mrJobId) {
         JobExecutionAPIEntity jobExecutionAPIEntity = mrJobEntityMap.get(mrJobId);
-        jobExecutionAPIEntity.setStatus(Constants.AppState.FINISHED.toString());
+        jobExecutionAPIEntity.setCurrentState(Constants.AppState.FINISHED.toString());
         mrJobConfigs.remove(mrJobId);
         if (mrJobConfigs.size() == 0) {
             this.parserStatus = ParserStatus.APP_FINISHED;
@@ -186,19 +187,20 @@ 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());
-            jobExecutionAPIEntity.getTags().put(MRJobTagName.JOD_DEF_ID.toString(), mrJob.getName());
+            jobExecutionAPIEntity.getTags().put(MRJobTagName.JOD_DEF_ID.toString(), jobDefId);
             jobExecutionAPIEntity.setTimestamp(app.getStartedTime());
             jobExecutionAPIEntity.setSubmissionTime(app.getStartedTime());
             jobExecutionAPIEntity.setStartTime(mrJob.getStartTime());
-            jobExecutionAPIEntity.setElapsedTime(mrJob.getElapsedTime());
-            jobExecutionAPIEntity.setStatus(mrJob.getState());
-            jobExecutionAPIEntity.setMapsTotal(mrJob.getMapsTotal());
+            jobExecutionAPIEntity.setDurationTime(mrJob.getElapsedTime());
+            jobExecutionAPIEntity.setCurrentState(mrJob.getState());
+            jobExecutionAPIEntity.setNumTotalMaps(mrJob.getMapsTotal());
             jobExecutionAPIEntity.setMapsCompleted(mrJob.getMapsCompleted());
-            jobExecutionAPIEntity.setReducesTotal(mrJob.getReducesTotal());
+            jobExecutionAPIEntity.setNumTotalReduces(mrJob.getReducesTotal());
             jobExecutionAPIEntity.setReducesCompleted(mrJob.getReducesCompleted());
             jobExecutionAPIEntity.setMapProgress(mrJob.getMapProgress());
             jobExecutionAPIEntity.setReduceProgress(mrJob.getReduceProgress());
@@ -220,7 +222,6 @@ public class MRJobParser implements Runnable {
             jobExecutionAPIEntity.setAllocatedMB(app.getAllocatedMB());
             jobExecutionAPIEntity.setAllocatedVCores(app.getAllocatedVCores());
             jobExecutionAPIEntity.setRunningContainers(app.getRunningContainers());
-            runningJobManager.update(app.getId(), id, jobExecutionAPIEntity);
         }
 
         return true;
@@ -434,10 +435,10 @@ public class MRJobParser implements Runnable {
 
             taskExecutionAPIEntity.setTimestamp(app.getStartedTime());
             taskExecutionAPIEntity.setStartTime(task.getStartTime());
-            taskExecutionAPIEntity.setFinishTime(task.getFinishTime());
-            taskExecutionAPIEntity.setElapsedTime(task.getElapsedTime());
+            taskExecutionAPIEntity.setEndTime(task.getFinishTime());
+            taskExecutionAPIEntity.setDuration(task.getElapsedTime());
             taskExecutionAPIEntity.setProgress(task.getProgress());
-            taskExecutionAPIEntity.setStatus(task.getState());
+            taskExecutionAPIEntity.setTaskStatus(task.getState());
             taskExecutionAPIEntity.setSuccessfulAttempt(task.getSuccessfulAttempt());
             taskExecutionAPIEntity.setStatusDesc(task.getStatus());
 
@@ -449,7 +450,8 @@ public class MRJobParser implements Runnable {
 
                 TaskAttemptExecutionAPIEntity taskAttemptExecutionAPIEntity = fetchTaskAttempt.apply(Pair.of(jobId, task.getId()));
                 if (taskAttemptExecutionAPIEntity != null) {
-                    taskExecutionAPIEntity.setHost(taskAttemptExecutionAPIEntity.getTags().get(MRJobTagName.HOSTNAME.toString()));
+                    taskExecutionAPIEntity.getTags().put(MRJobTagName.HOSTNAME.toString(), taskAttemptExecutionAPIEntity.getTags().get(MRJobTagName.HOSTNAME.toString()));
+                    //taskExecutionAPIEntity.setHost(taskAttemptExecutionAPIEntity.getTags().get(MRJobTagName.HOSTNAME.toString()));
                 }
             }
 
@@ -503,6 +505,7 @@ public class MRJobParser implements Runnable {
             mrJobConfigs.put(jobId, config);
 
             mrJobEntityCreationHandler.add(mrJobEntityMap.get(jobId));
+            runningJobManager.update(app.getId(), jobId, mrJobEntityMap.get(jobId));
         } catch (Exception e) {
             LOG.warn("fetch job conf from {} failed, {}", confURL, e);
             return false;
@@ -537,8 +540,8 @@ public class MRJobParser implements Runnable {
                     mrJobEntityMap.keySet()
                             .stream()
                             .filter(
-                                    jobId -> mrJobEntityMap.get(jobId).getStatus().equals(Constants.AppState.FINISHED.toString()) ||
-                                            mrJobEntityMap.get(jobId).getStatus().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));
                 }

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/d6ec142d/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 d7b84cc..76d2a19 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
@@ -18,7 +18,7 @@
 
 package org.apache.eagle.jpm.mr.running.parser.metrics;
 
-import org.apache.eagle.jpm.mr.running.entities.JobExecutionAPIEntity;
+import org.apache.eagle.jpm.mr.runningentity.JobExecutionAPIEntity;
 import org.apache.eagle.jpm.util.Constants;
 import org.apache.eagle.log.entity.GenericMetricEntity;
 
@@ -52,7 +52,7 @@ public class JobExecutionMetricsCreationListener extends AbstractMetricsCreation
 
     @Override
     public String buildMetricName(String field) {
-        return String.format(Constants.metricFormat, Constants.JOB_LEVEL, field);
+        return String.format(Constants.hadoopMetricFormat, Constants.JOB_LEVEL, field);
     }
 
 

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/d6ec142d/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 6d9525e..d0b0d57 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
@@ -18,7 +18,7 @@
 
 package org.apache.eagle.jpm.mr.running.parser.metrics;
 
-import org.apache.eagle.jpm.mr.running.entities.TaskExecutionAPIEntity;
+import org.apache.eagle.jpm.mr.runningentity.TaskExecutionAPIEntity;
 import org.apache.eagle.jpm.util.Constants;
 import org.apache.eagle.log.entity.GenericMetricEntity;
 
@@ -33,13 +33,13 @@ 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.getElapsedTime(), tags));
+            metrics.add(metricWrapper(currentTime, Constants.TASK_EXECUTION_TIME, entity.getDuration(), tags));
         }
         return metrics;
     }
 
     @Override
     public String buildMetricName(String field) {
-        return String.format(Constants.metricFormat, Constants.TASK_LEVEL, field);
+        return String.format(Constants.hadoopMetricFormat, Constants.TASK_LEVEL, field);
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/d6ec142d/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 352696b..978c3ec 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
@@ -20,7 +20,7 @@ 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.running.entities.JobExecutionAPIEntity;
+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;
 

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/d6ec142d/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 1703b25..a701d74 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
@@ -25,8 +25,8 @@ 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.entities.JobExecutionAPIEntity;
 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;

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/d6ec142d/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 92dfbe3..51307e1 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
@@ -24,9 +24,9 @@ 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.entities.JobExecutionAPIEntity;
 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;

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/d6ec142d/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 5c0d8f9..f15fc2d 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
@@ -63,6 +63,7 @@
   },
 
   "MRConfigureKeys" : {
+    "jobNameKey" : "eagle.job.name",
     "jobConfigKey" : [
     "mapreduce.map.output.compress",
     "mapreduce.map.output.compress.codec",

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/d6ec142d/eagle-jpm/eagle-jpm-service/pom.xml
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-service/pom.xml b/eagle-jpm/eagle-jpm-service/pom.xml
new file mode 100644
index 0000000..e218435
--- /dev/null
+++ b/eagle-jpm/eagle-jpm-service/pom.xml
@@ -0,0 +1,48 @@
+<?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-jpm-parent</artifactId>
+        <groupId>org.apache.eagle</groupId>
+        <version>0.5.0-incubating-SNAPSHOT</version>
+    </parent>
+    <modelVersion>4.0.0</modelVersion>
+
+    <artifactId>eagle-jpm-service</artifactId>
+
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.eagle</groupId>
+            <artifactId>eagle-jpm-util</artifactId>
+            <version>${project.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.eagle</groupId>
+            <artifactId>eagle-service-base</artifactId>
+            <version>${project.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.eagle</groupId>
+            <artifactId>eagle-jpm-entity</artifactId>
+            <version>${project.version}</version>
+        </dependency>
+    </dependencies>
+</project>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/d6ec142d/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
new file mode 100644
index 0000000..be90456
--- /dev/null
+++ b/eagle-jpm/eagle-jpm-service/src/main/java/org/apache/eagle/service/jpm/MRJobExecutionResource.java
@@ -0,0 +1,306 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.commons.lang.time.StopWatch;
+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.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+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";
+
+    private final static Logger LOG = LoggerFactory.getLogger(MRJobExecutionResource.class);
+
+    @GET
+    @Produces(MediaType.APPLICATION_JSON)
+    public GenericServiceAPIResponseEntity listJobs(@QueryParam("query") String query,
+                                                    @QueryParam("startTime") String startTime, @QueryParam("endTime") String endTime,
+                                                    @QueryParam("pageSize") int pageSize, @QueryParam("startRowkey") String startRowkey,
+                                                    @QueryParam("treeAgg") boolean treeAgg, @QueryParam("timeSeries") boolean timeSeries,
+                                                    @QueryParam("intervalmin") long intervalmin, @QueryParam("top") int top,
+                                                    @QueryParam("filterIfMissing") boolean filterIfMissing,
+                                                    @QueryParam("parallel") int parallel,
+                                                    @QueryParam("metricName") String metricName,
+                                                    @QueryParam("verbose") Boolean verbose) {
+        GenericServiceAPIResponseEntity response = new GenericServiceAPIResponseEntity();
+
+        List<TaggedLogAPIEntity> jobs = new ArrayList<>();
+        List<TaggedLogAPIEntity> finishedJobs = new ArrayList<>();
+        Set<String> jobIds = new HashSet<>();
+        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);
+        if (res.isSuccess() && res.getObj() != null) {
+            for (TaggedLogAPIEntity o : res.getObj()) {
+                finishedJobs.add(o);
+                jobIds.add(o.getTags().get(JOB_ID.toString()));
+            }
+            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);
+            if (res.isSuccess() && res.getObj() != null) {
+                for (TaggedLogAPIEntity o : res.getObj()) {
+                    if (! isDuplicate(jobIds, o)) {
+                        jobs.add(o);
+                    }
+                }
+                jobs.addAll(finishedJobs);
+            }
+        }
+        stopWatch.stop();
+        if (res.isSuccess()) {
+            response.setSuccess(true);
+        } else {
+            response.setSuccess(false);
+            response.setException(new Exception(res.getException()));
+        }
+        meta.put(TOTAL_RESULTS, jobs.size());
+        meta.put(ELAPSEDMS,stopWatch.getTime());
+        response.setObj(jobs);
+        response.setMeta(meta);
+        return response;
+
+    }
+
+    private boolean isDuplicate(Set<String> keys, TaggedLogAPIEntity o) {
+        if (keys.isEmpty()) {
+            return false;
+        }
+        return keys.contains(o.getTags().get(JOB_ID.toString()));
+    }
+
+    private String buildCondition(String jobId, String jobDefId, String site) {
+        String conditionFormat = "@site=\"%s\"" ;
+        String condition = null;
+        if (jobDefId != null) {
+            conditionFormat = conditionFormat + " AND @jobDefId=\"%s\"";
+            condition = String.format(conditionFormat, site, jobDefId);
+        }
+        if (jobId != null) {
+            conditionFormat = conditionFormat + " AND @jobId=\"%s\"";
+            condition = String.format(conditionFormat, site, jobId);
+        }
+        return condition;
+    }
+
+    @GET
+    @Path("search")
+    @Produces(MediaType.APPLICATION_JSON)
+    public GenericServiceAPIResponseEntity searchJobsById(@QueryParam("jobId") String jobId,
+                                                          @QueryParam("jobDefId") String jobDefId,
+                                                          @QueryParam("site") String site) {
+        GenericServiceAPIResponseEntity response = new GenericServiceAPIResponseEntity();
+        if ((jobId == null && jobDefId == null) || site == null) {
+            response.setException(new IllegalArgumentException("Error: (jobId == null && jobDefId == null) || site == null"));
+            response.setSuccess(false);
+            return response;
+        }
+
+        List<TaggedLogAPIEntity> jobs = new ArrayList<>();
+        Set<String> jobIds = new HashSet<>();
+        String condition = buildCondition(jobId, jobDefId, site);
+        int pageSize = Integer.MAX_VALUE;
+        if (condition == null) {
+            response.setException(new Exception("Search condition is empty"));
+            response.setSuccess(false);
+            return response;
+        }
+        LOG.debug("search condition=" + condition);
+
+        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);
+        if (res.isSuccess() && res.getObj() != null) {
+            for (TaggedLogAPIEntity o : res.getObj()) {
+                jobs.add(o);
+                jobIds.add(o.getTags().get(JOB_ID.toString()));
+            }
+        }
+        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);
+        if (res.isSuccess() && res.getObj() != null) {
+            for (TaggedLogAPIEntity o : res.getObj()) {
+                if (! isDuplicate(jobIds, o)) {
+                    jobs.add(o);
+                }
+            }
+        }
+        if (jobs.size() > 0) {
+            Collections.sort(jobs, new Comparator<TaggedLogAPIEntity>() {
+                @Override
+                public int compare(TaggedLogAPIEntity o1, TaggedLogAPIEntity o2) {
+                    return o1.getTimestamp() > o2.getTimestamp() ? 1 : (o1.getTimestamp() == o2.getTimestamp() ? 0 : -1);
+                }
+            });
+        }
+        stopWatch.stop();
+        if (res.isSuccess()) {
+            response.setSuccess(true);
+        } else {
+            response.setSuccess(false);
+            response.setException(new Exception(res.getException()));
+        }
+        meta.put(TOTAL_RESULTS, jobs.size());
+        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")
+    @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();
+        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<>();
+
+        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 =
+                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) {
+            initTaskCountList(runningTaskCount, finishedTaskCount, times, new HistoryTaskComparator());
+            for (org.apache.eagle.jpm.mr.historyentity.TaskExecutionAPIEntity o : history_res.getObj()) {
+                int index = getPosition(times, o.getDuration());
+                MRJobTaskGroupResponse.UnitTaskCount counter = finishedTaskCount.get(index);
+                counter.taskCount++;
+                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> running_res =
+                    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) {
+                initTaskCountList(runningTaskCount, finishedTaskCount, times, new RunningTaskComparator());
+                for (TaskExecutionAPIEntity o : running_res.getObj()) {
+                    int index = 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);
+                    }
+                }
+            }
+        }
+        if (top > 0)  {
+            getTopTasks(runningTaskCount, top);
+            response.runningTaskCount = runningTaskCount;
+            getTopTasks(finishedTaskCount, top);
+            response.finishedTaskCount = finishedTaskCount;
+        }
+        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);
+        }
+    }
+
+    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/d6ec142d/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
new file mode 100644
index 0000000..3be9b43
--- /dev/null
+++ b/eagle-jpm/eagle-jpm-service/src/main/java/org/apache/eagle/service/jpm/MRJobTaskGroupResponse.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
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT 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/d6ec142d/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
new file mode 100644
index 0000000..824556b
--- /dev/null
+++ b/eagle-jpm/eagle-jpm-service/src/test/java/org/apache/eagle/service/jpm/TestMRJobExecutionResource.java
@@ -0,0 +1,99 @@
+/*
+ *  Licensed to the Apache Software Foundation (ASF) under one
+ *  or more contributor license agreements.  See the NOTICE file
+ *  distributed with this work for additional information
+ *  regarding copyright ownership.  The ASF licenses this file
+ *  to you under the Apache License, Version 2.0 (the
+ *  "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT 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/d6ec142d/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 d11d3d5..2e5a657 100644
--- a/eagle-jpm/eagle-jpm-spark-history/pom.xml
+++ b/eagle-jpm/eagle-jpm-spark-history/pom.xml
@@ -17,95 +17,114 @@
   -->
 
 <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-jpm-parent</artifactId>
-    <version>0.5.0-incubating-SNAPSHOT</version>
-    <relativePath>../pom.xml</relativePath>
-  </parent>
-  <artifactId>eagle-jpm-spark-history</artifactId>
-  <name>eagle-jpm-spark-history</name>
-  <url>http://maven.apache.org</url>
-  <dependencies>
-	  <dependency>
-		  <groupId>org.apache.eagle</groupId>
-		  <artifactId>eagle-jpm-util</artifactId>
-		  <version>${project.version}</version>
-	  </dependency>
-	  <dependency>
-		  <groupId>org.apache.eagle</groupId>
-		  <artifactId>eagle-jpm-entity</artifactId>
-		  <version>${project.version}</version>
-	  </dependency>
+         xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance">
+    <modelVersion>4.0.0</modelVersion>
+    <parent>
+        <groupId>org.apache.eagle</groupId>
+        <artifactId>eagle-jpm-parent</artifactId>
+        <version>0.5.0-incubating-SNAPSHOT</version>
+        <relativePath>../pom.xml</relativePath>
+    </parent>
+    <artifactId>eagle-jpm-spark-history</artifactId>
+    <name>eagle-jpm-spark-history</name>
+    <url>http://maven.apache.org</url>
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.storm</groupId>
+            <artifactId>storm-core</artifactId>
+            <version>${storm.version}</version>
+            <exclusions>
+                <exclusion>
+                    <groupId>ch.qos.logback</groupId>
+                    <artifactId>logback-classic</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>log4j</groupId>
+                    <artifactId>log4j</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>org.slf4j</groupId>
+                    <artifactId>log4j-over-slf4j</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.eagle</groupId>
+            <artifactId>eagle-jpm-util</artifactId>
+            <version>${project.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.eagle</groupId>
+            <artifactId>eagle-jpm-entity</artifactId>
+            <version>${project.version}</version>
+        </dependency>
 
-      <dependency>
-          <groupId>jline</groupId>
-          <artifactId>jline</artifactId>
-          <version>2.12</version>
-      </dependency>
-      <dependency>
-          <groupId>org.apache.curator</groupId>
-          <artifactId>curator-recipes</artifactId>
-          <version>${curator.version}</version>
-      </dependency>
-      <dependency>
-          <groupId>org.apache.hadoop</groupId>
-          <artifactId>hadoop-common</artifactId>
-          <version>${hadoop.version}</version>
-      </dependency>
-      <dependency>
-          <groupId>org.apache.hadoop</groupId>
-          <artifactId>hadoop-annotations</artifactId>
-          <version>${hadoop.version}</version>
-      </dependency>
-      <dependency>
-          <groupId>org.apache.hadoop</groupId>
-          <artifactId>hadoop-hdfs</artifactId>
-          <version>${hadoop.version}</version>
-      </dependency>
-      <dependency>
-          <groupId>org.apache.hadoop</groupId>
-          <artifactId>hadoop-auth</artifactId>
-          <version>${hadoop.version}</version>
-      </dependency>
-      <dependency>
-          <groupId>org.apache.hadoop</groupId>
-          <artifactId>hadoop-mapreduce-client-app</artifactId>
-          <version>${hadoop.version}</version>
-      </dependency>
-      <dependency>
-          <groupId>org.apache.hadoop</groupId>
-          <artifactId>hadoop-mapreduce-client-core</artifactId>
-          <version>${hadoop.version}</version>
-      </dependency>
-  </dependencies>
-	<build>
-		<resources>
-			<resource>
-				<directory>src/main/resources</directory>
-			</resource>
-		</resources>
-		<plugins>
-			<plugin>
-				<artifactId>maven-assembly-plugin</artifactId>
-				<configuration>
-					<descriptor>src/assembly/eagle-jpm-spark-history-assembly.xml</descriptor>
-					<finalName>eagle-jpm-spark-history-${project.version}</finalName>
-				</configuration>
-				<executions>
-					<execution>
-						<phase>package</phase>
-						<goals>
-							<goal>single</goal>
-						</goals>
-						<configuration>
-							<tarLongFileMode>posix</tarLongFileMode>
-						</configuration>
-					</execution>
-				</executions>
-			</plugin>
-		</plugins>
-	</build>
+        <dependency>
+            <groupId>jline</groupId>
+            <artifactId>jline</artifactId>
+            <version>2.12</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.curator</groupId>
+            <artifactId>curator-recipes</artifactId>
+            <version>${curator.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.hadoop</groupId>
+            <artifactId>hadoop-common</artifactId>
+            <version>${hadoop.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.hadoop</groupId>
+            <artifactId>hadoop-annotations</artifactId>
+            <version>${hadoop.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.hadoop</groupId>
+            <artifactId>hadoop-hdfs</artifactId>
+            <version>${hadoop.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.hadoop</groupId>
+            <artifactId>hadoop-auth</artifactId>
+            <version>${hadoop.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.hadoop</groupId>
+            <artifactId>hadoop-mapreduce-client-app</artifactId>
+            <version>${hadoop.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.hadoop</groupId>
+            <artifactId>hadoop-mapreduce-client-core</artifactId>
+            <version>${hadoop.version}</version>
+        </dependency>
+    </dependencies>
+    <build>
+        <resources>
+            <resource>
+                <directory>src/main/resources</directory>
+            </resource>
+        </resources>
+        <plugins>
+            <plugin>
+                <artifactId>maven-assembly-plugin</artifactId>
+                <configuration>
+                    <descriptor>src/assembly/eagle-jpm-spark-history-assembly.xml</descriptor>
+                    <finalName>eagle-jpm-spark-history-${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/d6ec142d/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 3df73e7..6a8dd8b 100644
--- a/eagle-jpm/eagle-jpm-util/pom.xml
+++ b/eagle-jpm/eagle-jpm-util/pom.xml
@@ -36,25 +36,6 @@
       <version>1.1.1</version>
     </dependency>
     <dependency>
-      <groupId>org.apache.storm</groupId>
-      <artifactId>storm-core</artifactId>
-      <version>${storm.version}</version>
-      <exclusions>
-        <exclusion>
-          <groupId>ch.qos.logback</groupId>
-          <artifactId>logback-classic</artifactId>
-        </exclusion>
-        <exclusion>
-          <groupId>log4j</groupId>
-          <artifactId>log4j</artifactId>
-        </exclusion>
-        <exclusion>
-          <groupId>org.slf4j</groupId>
-          <artifactId>log4j-over-slf4j</artifactId>
-        </exclusion>
-      </exclusions>
-    </dependency>
-    <dependency>
       <groupId>org.apache.commons</groupId>
       <artifactId>commons-lang3</artifactId>
     </dependency>
@@ -63,7 +44,6 @@
       <artifactId>commons-codec</artifactId>
       <version>1.9</version>
     </dependency>
-
   </dependencies>
 
 </project>

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/d6ec142d/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 b819340..07850f9 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
@@ -88,6 +88,9 @@ public class Constants {
         COMPLETE_MR_JOB
     }
 
+    public static final String TASK_RUNNING = "RUNNING";
+    public static final String TASK_FINISHED = "FINISHED";
+
     //MR
     public static final String JPA_JOB_CONFIG_SERVICE_NAME = "JobConfigService";
     public static final String JPA_JOB_EVENT_SERVICE_NAME = "JobEventService";
@@ -154,7 +157,7 @@ public class Constants {
         TOTAL_LAUNCHED_MAPS
     }
 
-    public static final String metricFormat = "%s.%s";
+    public static final String hadoopMetricFormat = "hadoop.%s.%s";
     public static final String ALLOCATED_MB = "allocatedmb";
     public static final String ALLOCATED_VCORES = "allocatedvcores";
     public static final String RUNNING_CONTAINERS = "runningcontainers";
@@ -162,6 +165,4 @@ public class Constants {
     public static final String JOB_LEVEL = "job";
     public static final String TASK_LEVEL = "task";
 
-    public static final String JOB_DEFINITION_ID_KEY = "jobDefId";
-
 }

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/d6ec142d/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 7a613eb..12eb1b5 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
@@ -18,7 +18,6 @@
 
 package org.apache.eagle.jpm.util;
 
-import jline.internal.Log;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -83,7 +82,7 @@ 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.info("Cannot parse memory info " +  memory);
         return 0l;
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/d6ec142d/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 44336e2..2b49f9f 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
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 /**
- * 
+ *
  */
 package org.apache.eagle.jpm.util.resourceFetch;
 
@@ -42,37 +42,37 @@ 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) throws Exception {
+
+	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, Constants.CompressionType.GZIP);
+			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) {
@@ -85,65 +85,73 @@ public class RMResourceFetcher implements ResourceFetcher<AppInfo> {
 		}
 	}
 
-    private String getSparkRunningJobURL() {
+	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) 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, Constants.CompressionType.GZIP);
-            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) { } }
-        }
-    }
-
-	public List<AppInfo> getResource(Constants.ResourceType resoureType, Object... parameter) throws Exception{
-		switch(resoureType) {
+	}
+
+	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);
+				final String urlString = sparkCompleteJobServiceURLBuilder.build((String) parameter[0]);
+				return doFetchFinishApplicationsList(urlString, compressionType);
 			case RUNNING_SPARK_JOB:
-                return doFetchRunningApplicationsList(getSparkRunningJobURL());
-            case RUNNING_MR_JOB:
-                return doFetchRunningApplicationsList(getMRRunningJobURL());
-            case COMPLETE_MR_JOB:
-                return doFetchFinishApplicationsList(getMRFinishedJobURL((String)parameter[0]));
+				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);

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/d6ec142d/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 6518ca1..4052ed0 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
@@ -35,7 +35,7 @@ public class HAURLSelectorImpl implements HAURLSelector {
 	
 	private volatile boolean reselectInProgress;
 	private final Constants.CompressionType compressionType;
-	private static final long MAX_RETRY_TIME = 3;
+	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) {
@@ -86,7 +86,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(5 * 1000);
+							Thread.sleep(1 * 1000);
 						}
 						catch (InterruptedException ex) { /* Do Nothing */}
 					}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/d6ec142d/eagle-jpm/pom.xml
----------------------------------------------------------------------
diff --git a/eagle-jpm/pom.xml b/eagle-jpm/pom.xml
index c48cdeb..91ef154 100644
--- a/eagle-jpm/pom.xml
+++ b/eagle-jpm/pom.xml
@@ -40,6 +40,7 @@
         <module>eagle-jpm-util</module>
         <module>eagle-jpm-mr-running</module>
         <module>eagle-jpm-app</module>
+      <module>eagle-jpm-service</module>
   </modules>
 
     <dependencies>

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/d6ec142d/eagle-security/eagle-security-hive/pom.xml
----------------------------------------------------------------------
diff --git a/eagle-security/eagle-security-hive/pom.xml b/eagle-security/eagle-security-hive/pom.xml
index a2e5a32..5f8f3b0 100644
--- a/eagle-security/eagle-security-hive/pom.xml
+++ b/eagle-security/eagle-security-hive/pom.xml
@@ -17,82 +17,82 @@
   -->
 
 <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>
-    <artifactId>eagle-security-parent</artifactId>
-    <version>0.5.0-incubating-SNAPSHOT</version>
-  </parent>
-  <artifactId>eagle-security-hive</artifactId>
-  <name>eagle-security-hive</name>
-  <url>http://maven.apache.org</url>
-  <packaging>jar</packaging>
+	<modelVersion>4.0.0</modelVersion>
+	<parent>
+		<groupId>org.apache.eagle</groupId>
+		<artifactId>eagle-security-parent</artifactId>
+		<version>0.5.0-incubating-SNAPSHOT</version>
+	</parent>
+	<artifactId>eagle-security-hive</artifactId>
+	<name>eagle-security-hive</name>
+	<url>http://maven.apache.org</url>
+	<packaging>jar</packaging>
 
-  <dependencies>
-	   <dependency>
-	      <groupId>org.apache.curator</groupId>
-	  	  <artifactId>curator-framework</artifactId>
-	   </dependency>
-	  <dependency>
-		  <groupId>org.apache.eagle</groupId>
-		  <artifactId>eagle-jpm-util</artifactId>
-          <version>${project.version}</version>
-      </dependency>
-      <dependency>
-          <groupId>org.jsoup</groupId>
-          <artifactId>jsoup</artifactId>
-      </dependency>
-	   <dependency>
-	      <groupId>org.apache.curator</groupId>
-	  	  <artifactId>curator-recipes</artifactId>
-	   </dependency>
-	   <dependency>
-	      <groupId>org.apache.curator</groupId>
-	  	  <artifactId>curator-client</artifactId>
-	   </dependency>
-	  <dependency>
-	      <groupId>org.apache.eagle</groupId>
-	  	  <artifactId>eagle-embed-server</artifactId>
-          <version>${project.version}</version>
-          <scope>test</scope>
-	   </dependency>
-	   <dependency>
-	      <groupId>org.apache.eagle</groupId>
-	   	  <artifactId>eagle-embed-server</artifactId>
-           <version>${project.version}</version>
-	   	  <classifier>tests</classifier>
-	   	  <scope>test</scope>
-	   </dependency>
-      <dependency>
-          <groupId>org.apache.eagle</groupId>
-          <artifactId>eagle-embed-hbase</artifactId>
-          <version>${project.version}</version>
-          <classifier>tests</classifier>
-          <scope>test</scope>
-      </dependency>
-      <dependency>
-          <groupId>org.apache.eagle</groupId>
-          <artifactId>eagle-embed-hbase</artifactId>
-          <version>${project.version}</version>
-          <scope>test</scope>
-      </dependency>
-	   <dependency>
-	  	  <groupId>org.apache.eagle</groupId>
-	  	  <artifactId>eagle-security-common</artifactId>
-           <version>${project.version}</version>
-	   </dependency>
-	   <dependency>
-	   	<groupId>org.apache.hive</groupId>
-	   	<artifactId>hive-exec</artifactId>
-	   </dependency>
-	  <dependency>
-		  <groupId>org.scala-lang</groupId>
-		  <artifactId>scala-library</artifactId>
-	  </dependency>
-	  <dependency>
-		  <groupId>org.apache.eagle</groupId>
-		  <artifactId>eagle-app-base</artifactId>
-		  <version>${project.version}</version>
-	  </dependency>
-  </dependencies>
+	<dependencies>
+		<dependency>
+			<groupId>org.apache.curator</groupId>
+			<artifactId>curator-framework</artifactId>
+		</dependency>
+		<dependency>
+			<groupId>org.apache.eagle</groupId>
+			<artifactId>eagle-jpm-util</artifactId>
+			<version>${project.version}</version>
+		</dependency>
+		<dependency>
+			<groupId>org.jsoup</groupId>
+			<artifactId>jsoup</artifactId>
+		</dependency>
+		<dependency>
+			<groupId>org.apache.curator</groupId>
+			<artifactId>curator-recipes</artifactId>
+		</dependency>
+		<dependency>
+			<groupId>org.apache.curator</groupId>
+			<artifactId>curator-client</artifactId>
+		</dependency>
+		<dependency>
+			<groupId>org.apache.eagle</groupId>
+			<artifactId>eagle-embed-server</artifactId>
+			<version>${project.version}</version>
+			<scope>test</scope>
+		</dependency>
+		<dependency>
+			<groupId>org.apache.eagle</groupId>
+			<artifactId>eagle-embed-server</artifactId>
+			<version>${project.version}</version>
+			<classifier>tests</classifier>
+			<scope>test</scope>
+		</dependency>
+		<dependency>
+			<groupId>org.apache.eagle</groupId>
+			<artifactId>eagle-embed-hbase</artifactId>
+			<version>${project.version}</version>
+			<classifier>tests</classifier>
+			<scope>test</scope>
+		</dependency>
+		<dependency>
+			<groupId>org.apache.eagle</groupId>
+			<artifactId>eagle-embed-hbase</artifactId>
+			<version>${project.version}</version>
+			<scope>test</scope>
+		</dependency>
+		<dependency>
+			<groupId>org.apache.eagle</groupId>
+			<artifactId>eagle-security-common</artifactId>
+			<version>${project.version}</version>
+		</dependency>
+		<dependency>
+			<groupId>org.apache.hive</groupId>
+			<artifactId>hive-exec</artifactId>
+		</dependency>
+		<dependency>
+			<groupId>org.scala-lang</groupId>
+			<artifactId>scala-library</artifactId>
+		</dependency>
+		<dependency>
+			<groupId>org.apache.eagle</groupId>
+			<artifactId>eagle-app-base</artifactId>
+			<version>${project.version}</version>
+		</dependency>
+	</dependencies>
 </project>


[3/4] incubator-eagle git commit: [EAGLE-467] Job list apis for querying jobs regardless of the status

Posted by qi...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/d6ec142d/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
new file mode 100644
index 0000000..1445a24
--- /dev/null
+++ b/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/mr/historyentity/TaskFailureCountAPIEntity.java
@@ -0,0 +1,67 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.entity.meta.*;
+import org.codehaus.jackson.map.annotate.JsonSerialize;
+
+@JsonSerialize(include=JsonSerialize.Inclusion.NON_NULL)
+@Table("eaglejpa_anomaly")
+@ColumnFamily("f")
+@Prefix("taskfailurecount")
+@Service(Constants.JPA_TASK_FAILURE_COUNT_SERVICE_NAME)
+@TimeSeries(true)
+@Partition({"site"})
+public class TaskFailureCountAPIEntity extends JobBaseAPIEntity {
+    @Column("a")
+    private int failureCount;
+    @Column("b")
+    private String error;
+    @Column("c")
+    private String taskStatus;
+
+
+    public String getTaskStatus() {
+        return taskStatus;
+    }
+
+    public void setTaskStatus(String taskStatus) {
+        this.taskStatus = taskStatus;
+        _pcs.firePropertyChange("taskStatus", null, null);
+    }
+
+    public String getError() {
+        return error;
+    }
+
+    public void setError(String error) {
+        this.error = error;
+        _pcs.firePropertyChange("error", null, null);
+    }
+
+    public int getFailureCount() {
+        return failureCount;
+    }
+
+    public void setFailureCount(int failureCount) {
+        this.failureCount = failureCount;
+        _pcs.firePropertyChange("failureCount", null, null);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/d6ec142d/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/mr/runningentity/JPMEntityRepository.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/mr/runningentity/JPMEntityRepository.java b/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/mr/runningentity/JPMEntityRepository.java
new file mode 100644
index 0000000..8af853a
--- /dev/null
+++ b/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/mr/runningentity/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.mr.runningentity;
+
+import org.apache.eagle.jpm.util.jobcounter.JobCounters;
+import org.apache.eagle.jpm.util.jobcounter.JobCountersSerDeser;
+import org.apache.eagle.log.entity.repo.EntityRepository;
+
+public class JPMEntityRepository extends EntityRepository {
+    public JPMEntityRepository() {
+        entitySet.add(JobExecutionAPIEntity.class);
+        entitySet.add(TaskExecutionAPIEntity.class);
+        entitySet.add(TaskAttemptExecutionAPIEntity.class);
+        serDeserMap.put(JobConfig.class, new JobConfigSerDeser());
+        serDeserMap.put(JobCounters.class, new JobCountersSerDeser());
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/d6ec142d/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/mr/runningentity/JobConfig.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/mr/runningentity/JobConfig.java b/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/mr/runningentity/JobConfig.java
new file mode 100644
index 0000000..6d8145f
--- /dev/null
+++ b/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/mr/runningentity/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.mr.runningentity;
+
+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/d6ec142d/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/mr/runningentity/JobConfigSerDeser.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/mr/runningentity/JobConfigSerDeser.java b/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/mr/runningentity/JobConfigSerDeser.java
new file mode 100644
index 0000000..409e84a
--- /dev/null
+++ b/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/mr/runningentity/JobConfigSerDeser.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.mr.runningentity;
+
+import org.apache.eagle.log.entity.meta.EntitySerDeser;
+import org.apache.eagle.log.entity.meta.MapSerDeser;
+
+import java.util.Map;
+
+public class JobConfigSerDeser implements EntitySerDeser<JobConfig> {
+    private static final MapSerDeser INSTANCE = new MapSerDeser();
+
+    @Override
+    public JobConfig deserialize(byte[] bytes) {
+        Map map = INSTANCE.deserialize(bytes);
+        JobConfig config = new JobConfig();
+        config.putAll(map);
+        return config;
+    }
+
+    @Override
+    public byte[] serialize(JobConfig jobConfig) {
+        return INSTANCE.serialize(jobConfig);
+    }
+
+    @Override
+    public Class<JobConfig> type() {
+        return JobConfig.class;
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/d6ec142d/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
new file mode 100644
index 0000000..653f1c9
--- /dev/null
+++ b/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/mr/runningentity/JobExecutionAPIEntity.java
@@ -0,0 +1,437 @@
+/*
+ *  Licensed to the Apache Software Foundation (ASF) under one
+ *  or more contributor license agreements.  See the NOTICE file
+ *  distributed with this work for additional information
+ *  regarding copyright ownership.  The ASF licenses this file
+ *  to you under the Apache License, Version 2.0 (the
+ *  "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT 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.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.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("eagleMRRunningJobs")
+@ColumnFamily("f")
+@Prefix("jobs")
+@Service(Constants.JPA_RUNNING_JOB_EXECUTION_SERVICE_NAME)
+@TimeSeries(true)
+@Partition({"site"})
+@Indexes({
+        @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")
+    private long startTime;
+    @Column("b")
+    private long endTime;
+    @Column("c")
+    private long durationTime;
+    @Column("d")
+    private String currentState;
+    @Column("e")
+    private int numTotalMaps;
+    @Column("f")
+    private int mapsCompleted;
+    @Column("g")
+    private int numTotalReduces;
+    @Column("h")
+    private int reducesCompleted;
+    @Column("i")
+    private double mapProgress;
+    @Column("j")
+    private double reduceProgress;
+    @Column("k")
+    private int mapsPending;
+    @Column("l")
+    private int mapsRunning;
+    @Column("m")
+    private int reducesPending;
+    @Column("n")
+    private int reducesRunning;
+    @Column("o")
+    private int newReduceAttempts;
+    @Column("p")
+    private int runningReduceAttempts;
+    @Column("q")
+    private int failedReduceAttempts;
+    @Column("r")
+    private int killedReduceAttempts;
+    @Column("s")
+    private int successfulReduceAttempts;
+    @Column("t")
+    private int newMapAttempts;
+    @Column("u")
+    private int runningMapAttempts;
+    @Column("v")
+    private int failedMapAttempts;
+    @Column("w")
+    private int killedMapAttempts;
+    @Column("x")
+    private int successfulMapAttempts;
+    @Column("y")
+    private AppInfo appInfo;
+    @Column("z")
+    private JobCounters jobCounters;
+    @Column("aa")
+    private JobConfig jobConfig;
+    @Column("ab")
+    private long allocatedMB;
+    @Column("ac")
+    private int allocatedVCores;
+    @Column("ad")
+    private int runningContainers;
+    @Column("ae")
+    private int dataLocalMaps;
+    @Column("af")
+    private double dataLocalMapsPercentage;
+    @Column("ag")
+    private int rackLocalMaps;
+    @Column("ah")
+    private double rackLocalMapsPercentage;
+    @Column("ai")
+    private int totalLaunchedMaps;
+    @Column("aj")
+    private long submissionTime;
+
+    public JobConfig getJobConfig() {
+        return jobConfig;
+    }
+
+    public void setJobConfig(JobConfig jobConfig) {
+        this.jobConfig = jobConfig;
+        valueChanged("jobConfig");
+    }
+
+    public JobCounters getJobCounters() {
+        return jobCounters;
+    }
+
+    public void setJobCounters(JobCounters jobCounters) {
+        this.jobCounters = jobCounters;
+        valueChanged("jobCounters");
+    }
+
+    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;
+        valueChanged("endTime");
+    }
+
+    public long getDurationTime() {
+        return durationTime;
+    }
+
+    public void setDurationTime(long durationTime) {
+        this.durationTime = durationTime;
+        valueChanged("durationTime");
+    }
+
+    public String getCurrentState() {
+        return currentState;
+    }
+
+    public void setCurrentState(String currentState) {
+        this.currentState = currentState;
+        valueChanged("currentState");
+    }
+
+    public int getNumTotalMaps() {
+        return numTotalMaps;
+    }
+
+    public void setNumTotalMaps(int numTotalMaps) {
+        this.numTotalMaps = numTotalMaps;
+        valueChanged("numTotalMaps");
+    }
+
+    public int getMapsCompleted() {
+        return mapsCompleted;
+    }
+
+    public void setMapsCompleted(int mapsCompleted) {
+        this.mapsCompleted = mapsCompleted;
+        valueChanged("mapsCompleted");
+    }
+
+    public int getNumTotalReduces() {
+        return numTotalReduces;
+    }
+
+    public void setNumTotalReduces(int numTotalReduces) {
+        this.numTotalReduces = numTotalReduces;
+        valueChanged("numTotalReduces");
+    }
+
+    public int getReducesCompleted() {
+        return reducesCompleted;
+    }
+
+    public void setReducesCompleted(int reducesCompleted) {
+        this.reducesCompleted = reducesCompleted;
+        valueChanged("reducesCompleted");
+    }
+
+    public double getMapProgress() {
+        return mapProgress;
+    }
+
+    public void setMapProgress(double mapProgress) {
+        this.mapProgress = mapProgress;
+        valueChanged("mapProgress");
+    }
+
+    public double getReduceProgress() {
+        return reduceProgress;
+    }
+
+    public void setReduceProgress(double reduceProgress) {
+        this.reduceProgress = reduceProgress;
+        valueChanged("reduceProgress");
+    }
+
+    public int getMapsPending() {
+        return mapsPending;
+    }
+
+    public void setMapsPending(int mapsPending) {
+        this.mapsPending = mapsPending;
+        valueChanged("mapsPending");
+    }
+
+    public int getMapsRunning() {
+        return mapsRunning;
+    }
+
+    public void setMapsRunning(int mapsRunning) {
+        this.mapsRunning = mapsRunning;
+        valueChanged("mapsRunning");
+    }
+
+    public int getReducesPending() {
+        return reducesPending;
+    }
+
+    public void setReducesPending(int reducesPending) {
+        this.reducesPending = reducesPending;
+        valueChanged("reducesPending");
+    }
+
+    public int getReducesRunning() {
+        return reducesRunning;
+    }
+
+    public void setReducesRunning(int reducesRunning) {
+        this.reducesRunning = reducesRunning;
+        valueChanged("reducesRunning");
+    }
+
+    public int getNewReduceAttempts() {
+        return newReduceAttempts;
+    }
+
+    public void setNewReduceAttempts(int newReduceAttempts) {
+        this.newReduceAttempts = newReduceAttempts;
+        valueChanged("newReduceAttempts");
+    }
+
+    public int getRunningReduceAttempts() {
+        return runningReduceAttempts;
+    }
+
+    public void setRunningReduceAttempts(int runningReduceAttempts) {
+        this.runningReduceAttempts = runningReduceAttempts;
+        valueChanged("runningReduceAttempts");
+    }
+
+    public int getFailedReduceAttempts() {
+        return failedReduceAttempts;
+    }
+
+    public void setFailedReduceAttempts(int failedReduceAttempts) {
+        this.failedReduceAttempts = failedReduceAttempts;
+        valueChanged("failedReduceAttempts");
+    }
+
+    public int getKilledReduceAttempts() {
+        return killedReduceAttempts;
+    }
+
+    public void setKilledReduceAttempts(int killedReduceAttempts) {
+        this.killedReduceAttempts = killedReduceAttempts;
+        valueChanged("killedReduceAttempts");
+    }
+
+    public int getSuccessfulReduceAttempts() {
+        return successfulReduceAttempts;
+    }
+
+    public void setSuccessfulReduceAttempts(int successfulReduceAttempts) {
+        this.successfulReduceAttempts = successfulReduceAttempts;
+        valueChanged("successfulReduceAttempts");
+    }
+
+    public int getNewMapAttempts() {
+        return newMapAttempts;
+    }
+
+    public void setNewMapAttempts(int newMapAttempts) {
+        this.newMapAttempts = newMapAttempts;
+        valueChanged("newMapAttempts");
+    }
+
+    public int getRunningMapAttempts() {
+        return runningMapAttempts;
+    }
+
+    public void setRunningMapAttempts(int runningMapAttempts) {
+        this.runningMapAttempts = runningMapAttempts;
+        valueChanged("runningMapAttempts");
+    }
+
+    public int getFailedMapAttempts() {
+        return failedMapAttempts;
+    }
+
+    public void setFailedMapAttempts(int failedMapAttempts) {
+        this.failedMapAttempts = failedMapAttempts;
+        valueChanged("failedMapAttempts");
+    }
+
+    public int getKilledMapAttempts() {
+        return killedMapAttempts;
+    }
+
+    public void setKilledMapAttempts(int killedMapAttempts) {
+        this.killedMapAttempts = killedMapAttempts;
+        valueChanged("killedMapAttempts");
+    }
+
+    public int getSuccessfulMapAttempts() {
+        return successfulMapAttempts;
+    }
+
+    public void setSuccessfulMapAttempts(int successfulMapAttempts) {
+        this.successfulMapAttempts = successfulMapAttempts;
+        valueChanged("successfulMapAttempts");
+    }
+
+    public AppInfo getAppInfo() {
+        return appInfo;
+    }
+
+    public void setAppInfo(AppInfo appInfo) {
+        this.appInfo = appInfo;
+        valueChanged("appInfo");
+    }
+
+    public long getAllocatedMB() {
+        return allocatedMB;
+    }
+
+    public void setAllocatedMB(long allocatedMB) {
+        this.allocatedMB = allocatedMB;
+        valueChanged("allocatedMB");
+    }
+
+    public int getAllocatedVCores() {
+        return allocatedVCores;
+    }
+
+    public void setAllocatedVCores(int allocatedVCores) {
+        this.allocatedVCores = allocatedVCores;
+        valueChanged("allocatedVCores");
+    }
+
+    public int getRunningContainers() {
+        return runningContainers;
+    }
+
+    public void setRunningContainers(int runningContainers) {
+        this.runningContainers = runningContainers;
+        valueChanged("runningContainers");
+    }
+
+    public int getDataLocalMaps() {
+        return dataLocalMaps;
+    }
+
+    public void setDataLocalMaps(int dataLocalMaps) {
+        this.dataLocalMaps = dataLocalMaps;
+        valueChanged("dataLocalMaps");
+    }
+
+    public double getDataLocalMapsPercentage() {
+        return dataLocalMapsPercentage;
+    }
+
+    public void setDataLocalMapsPercentage(double dataLocalMapsPercentage) {
+        this.dataLocalMapsPercentage = dataLocalMapsPercentage;
+        valueChanged("dataLocalMapsPercentage");
+    }
+
+    public int getRackLocalMaps() {
+        return rackLocalMaps;
+    }
+
+    public void setRackLocalMaps(int rackLocalMaps) {
+        this.rackLocalMaps = rackLocalMaps;
+        valueChanged("rackLocalMaps");
+    }
+
+    public double getRackLocalMapsPercentage() {
+        return rackLocalMapsPercentage;
+    }
+
+    public void setRackLocalMapsPercentage(double rackLocalMapsPercentage) {
+        this.rackLocalMapsPercentage = rackLocalMapsPercentage;
+        valueChanged("rackLocalMapsPercentage");
+    }
+
+    public int getTotalLaunchedMaps() {
+        return totalLaunchedMaps;
+    }
+
+    public void setTotalLaunchedMaps(int totalLaunchedMaps) {
+        this.totalLaunchedMaps = totalLaunchedMaps;
+        valueChanged("totalLaunchedMaps");
+    }
+
+    public long getSubmissionTime() {
+        return submissionTime;
+    }
+
+    public void setSubmissionTime(long submissionTime) {
+        this.submissionTime = submissionTime;
+        valueChanged("submissionTime");
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/d6ec142d/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
new file mode 100644
index 0000000..11a8b4c
--- /dev/null
+++ b/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/mr/runningentity/TaskAttemptExecutionAPIEntity.java
@@ -0,0 +1,137 @@
+/*
+ *  Licensed to the Apache Software Foundation (ASF) under one
+ *  or more contributor license agreements.  See the NOTICE file
+ *  distributed with this work for additional information
+ *  regarding copyright ownership.  The ASF licenses this file
+ *  to you under the Apache License, Version 2.0 (the
+ *  "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT 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.runningentity;
+
+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("eagleMRRunningTasks")
+@ColumnFamily("f")
+@Prefix("tasks_exec_attempt")
+@Service(Constants.JPA_RUNNING_TASK_ATTEMPT_EXECUTION_SERVICE_NAME)
+@TimeSeries(true)
+@Partition({"site"})
+@Indexes({
+        @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")
+    private long startTime;
+    @Column("b")
+    private long finishTime;
+    @Column("c")
+    private long elapsedTime;
+    @Column("d")
+    private double progress;
+    @Column("e")
+    private String id;
+    @Column("f")
+    private String status;
+    @Column("g")
+    private String diagnostics;
+    @Column("h")
+    private String type;
+    @Column("i")
+    private String assignedContainerId;
+
+    public long getStartTime() {
+        return startTime;
+    }
+
+    public void setStartTime(long startTime) {
+        this.startTime = startTime;
+        valueChanged("startTime");
+    }
+
+    public long getFinishTime() {
+        return finishTime;
+    }
+
+    public void setFinishTime(long finishTime) {
+        this.finishTime = finishTime;
+        valueChanged("finishTime");
+    }
+
+    public long getElapsedTime() {
+        return elapsedTime;
+    }
+
+    public void setElapsedTime(long elapsedTime) {
+        this.elapsedTime = elapsedTime;
+        valueChanged("elapsedTime");
+    }
+
+    public double getProgress() {
+        return progress;
+    }
+
+    public void setProgress(double progress) {
+        this.progress = progress;
+        valueChanged("progress");
+    }
+
+    public String getId() {
+        return id;
+    }
+
+    public void setId(String id) {
+        this.id = id;
+        valueChanged("id");
+    }
+
+    public String getStatus() {
+        return status;
+    }
+
+    public void setStatus(String status) {
+        this.status = status;
+        valueChanged("status");
+    }
+
+    public String getDiagnostics() {
+        return diagnostics;
+    }
+
+    public void setDiagnostics(String diagnostics) {
+        this.diagnostics = diagnostics;
+        valueChanged("diagnostics");
+    }
+
+    public String getType() {
+        return type;
+    }
+
+    public void setType(String type) {
+        this.type = type;
+        valueChanged("type");
+    }
+
+    public String getAssignedContainerId() {
+        return assignedContainerId;
+    }
+
+    public void setAssignedContainerId(String assignedContainerId) {
+        this.assignedContainerId = assignedContainerId;
+        valueChanged("assignedContainerId");
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/d6ec142d/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
new file mode 100644
index 0000000..50e042f
--- /dev/null
+++ b/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/mr/runningentity/TaskExecutionAPIEntity.java
@@ -0,0 +1,127 @@
+/*
+ *  Licensed to the Apache Software Foundation (ASF) under one
+ *  or more contributor license agreements.  See the NOTICE file
+ *  distributed with this work for additional information
+ *  regarding copyright ownership.  The ASF licenses this file
+ *  to you under the Apache License, Version 2.0 (the
+ *  "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT 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.runningentity;
+
+import org.apache.eagle.jpm.util.Constants;
+import org.apache.eagle.jpm.util.jobcounter.JobCounters;
+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("eagleMRRunningTasks")
+@ColumnFamily("f")
+@Prefix("tasks_exec")
+@Service(Constants.JPA_RUNNING_TASK_EXECUTION_SERVICE_NAME)
+@TimeSeries(true)
+@Partition({"site"})
+@Indexes({
+        @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")
+    private long startTime;
+    @Column("b")
+    private long endTime;
+    @Column("c")
+    private long duration;
+    @Column("d")
+    private double progress;
+    @Column("e")
+    private String taskStatus;
+    @Column("f")
+    private String successfulAttempt;
+    @Column("g")
+    private String statusDesc;
+    @Column("h")
+    private JobCounters jobCounters;
+
+    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;
+        valueChanged("endTime");
+    }
+
+    public long getDuration() {
+        return duration;
+    }
+
+    public void setDuration(long duration) {
+        this.duration = duration;
+        valueChanged("duration");
+    }
+
+    public double getProgress() {
+        return progress;
+    }
+
+    public void setProgress(double progress) {
+        this.progress = progress;
+        valueChanged("progress");
+    }
+
+    public String getTaskStatus() {
+        return taskStatus;
+    }
+
+    public void setTaskStatus(String taskStatus) {
+        this.taskStatus = taskStatus;
+        valueChanged("taskStatus");
+    }
+
+    public String getSuccessfulAttempt() {
+        return successfulAttempt;
+    }
+
+    public void setSuccessfulAttempt(String successfulAttempt) {
+        this.successfulAttempt = successfulAttempt;
+        valueChanged("successfulAttempt");
+    }
+
+    public String getStatusDesc() {
+        return statusDesc;
+    }
+
+    public void setStatusDesc(String statusDesc) {
+        this.statusDesc = statusDesc;
+        valueChanged("statusDesc");
+    }
+
+    public JobCounters getJobCounters() {
+        return jobCounters;
+    }
+
+    public void setJobCounters(JobCounters jobCounters) {
+        this.jobCounters = jobCounters;
+        valueChanged("jobCounters");
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/d6ec142d/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 d5fda5a..1b75e81 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
@@ -19,8 +19,8 @@ package org.apache.eagle.jpm.spark.crawl;
 
 import com.typesafe.config.Config;
 import com.typesafe.config.ConfigFactory;
-import jline.internal.Log;
-import org.apache.eagle.jpm.entity.*;
+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;
 import org.apache.eagle.jpm.util.SparkEntityConstant;
@@ -673,7 +673,7 @@ public class JHFSparkEventReader {
             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.info("Cannot parse memory info " +  memory);
         return 0l;
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/d6ec142d/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/spark/entity/JPMEntityRepository.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/spark/entity/JPMEntityRepository.java b/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/spark/entity/JPMEntityRepository.java
new file mode 100644
index 0000000..3cb57f5
--- /dev/null
+++ b/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/spark/entity/JPMEntityRepository.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
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT 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.entity;
+
+import org.apache.eagle.log.entity.repo.EntityRepository;
+
+public class JPMEntityRepository extends EntityRepository {
+    public JPMEntityRepository() {
+        entitySet.add(SparkApp.class);
+        entitySet.add(SparkJob.class);
+        entitySet.add(SparkStage.class);
+        entitySet.add(SparkTask.class);
+        entitySet.add(SparkExecutor.class);
+        serDeserMap.put(JobConfig.class, new JobConfigSerDeser());
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/d6ec142d/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
new file mode 100644
index 0000000..11c4a22
--- /dev/null
+++ b/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/spark/entity/JobConfig.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.spark.entity;
+
+import java.io.Serializable;
+import java.util.Map;
+import java.util.TreeMap;
+
+public class JobConfig implements Serializable {
+    private Map<String, String> config = new TreeMap<>();
+
+    public Map<String, String> getConfig() {
+        return config;
+    }
+
+    public void setConfig(Map<String, String> config) {
+        this.config = config;
+    }
+    @Override
+    public String toString(){
+        return config.toString();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/d6ec142d/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/spark/entity/JobConfigSerDeser.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/spark/entity/JobConfigSerDeser.java b/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/spark/entity/JobConfigSerDeser.java
new file mode 100644
index 0000000..69f0f0d
--- /dev/null
+++ b/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/spark/entity/JobConfigSerDeser.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.spark.entity;
+
+import org.apache.eagle.log.entity.meta.EntitySerDeser;
+import org.apache.eagle.log.entity.meta.MapSerDeser;
+
+import java.util.Map;
+
+public class JobConfigSerDeser implements EntitySerDeser<JobConfig> {
+    private static final MapSerDeser INSTANCE = new MapSerDeser();
+
+    @Override
+    public JobConfig deserialize(byte[] bytes) {
+        Map map = INSTANCE.deserialize(bytes);
+        JobConfig config = new JobConfig();
+        config.setConfig(map);
+        return config;
+    }
+
+    @Override
+    public byte[] serialize(JobConfig jobConfig) {
+        return INSTANCE.serialize(jobConfig.getConfig());
+    }
+
+    @Override
+    public Class<JobConfig> type() {
+        return JobConfig.class;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/d6ec142d/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
new file mode 100644
index 0000000..528a91f
--- /dev/null
+++ b/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/spark/entity/SparkApp.java
@@ -0,0 +1,429 @@
+/*
+ *  Licensed to the Apache Software Foundation (ASF) under one
+ *  or more contributor license agreements.  See the NOTICE file
+ *  distributed with this work for additional information
+ *  regarding copyright ownership.  The ASF licenses this file
+ *  to you under the Apache License, Version 2.0 (the
+ *  "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package org.apache.eagle.jpm.spark.entity;
+
+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")
+@Prefix("sprkapp")
+@Service(Constants.SPARK_APP_SERVICE_ENDPOINT_NAME)
+@JsonIgnoreProperties(ignoreUnknown = true)
+@TimeSeries(true)
+@Tags({"site","sprkAppId", "sprkAppAttemptId", "sprkAppName", "normSprkAppName","user", "queue"})
+@Partition({"site"})
+public class SparkApp 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;
+
+    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/d6ec142d/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
new file mode 100644
index 0000000..366e4aa
--- /dev/null
+++ b/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/spark/entity/SparkExecutor.java
@@ -0,0 +1,234 @@
+/*
+ *  Licensed to the Apache Software Foundation (ASF) under one
+ *  or more contributor license agreements.  See the NOTICE file
+ *  distributed with this work for additional information
+ *  regarding copyright ownership.  The ASF licenses this file
+ *  to you under the Apache License, Version 2.0 (the
+ *  "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT 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.entity;
+
+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")
+@Prefix("sprkexcutr")
+@Service(Constants.SPARK_EXECUTOR_SERVICE_ENDPOINT_NAME)
+@JsonIgnoreProperties(ignoreUnknown = true)
+@TimeSeries(true)
+@Tags({"site","sprkAppId", "sprkAppAttemptId", "sprkAppName", "normSprkAppName", "executorId","user", "queue"})
+@Partition({"site"})
+public class SparkExecutor 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/d6ec142d/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
new file mode 100644
index 0000000..acecb3a
--- /dev/null
+++ b/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/spark/entity/SparkJob.java
@@ -0,0 +1,179 @@
+/*
+ *  Licensed to the Apache Software Foundation (ASF) under one
+ *  or more contributor license agreements.  See the NOTICE file
+ *  distributed with this work for additional information
+ *  regarding copyright ownership.  The ASF licenses this file
+ *  to you under the Apache License, Version 2.0 (the
+ *  "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package org.apache.eagle.jpm.spark.entity;
+
+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")
+@Prefix("sprkjob")
+@Service(Constants.SPARK_JOB_SERVICE_ENDPOINT_NAME)
+@JsonIgnoreProperties(ignoreUnknown = true)
+@TimeSeries(true)
+@Tags({"site","sprkAppId", "sprkAppAttemptId", "sprkAppName", "normSprkAppName", "jobId","user", "queue"})
+@Partition({"site"})
+public class SparkJob 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;
+
+    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/d6ec142d/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
new file mode 100644
index 0000000..fcca889
--- /dev/null
+++ b/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/spark/entity/SparkStage.java
@@ -0,0 +1,300 @@
+/*
+ *  Licensed to the Apache Software Foundation (ASF) under one
+ *  or more contributor license agreements.  See the NOTICE file
+ *  distributed with this work for additional information
+ *  regarding copyright ownership.  The ASF licenses this file
+ *  to you under the Apache License, Version 2.0 (the
+ *  "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT 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.entity;
+
+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")
+@Prefix("sprkstage")
+@Service(Constants.SPARK_STAGE_SERVICE_ENDPOINT_NAME)
+@JsonIgnoreProperties(ignoreUnknown = true)
+@TimeSeries(true)
+@Tags({"site","sprkAppId", "sprkAppAttemptId", "sprkAppName", "normSprkAppName", "jobId", "stageId","stageAttemptId","user", "queue"})
+@Partition({"site"})
+public class SparkStage 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/d6ec142d/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
new file mode 100644
index 0000000..6ef7c69
--- /dev/null
+++ b/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/spark/entity/SparkTask.java
@@ -0,0 +1,291 @@
+/*
+ *  Licensed to the Apache Software Foundation (ASF) under one
+ *  or more contributor license agreements.  See the NOTICE file
+ *  distributed with this work for additional information
+ *  regarding copyright ownership.  The ASF licenses this file
+ *  to you under the Apache License, Version 2.0 (the
+ *  "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT 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.entity;
+
+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")
+@Prefix("sprktask")
+@Service(Constants.SPARK_TASK_SERVICE_ENDPOINT_NAME)
+@JsonIgnoreProperties(ignoreUnknown = true)
+@TimeSeries(true)
+@Tags({"site","sprkAppId", "sprkAppAttemptId", "sprkAppName", "normSprkAppName", "jobId", "jobName", "stageId","stageAttemptId","taskIndex","taskAttemptId","user", "queue"})
+@Partition({"site"})
+public class SparkTask 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/d6ec142d/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 97be7ec..5568284 100644
--- a/eagle-jpm/eagle-jpm-mr-history/pom.xml
+++ b/eagle-jpm/eagle-jpm-mr-history/pom.xml
@@ -43,6 +43,11 @@
         </dependency>
         <dependency>
             <groupId>org.apache.eagle</groupId>
+            <artifactId>eagle-jpm-entity</artifactId>
+            <version>${project.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.eagle</groupId>
             <artifactId>eagle-stream-process-api</artifactId>
             <version>${project.version}</version>
             <exclusions>

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/d6ec142d/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 c6f1b98..9f030a7 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
@@ -40,12 +40,15 @@ public class MRHistoryJobMain {
 
             //2. init JobHistoryContentFilter
             JobHistoryContentFilterBuilder builder = JobHistoryContentFilterBuilder.newBuilder().acceptJobFile().acceptJobConfFile();
-            List<String> confKeyPatterns = jhfAppConf.getStringList("MRConfigureKeys");
+            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));
             }

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/d6ec142d/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 6fbf3d3..66dbce1 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
@@ -22,15 +22,11 @@ import java.io.Serializable;
 import java.util.List;
 import java.util.regex.Pattern;
 
-/**
- * define what content in job history stream should be streamed
- * @author yonzhang
- *
- */
 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/d6ec142d/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 43234c2..65b8dab 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
@@ -35,6 +35,8 @@ public class JobHistoryContentFilterBuilder {
     private List<Pattern> m_jobConfKeyInclusionPatterns;
     private List<Pattern> m_jobConfKeyExclusionPatterns;
 
+    private String jobNameKey;
+
     public static JobHistoryContentFilterBuilder newBuilder(){
         return new JobHistoryContentFilterBuilder();
     }
@@ -78,6 +80,11 @@ public class JobHistoryContentFilterBuilder {
         return this;
     }
 
+    public JobHistoryContentFilterBuilder setJobNameKey(String jobNameKey) {
+        this.jobNameKey = jobNameKey;
+        return this;
+    }
+
     public JobHistoryContentFilter build() {
         JobHistoryContentFilterImpl filter = new JobHistoryContentFilterImpl();
         filter.setAcceptJobFile(m_acceptJobFile);
@@ -85,6 +92,7 @@ public class JobHistoryContentFilterBuilder {
         filter.setMustHaveJobConfKeyPatterns(m_mustHaveJobConfKeyPatterns);
         filter.setJobConfKeyInclusionPatterns(m_jobConfKeyInclusionPatterns);
         filter.setJobConfKeyExclusionPatterns(m_jobConfKeyExclusionPatterns);
+        filter.setJobNameKey(jobNameKey);
         LOG.info("job history content filter:" + filter);
         return filter;
     }

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/d6ec142d/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 d8a482b..5e7a856 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
@@ -28,6 +28,8 @@ public class JobHistoryContentFilterImpl implements JobHistoryContentFilter {
     private List<Pattern> m_jobConfKeyInclusionPatterns;
     private List<Pattern> m_jobConfKeyExclusionPatterns;
 
+    private String jobNameKey;
+
     @Override
     public boolean acceptJobFile() {
         return m_acceptJobFile;
@@ -53,6 +55,15 @@ public class JobHistoryContentFilterImpl implements JobHistoryContentFilter {
         return m_jobConfKeyExclusionPatterns;
     }
 
+    @Override
+    public String getJobNameKey() {
+        return jobNameKey;
+    }
+
+    public void setJobNameKey(String jobNameKey) {
+        this.jobNameKey = jobNameKey;
+    }
+
     public void setAcceptJobFile(boolean acceptJobFile) {
         this.m_acceptJobFile = acceptJobFile;
     }



[2/4] incubator-eagle git commit: [EAGLE-467] Job list apis for querying jobs regardless of the status

Posted by qi...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/d6ec142d/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/entities/JPAEntityRepository.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/entities/JPAEntityRepository.java b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/entities/JPAEntityRepository.java
deleted file mode 100755
index 964d68a..0000000
--- a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/entities/JPAEntityRepository.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.mr.history.entities;
-
-import org.apache.eagle.jpm.util.jobcounter.JobCounters;
-import org.apache.eagle.jpm.util.jobcounter.JobCountersSerDeser;
-import org.apache.eagle.log.entity.repo.EntityRepository;
-
-public class JPAEntityRepository extends EntityRepository {
-
-    public JPAEntityRepository() {
-        serDeserMap.put(JobCounters.class, new JobCountersSerDeser());
-        serDeserMap.put(JobConfig.class, new JobConfigSerDeser());
-        entitySet.add(JobConfigurationAPIEntity.class);
-        entitySet.add(JobEventAPIEntity.class);
-        entitySet.add(JobExecutionAPIEntity.class);
-
-        entitySet.add(TaskAttemptExecutionAPIEntity.class);
-        entitySet.add(TaskExecutionAPIEntity.class);
-        entitySet.add(TaskFailureCountAPIEntity.class);
-        entitySet.add(TaskAttemptCounterAPIEntity.class);
-        entitySet.add(JobProcessTimeStampEntity.class);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/d6ec142d/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/entities/JobBaseAPIEntity.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/entities/JobBaseAPIEntity.java b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/entities/JobBaseAPIEntity.java
deleted file mode 100644
index 32c6f7c..0000000
--- a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/entities/JobBaseAPIEntity.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.mr.history.entities;
-
-import org.apache.eagle.log.base.taggedlog.TaggedLogAPIEntity;
-
-public class JobBaseAPIEntity extends TaggedLogAPIEntity {
-}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/d6ec142d/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/entities/JobConfig.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/entities/JobConfig.java b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/entities/JobConfig.java
deleted file mode 100644
index f1dc375..0000000
--- a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/entities/JobConfig.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.mr.history.entities;
-
-import java.util.Map;
-import java.util.TreeMap;
-
-public final class JobConfig {
-    private Map<String, String> config = new TreeMap<>();
-
-    public Map<String, String> getConfig() {
-        return config;
-    }
-
-    public void setConfig(Map<String, String> config) {
-        this.config = config;
-    }
-    
-    public String toString(){
-        return config.toString();
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/d6ec142d/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/entities/JobConfigSerDeser.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/entities/JobConfigSerDeser.java b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/entities/JobConfigSerDeser.java
deleted file mode 100755
index 8776f1f..0000000
--- a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/entities/JobConfigSerDeser.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.mr.history.entities;
-
-import org.apache.eagle.log.entity.meta.EntitySerDeser;
-import org.apache.hadoop.hbase.util.Bytes;
-
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.TreeMap;
-
-public class JobConfigSerDeser implements EntitySerDeser<JobConfig> {
-
-    @Override
-    public JobConfig deserialize(byte[] bytes) {
-        JobConfig jc = new JobConfig();
-        Map<String, String> map = new TreeMap<String, String>();
-        jc.setConfig(map);
-        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 key = str[0];
-            String value = "";
-            if (str.length == 2) value = str[1];
-            map.put(key, value);
-        }
-        return jc;
-    }
-    
-    @Override
-    public byte[] serialize(JobConfig conf) {
-        Map<String, String> map = conf.getConfig();
-        StringBuilder sb = new StringBuilder();
-        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(){
-        return JobConfig.class;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/d6ec142d/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/entities/JobConfigurationAPIEntity.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/entities/JobConfigurationAPIEntity.java b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/entities/JobConfigurationAPIEntity.java
deleted file mode 100755
index 295cc68..0000000
--- a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/entities/JobConfigurationAPIEntity.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.mr.history.entities;
-
-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)
-@Table("eaglejpa")
-@ColumnFamily("f")
-@Prefix("jconf")
-@Service(Constants.JPA_JOB_CONFIG_SERVICE_NAME)
-@TimeSeries(true)
-@Partition({"site"})
-@Indexes({
-        @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")
-    private String configJobName;
-    @Column("b")
-    private JobConfig jobConfig;
-    @Column("c")
-    private String alertEmailList;
-    
-    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/d6ec142d/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/entities/JobEventAPIEntity.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/entities/JobEventAPIEntity.java b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/entities/JobEventAPIEntity.java
deleted file mode 100644
index 31dd480..0000000
--- a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/entities/JobEventAPIEntity.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.mr.history.entities;
-
-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)
-@Table("eaglejpa")
-@ColumnFamily("f")
-@Prefix("jevent")
-@Service(Constants.JPA_JOB_EVENT_SERVICE_NAME)
-@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/d6ec142d/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/entities/JobExecutionAPIEntity.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/entities/JobExecutionAPIEntity.java b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/entities/JobExecutionAPIEntity.java
deleted file mode 100755
index e0ec330..0000000
--- a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/entities/JobExecutionAPIEntity.java
+++ /dev/null
@@ -1,220 +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.entities;
-
-import org.apache.eagle.jpm.util.Constants;
-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)
-@Table("eaglejpa")
-@ColumnFamily("f")
-@Prefix("jexec")
-@Service(Constants.JPA_JOB_EXECUTION_SERVICE_NAME)
-@TimeSeries(true)
-@Partition({"site"})
-@Indexes({
-    @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")
-    private String currentState;
-    @Column("b")
-    private long startTime;
-    @Column("c")
-    private long endTime;
-    @Column("d")
-    private int numTotalMaps;
-    @Column("e")
-    private int numFailedMaps;
-    @Column("f")
-    private int numFinishedMaps;
-    @Column("g")
-    private int numTotalReduces;
-    @Column("h")
-    private int numFailedReduces;
-    @Column("i")
-    private int numFinishedReduces;
-    @Column("j")
-    private JobCounters jobCounters;
-    @Column("k")
-    private int dataLocalMaps;
-    @Column("l")
-    private double dataLocalMapsPercentage;
-    @Column("m")
-    private int rackLocalMaps;
-    @Column("n")
-    private double rackLocalMapsPercentage;
-    @Column("o")
-    private int totalLaunchedMaps;
-    @Column("p")
-    private long submissionTime;
-    @Column("q")
-    private long lastMapDuration;
-    @Column("r")
-    private long lastReduceDuration;
-
-    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;
-    }
-
-    public void setNumFinishedReduces(int numFinishedReduces) {
-        this.numFinishedReduces = numFinishedReduces;
-        _pcs.firePropertyChange("numFinishedReduces", null, null);
-    }
-
-    public JobCounters getJobCounters() {
-        return jobCounters;
-    }
-
-    public void setJobCounters(JobCounters jobCounters) {
-        this.jobCounters = jobCounters;
-        _pcs.firePropertyChange("jobCounters", null, null);
-    }
-
-    public int getDataLocalMaps() {
-        return dataLocalMaps;
-    }
-
-    public void setDataLocalMaps(int dataLocalMaps) {
-        this.dataLocalMaps = dataLocalMaps;
-        valueChanged("dataLocalMaps");
-    }
-
-    public double getDataLocalMapsPercentage() {
-        return dataLocalMapsPercentage;
-    }
-
-    public void setDataLocalMapsPercentage(double dataLocalMapsPercentage) {
-        this.dataLocalMapsPercentage = dataLocalMapsPercentage;
-        valueChanged("dataLocalMapsPercentage");
-    }
-
-    public int getRackLocalMaps() {
-        return rackLocalMaps;
-    }
-
-    public void setRackLocalMaps(int rackLocalMaps) {
-        this.rackLocalMaps = rackLocalMaps;
-        valueChanged("rackLocalMaps");
-    }
-
-    public double getRackLocalMapsPercentage() {
-        return rackLocalMapsPercentage;
-    }
-
-    public void setRackLocalMapsPercentage(double rackLocalMapsPercentage) {
-        this.rackLocalMapsPercentage = rackLocalMapsPercentage;
-        valueChanged("rackLocalMapsPercentage");
-    }
-
-    public int getTotalLaunchedMaps() {
-        return totalLaunchedMaps;
-    }
-
-    public void setTotalLaunchedMaps(int totalLaunchedMaps) {
-        this.totalLaunchedMaps = totalLaunchedMaps;
-        valueChanged("totalLaunchedMaps");
-    }
-
-    public long getSubmissionTime() {
-        return submissionTime;
-    }
-
-    public void setSubmissionTime(long submissionTime) {
-        this.submissionTime = submissionTime;
-        valueChanged("submissionTime");
-    }
-
-    public long getLastMapDuration() {
-        return lastMapDuration;
-    }
-
-    public void setLastMapDuration(long lastMapDuration) {
-        this.lastMapDuration = lastMapDuration;
-        valueChanged("lastMapDuration");
-    }
-
-    public long getLastReduceDuration() {
-        return lastReduceDuration;
-    }
-
-    public void setLastReduceDuration(long lastReduceDuration) {
-        this.lastReduceDuration = lastReduceDuration;
-        valueChanged("lastReduceDuration");
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/d6ec142d/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/entities/JobProcessTimeStampEntity.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/entities/JobProcessTimeStampEntity.java b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/entities/JobProcessTimeStampEntity.java
deleted file mode 100644
index 9e8a372..0000000
--- a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/entities/JobProcessTimeStampEntity.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.mr.history.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.map.annotate.JsonSerialize;
-
-@JsonSerialize(include=JsonSerialize.Inclusion.NON_NULL)
-@Table("eaglejpa_process")
-@ColumnFamily("f")
-@Prefix("process")
-@Service(Constants.JPA_JOB_PROCESS_TIME_STAMP_NAME)
-@TimeSeries(true)
-@Partition({"site"})
-public class JobProcessTimeStampEntity extends TaggedLogAPIEntity {
-    @Column("a")
-    private long currentTimeStamp;
-
-    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/d6ec142d/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/entities/TaskAttemptCounterAPIEntity.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/entities/TaskAttemptCounterAPIEntity.java b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/entities/TaskAttemptCounterAPIEntity.java
deleted file mode 100644
index 929a98f..0000000
--- a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/entities/TaskAttemptCounterAPIEntity.java
+++ /dev/null
@@ -1,61 +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.entities;
-
-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)
-@Table("eaglejpa_anomaly")
-@ColumnFamily("f")
-@Prefix("tacount")
-@Service(Constants.JPA_TASK_ATTEMPT_COUNTER_SERVICE_NAME)
-@TimeSeries(true)
-@Partition({"site"})
-public class TaskAttemptCounterAPIEntity extends JobBaseAPIEntity {
-    @Column("a")
-    private int totalCount;
-    @Column("b")
-    private int failedCount;
-    @Column("c")
-    private int killedCount;
-    
-    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/d6ec142d/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/entities/TaskAttemptExecutionAPIEntity.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/entities/TaskAttemptExecutionAPIEntity.java b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/entities/TaskAttemptExecutionAPIEntity.java
deleted file mode 100755
index abc28e2..0000000
--- a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/entities/TaskAttemptExecutionAPIEntity.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.mr.history.entities;
-
-import org.apache.eagle.jpm.util.Constants;
-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)
-@Table("eaglejpa_task")
-@ColumnFamily("f")
-@Prefix("taexec")
-@Service(Constants.JPA_TASK_ATTEMPT_EXECUTION_SERVICE_NAME)
-@TimeSeries(true)
-@Partition({"site"})
-@Indexes({
-    @Index(name="Index_1_jobId", columns = { "jobID" }, unique = false)
-    })
-public class TaskAttemptExecutionAPIEntity extends JobBaseAPIEntity {
-    @Column("a")
-    private String taskStatus;
-    @Column("b")
-    private long startTime;
-    @Column("c")
-    private long endTime;
-    @Column("d")
-    private long duration;
-    @Column("e")
-    private String error;
-    @Column("f")
-    private JobCounters jobCounters;
-    @Column("g")
-    private String taskAttemptID;
-
-    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/d6ec142d/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/entities/TaskExecutionAPIEntity.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/entities/TaskExecutionAPIEntity.java b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/entities/TaskExecutionAPIEntity.java
deleted file mode 100644
index c1f71b8..0000000
--- a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/entities/TaskExecutionAPIEntity.java
+++ /dev/null
@@ -1,89 +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.entities;
-
-import org.apache.eagle.jpm.util.Constants;
-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)
-@Table("eaglejpa_task")
-@ColumnFamily("f")
-@Prefix("texec")
-@Service(Constants.JPA_TASK_EXECUTION_SERVICE_NAME)
-@TimeSeries(true)
-@Partition({"site"})
-public class TaskExecutionAPIEntity extends JobBaseAPIEntity {
-    @Column("a")
-    private String taskStatus;
-    @Column("b")
-    private long startTime;
-    @Column("c")
-    private long endTime;
-    @Column("d")
-    private long duration;
-    @Column("e")
-    private String error;
-    @Column("f")
-    private JobCounters jobCounters;
-
-    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/d6ec142d/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/entities/TaskFailureCountAPIEntity.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/entities/TaskFailureCountAPIEntity.java b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/entities/TaskFailureCountAPIEntity.java
deleted file mode 100755
index 7456522..0000000
--- a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/entities/TaskFailureCountAPIEntity.java
+++ /dev/null
@@ -1,67 +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.entities;
-
-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)
-@Table("eaglejpa_anomaly")
-@ColumnFamily("f")
-@Prefix("taskfailurecount")
-@Service(Constants.JPA_TASK_FAILURE_COUNT_SERVICE_NAME)
-@TimeSeries(true)
-@Partition({"site"})
-public class TaskFailureCountAPIEntity extends JobBaseAPIEntity {
-    @Column("a")
-    private int failureCount;
-    @Column("b")
-    private String error;
-    @Column("c")
-    private String taskStatus;
-
-
-    public String getTaskStatus() {
-        return taskStatus;
-    }
-
-    public void setTaskStatus(String taskStatus) {
-        this.taskStatus = taskStatus;
-        _pcs.firePropertyChange("taskStatus", null, null);
-    }
-
-    public String getError() {
-        return error;
-    }
-
-    public void setError(String error) {
-        this.error = error;
-        _pcs.firePropertyChange("error", null, null);
-    }
-
-    public int getFailureCount() {
-        return failureCount;
-    }
-
-    public void setFailureCount(int failureCount) {
-        this.failureCount = failureCount;
-        _pcs.firePropertyChange("failureCount", null, null);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/d6ec142d/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 559f7a8..bdaedd4 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
@@ -19,7 +19,7 @@
 package org.apache.eagle.jpm.mr.history.parser;
 
 
-import org.apache.eagle.jpm.mr.history.entities.JobBaseAPIEntity;
+import org.apache.eagle.jpm.mr.historyentity.JobBaseAPIEntity;
 
 /**
  * generalizing this listener would decouple entity creation and entity handling, also will help unit testing

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/d6ec142d/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 fc678f8..ae6b5c9 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
@@ -18,7 +18,7 @@
 
 package org.apache.eagle.jpm.mr.history.parser;
 
-import org.apache.eagle.jpm.mr.history.entities.JobBaseAPIEntity;
+import org.apache.eagle.jpm.mr.historyentity.JobBaseAPIEntity;
 
 public interface HistoryJobEntityLifecycleListener extends HistoryJobEntityCreationListener {
     /**

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/d6ec142d/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 6442699..af3df85 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.eagle.jpm.mr.history.entities.JobConfig;
+import org.apache.eagle.jpm.mr.historyentity.JobConfig;
 import org.apache.eagle.jpm.mr.history.crawler.JobHistoryContentFilter;
-import org.apache.eagle.jpm.mr.history.entities.*;
+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;
@@ -167,20 +167,21 @@ public abstract class JHFEventReaderBase extends JobEntityCreationPublisher impl
            m_user = values.get(Keys.USER);
            m_queueName = values.get(Keys.JOB_QUEUE);
            m_jobName = values.get(Keys.JOBNAME);
-           m_jobDefId = m_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(Constants.JOB_DEFINITION_ID_KEY);
+           if(configuration != null ) {
+               jobDefId = configuration.get(m_filter.getJobNameKey());
+           }
 
            if(jobDefId == null) {
                m_jobDefId = JobNameNormalization.getInstance().normalize(m_jobName);
            } else {
-               LOG.debug("Got normJobName from job configuration for " + id + ": " + jobDefId);
+               LOG.debug("Got JobDefId from job configuration for " + id + ": " + jobDefId);
                m_jobDefId = jobDefId;
            }
 
-           LOG.info("NormJobName of " + id + ": " + m_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);
@@ -222,6 +223,7 @@ public abstract class JHFEventReaderBase extends JobEntityCreationPublisher impl
            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) {

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/d6ec142d/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 278deca..654f63f 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,7 +19,7 @@
 package org.apache.eagle.jpm.mr.history.parser;
 
 import org.apache.eagle.jpm.mr.history.crawler.JobHistoryContentFilter;
-import org.apache.eagle.jpm.mr.history.entities.JobExecutionAPIEntity;
+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;

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/d6ec142d/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 a748565..0334c9b 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
@@ -19,8 +19,8 @@
 package org.apache.eagle.jpm.mr.history.parser;
 
 import org.apache.eagle.jpm.mr.history.common.JHFConfigManager;
-import org.apache.eagle.jpm.mr.history.entities.JobBaseAPIEntity;
-import org.apache.eagle.jpm.mr.history.entities.JobConfigurationAPIEntity;
+import org.apache.eagle.jpm.mr.historyentity.JobBaseAPIEntity;
+import org.apache.eagle.jpm.mr.historyentity.JobConfigurationAPIEntity;
 import org.apache.eagle.service.client.IEagleServiceClient;
 import org.apache.eagle.service.client.impl.EagleServiceClientImpl;
 import org.slf4j.Logger;

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/d6ec142d/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 15c932d..e1a3c69 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,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.entities.*;
+import org.apache.eagle.jpm.mr.historyentity.*;
 import org.apache.eagle.log.entity.GenericServiceAPIResponseEntity;
 import org.apache.eagle.service.client.IEagleServiceClient;
 import org.apache.eagle.service.client.impl.EagleServiceClientImpl;

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/d6ec142d/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 b9dc13b..6e0371c 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
@@ -18,7 +18,7 @@
 
 package org.apache.eagle.jpm.mr.history.parser;
 
-import org.apache.eagle.jpm.mr.history.entities.JobBaseAPIEntity;
+import org.apache.eagle.jpm.mr.historyentity.JobBaseAPIEntity;
 
 import java.util.Vector;
 

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/d6ec142d/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 38ca35c..27de19c 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
@@ -20,9 +20,9 @@ package org.apache.eagle.jpm.mr.history.parser;
 
 import com.fasterxml.jackson.core.JsonProcessingException;
 import com.fasterxml.jackson.databind.ObjectMapper;
-import org.apache.eagle.jpm.mr.history.entities.JobBaseAPIEntity;
-import org.apache.eagle.jpm.mr.history.entities.JobExecutionAPIEntity;
-import org.apache.eagle.jpm.mr.history.entities.TaskAttemptExecutionAPIEntity;
+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 org.slf4j.Logger;

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/d6ec142d/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 94de068..d819baf 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
@@ -19,9 +19,9 @@
 package org.apache.eagle.jpm.mr.history.parser;
 
 import org.apache.eagle.jpm.mr.history.common.JHFConfigManager;
-import org.apache.eagle.jpm.mr.history.entities.JobBaseAPIEntity;
-import org.apache.eagle.jpm.mr.history.entities.TaskAttemptCounterAPIEntity;
-import org.apache.eagle.jpm.mr.history.entities.TaskAttemptExecutionAPIEntity;
+import org.apache.eagle.jpm.mr.historyentity.JobBaseAPIEntity;
+import org.apache.eagle.jpm.mr.historyentity.TaskAttemptCounterAPIEntity;
+import org.apache.eagle.jpm.mr.historyentity.TaskAttemptExecutionAPIEntity;
 import org.apache.eagle.jpm.util.MRJobTagName;
 import org.apache.eagle.service.client.IEagleServiceClient;
 import org.apache.eagle.service.client.impl.EagleServiceClientImpl;

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/d6ec142d/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 177fdc1..6e42fe2 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
@@ -19,9 +19,9 @@
 package org.apache.eagle.jpm.mr.history.parser;
 
 import org.apache.eagle.jpm.mr.history.common.JHFConfigManager;
-import org.apache.eagle.jpm.mr.history.entities.JobBaseAPIEntity;
-import org.apache.eagle.jpm.mr.history.entities.TaskAttemptExecutionAPIEntity;
-import org.apache.eagle.jpm.mr.history.entities.TaskFailureCountAPIEntity;
+import org.apache.eagle.jpm.mr.historyentity.JobBaseAPIEntity;
+import org.apache.eagle.jpm.mr.historyentity.TaskAttemptExecutionAPIEntity;
+import org.apache.eagle.jpm.mr.historyentity.TaskFailureCountAPIEntity;
 import org.apache.eagle.jpm.util.MRJobTagName;
 import org.apache.eagle.service.client.IEagleServiceClient;
 import org.apache.eagle.service.client.impl.EagleServiceClientImpl;

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/d6ec142d/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 6e9ccfa..3daae37 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
@@ -24,7 +24,7 @@ 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.crawler.*;
-import org.apache.eagle.jpm.mr.history.entities.JobProcessTimeStampEntity;
+import org.apache.eagle.jpm.mr.historyentity.JobProcessTimeStampEntity;
 import org.apache.eagle.jpm.mr.history.zkres.JobHistoryZKStateManager;
 import org.apache.eagle.jpm.util.JobIdFilter;
 import org.apache.eagle.jpm.util.JobIdFilterByPartition;

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/d6ec142d/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 60f90a0..23a51fc 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
@@ -63,23 +63,20 @@
     }
   },
 
-  "MRConfigureKeys" : [
-    "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"
-  ]
+  "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"
+    ]
+  }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/d6ec142d/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 1786a44..e409630 100644
--- a/eagle-jpm/eagle-jpm-mr-running/pom.xml
+++ b/eagle-jpm/eagle-jpm-mr-running/pom.xml
@@ -90,6 +90,11 @@
             <artifactId>eagle-jpm-util</artifactId>
             <version>${project.version}</version>
         </dependency>
+        <dependency>
+            <groupId>org.apache.eagle</groupId>
+            <artifactId>eagle-jpm-entity</artifactId>
+            <version>${project.version}</version>
+        </dependency>
     </dependencies>
     <build>
         <resources>

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/d6ec142d/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 fb8b805..cef29fe 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
@@ -43,6 +43,7 @@ public class MRRunningJobMain {
             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();

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/d6ec142d/eagle-jpm/eagle-jpm-mr-running/src/main/java/org/apache/eagle/jpm/mr/running/entities/JPMEntityRepository.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-mr-running/src/main/java/org/apache/eagle/jpm/mr/running/entities/JPMEntityRepository.java b/eagle-jpm/eagle-jpm-mr-running/src/main/java/org/apache/eagle/jpm/mr/running/entities/JPMEntityRepository.java
deleted file mode 100644
index 6bb11ad..0000000
--- a/eagle-jpm/eagle-jpm-mr-running/src/main/java/org/apache/eagle/jpm/mr/running/entities/JPMEntityRepository.java
+++ /dev/null
@@ -1,32 +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.entities;
-
-import org.apache.eagle.jpm.util.jobcounter.JobCounters;
-import org.apache.eagle.jpm.util.jobcounter.JobCountersSerDeser;
-import org.apache.eagle.log.entity.repo.EntityRepository;
-
-public class JPMEntityRepository extends EntityRepository {
-    public JPMEntityRepository() {
-        entitySet.add(JobExecutionAPIEntity.class);
-        entitySet.add(TaskExecutionAPIEntity.class);
-        entitySet.add(TaskAttemptExecutionAPIEntity.class);
-
-        serDeserMap.put(JobCounters.class, new JobCountersSerDeser());
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/d6ec142d/eagle-jpm/eagle-jpm-mr-running/src/main/java/org/apache/eagle/jpm/mr/running/entities/JobConfig.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-mr-running/src/main/java/org/apache/eagle/jpm/mr/running/entities/JobConfig.java b/eagle-jpm/eagle-jpm-mr-running/src/main/java/org/apache/eagle/jpm/mr/running/entities/JobConfig.java
deleted file mode 100644
index 20f9cf4..0000000
--- a/eagle-jpm/eagle-jpm-mr-running/src/main/java/org/apache/eagle/jpm/mr/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.mr.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/d6ec142d/eagle-jpm/eagle-jpm-mr-running/src/main/java/org/apache/eagle/jpm/mr/running/entities/JobExecutionAPIEntity.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-mr-running/src/main/java/org/apache/eagle/jpm/mr/running/entities/JobExecutionAPIEntity.java b/eagle-jpm/eagle-jpm-mr-running/src/main/java/org/apache/eagle/jpm/mr/running/entities/JobExecutionAPIEntity.java
deleted file mode 100644
index 5fe7d5a..0000000
--- a/eagle-jpm/eagle-jpm-mr-running/src/main/java/org/apache/eagle/jpm/mr/running/entities/JobExecutionAPIEntity.java
+++ /dev/null
@@ -1,437 +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.entities;
-
-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.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("eagleMRRunningJobs")
-@ColumnFamily("f")
-@Prefix("jobs")
-@Service(Constants.JPA_RUNNING_JOB_EXECUTION_SERVICE_NAME)
-@TimeSeries(true)
-@Partition({"site"})
-@Indexes({
-        @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")
-    private long startTime;
-    @Column("b")
-    private long endTime;
-    @Column("c")
-    private long elapsedTime;
-    @Column("d")
-    private String status;
-    @Column("e")
-    private int mapsTotal;
-    @Column("f")
-    private int mapsCompleted;
-    @Column("g")
-    private int reducesTotal;
-    @Column("h")
-    private int reducesCompleted;
-    @Column("i")
-    private double mapProgress;
-    @Column("j")
-    private double reduceProgress;
-    @Column("k")
-    private int mapsPending;
-    @Column("l")
-    private int mapsRunning;
-    @Column("m")
-    private int reducesPending;
-    @Column("n")
-    private int reducesRunning;
-    @Column("o")
-    private int newReduceAttempts;
-    @Column("p")
-    private int runningReduceAttempts;
-    @Column("q")
-    private int failedReduceAttempts;
-    @Column("r")
-    private int killedReduceAttempts;
-    @Column("s")
-    private int successfulReduceAttempts;
-    @Column("t")
-    private int newMapAttempts;
-    @Column("u")
-    private int runningMapAttempts;
-    @Column("v")
-    private int failedMapAttempts;
-    @Column("w")
-    private int killedMapAttempts;
-    @Column("x")
-    private int successfulMapAttempts;
-    @Column("y")
-    private AppInfo appInfo;
-    @Column("z")
-    private JobCounters jobCounters;
-    @Column("aa")
-    private JobConfig jobConfig;
-    @Column("ab")
-    private long allocatedMB;
-    @Column("ac")
-    private int allocatedVCores;
-    @Column("ad")
-    private int runningContainers;
-    @Column("ae")
-    private int dataLocalMaps;
-    @Column("af")
-    private double dataLocalMapsPercentage;
-    @Column("ag")
-    private int rackLocalMaps;
-    @Column("ah")
-    private double rackLocalMapsPercentage;
-    @Column("ai")
-    private int totalLaunchedMaps;
-    @Column("aj")
-    private long submissionTime;
-
-    public JobConfig getJobConfig() {
-        return jobConfig;
-    }
-
-    public void setJobConfig(JobConfig jobConfig) {
-        this.jobConfig = jobConfig;
-        valueChanged("jobConfig");
-    }
-
-    public JobCounters getJobCounters() {
-        return jobCounters;
-    }
-
-    public void setJobCounters(JobCounters jobCounters) {
-        this.jobCounters = jobCounters;
-        valueChanged("jobCounters");
-    }
-
-    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;
-        valueChanged("endTime");
-    }
-
-    public long getElapsedTime() {
-        return elapsedTime;
-    }
-
-    public void setElapsedTime(long elapsedTime) {
-        this.elapsedTime = elapsedTime;
-        valueChanged("elapsedTime");
-    }
-
-    public String getStatus() {
-        return status;
-    }
-
-    public void setStatus(String status) {
-        this.status = status;
-        valueChanged("status");
-    }
-
-    public int getMapsTotal() {
-        return mapsTotal;
-    }
-
-    public void setMapsTotal(int mapsTotal) {
-        this.mapsTotal = mapsTotal;
-        valueChanged("mapsTotal");
-    }
-
-    public int getMapsCompleted() {
-        return mapsCompleted;
-    }
-
-    public void setMapsCompleted(int mapsCompleted) {
-        this.mapsCompleted = mapsCompleted;
-        valueChanged("mapsCompleted");
-    }
-
-    public int getReducesTotal() {
-        return reducesTotal;
-    }
-
-    public void setReducesTotal(int reducesTotal) {
-        this.reducesTotal = reducesTotal;
-        valueChanged("reducesTotal");
-    }
-
-    public int getReducesCompleted() {
-        return reducesCompleted;
-    }
-
-    public void setReducesCompleted(int reducesCompleted) {
-        this.reducesCompleted = reducesCompleted;
-        valueChanged("reducesCompleted");
-    }
-
-    public double getMapProgress() {
-        return mapProgress;
-    }
-
-    public void setMapProgress(double mapProgress) {
-        this.mapProgress = mapProgress;
-        valueChanged("mapProgress");
-    }
-
-    public double getReduceProgress() {
-        return reduceProgress;
-    }
-
-    public void setReduceProgress(double reduceProgress) {
-        this.reduceProgress = reduceProgress;
-        valueChanged("reduceProgress");
-    }
-
-    public int getMapsPending() {
-        return mapsPending;
-    }
-
-    public void setMapsPending(int mapsPending) {
-        this.mapsPending = mapsPending;
-        valueChanged("mapsPending");
-    }
-
-    public int getMapsRunning() {
-        return mapsRunning;
-    }
-
-    public void setMapsRunning(int mapsRunning) {
-        this.mapsRunning = mapsRunning;
-        valueChanged("mapsRunning");
-    }
-
-    public int getReducesPending() {
-        return reducesPending;
-    }
-
-    public void setReducesPending(int reducesPending) {
-        this.reducesPending = reducesPending;
-        valueChanged("reducesPending");
-    }
-
-    public int getReducesRunning() {
-        return reducesRunning;
-    }
-
-    public void setReducesRunning(int reducesRunning) {
-        this.reducesRunning = reducesRunning;
-        valueChanged("reducesRunning");
-    }
-
-    public int getNewReduceAttempts() {
-        return newReduceAttempts;
-    }
-
-    public void setNewReduceAttempts(int newReduceAttempts) {
-        this.newReduceAttempts = newReduceAttempts;
-        valueChanged("newReduceAttempts");
-    }
-
-    public int getRunningReduceAttempts() {
-        return runningReduceAttempts;
-    }
-
-    public void setRunningReduceAttempts(int runningReduceAttempts) {
-        this.runningReduceAttempts = runningReduceAttempts;
-        valueChanged("runningReduceAttempts");
-    }
-
-    public int getFailedReduceAttempts() {
-        return failedReduceAttempts;
-    }
-
-    public void setFailedReduceAttempts(int failedReduceAttempts) {
-        this.failedReduceAttempts = failedReduceAttempts;
-        valueChanged("failedReduceAttempts");
-    }
-
-    public int getKilledReduceAttempts() {
-        return killedReduceAttempts;
-    }
-
-    public void setKilledReduceAttempts(int killedReduceAttempts) {
-        this.killedReduceAttempts = killedReduceAttempts;
-        valueChanged("killedReduceAttempts");
-    }
-
-    public int getSuccessfulReduceAttempts() {
-        return successfulReduceAttempts;
-    }
-
-    public void setSuccessfulReduceAttempts(int successfulReduceAttempts) {
-        this.successfulReduceAttempts = successfulReduceAttempts;
-        valueChanged("successfulReduceAttempts");
-    }
-
-    public int getNewMapAttempts() {
-        return newMapAttempts;
-    }
-
-    public void setNewMapAttempts(int newMapAttempts) {
-        this.newMapAttempts = newMapAttempts;
-        valueChanged("newMapAttempts");
-    }
-
-    public int getRunningMapAttempts() {
-        return runningMapAttempts;
-    }
-
-    public void setRunningMapAttempts(int runningMapAttempts) {
-        this.runningMapAttempts = runningMapAttempts;
-        valueChanged("runningMapAttempts");
-    }
-
-    public int getFailedMapAttempts() {
-        return failedMapAttempts;
-    }
-
-    public void setFailedMapAttempts(int failedMapAttempts) {
-        this.failedMapAttempts = failedMapAttempts;
-        valueChanged("failedMapAttempts");
-    }
-
-    public int getKilledMapAttempts() {
-        return killedMapAttempts;
-    }
-
-    public void setKilledMapAttempts(int killedMapAttempts) {
-        this.killedMapAttempts = killedMapAttempts;
-        valueChanged("killedMapAttempts");
-    }
-
-    public int getSuccessfulMapAttempts() {
-        return successfulMapAttempts;
-    }
-
-    public void setSuccessfulMapAttempts(int successfulMapAttempts) {
-        this.successfulMapAttempts = successfulMapAttempts;
-        valueChanged("successfulMapAttempts");
-    }
-
-    public AppInfo getAppInfo() {
-        return appInfo;
-    }
-
-    public void setAppInfo(AppInfo appInfo) {
-        this.appInfo = appInfo;
-        valueChanged("appInfo");
-    }
-
-    public long getAllocatedMB() {
-        return allocatedMB;
-    }
-
-    public void setAllocatedMB(long allocatedMB) {
-        this.allocatedMB = allocatedMB;
-        valueChanged("allocatedMB");
-    }
-
-    public int getAllocatedVCores() {
-        return allocatedVCores;
-    }
-
-    public void setAllocatedVCores(int allocatedVCores) {
-        this.allocatedVCores = allocatedVCores;
-        valueChanged("allocatedVCores");
-    }
-
-    public int getRunningContainers() {
-        return runningContainers;
-    }
-
-    public void setRunningContainers(int runningContainers) {
-        this.runningContainers = runningContainers;
-        valueChanged("runningContainers");
-    }
-
-    public int getDataLocalMaps() {
-        return dataLocalMaps;
-    }
-
-    public void setDataLocalMaps(int dataLocalMaps) {
-        this.dataLocalMaps = dataLocalMaps;
-        valueChanged("dataLocalMaps");
-    }
-
-    public double getDataLocalMapsPercentage() {
-        return dataLocalMapsPercentage;
-    }
-
-    public void setDataLocalMapsPercentage(double dataLocalMapsPercentage) {
-        this.dataLocalMapsPercentage = dataLocalMapsPercentage;
-        valueChanged("dataLocalMapsPercentage");
-    }
-
-    public int getRackLocalMaps() {
-        return rackLocalMaps;
-    }
-
-    public void setRackLocalMaps(int rackLocalMaps) {
-        this.rackLocalMaps = rackLocalMaps;
-        valueChanged("rackLocalMaps");
-    }
-
-    public double getRackLocalMapsPercentage() {
-        return rackLocalMapsPercentage;
-    }
-
-    public void setRackLocalMapsPercentage(double rackLocalMapsPercentage) {
-        this.rackLocalMapsPercentage = rackLocalMapsPercentage;
-        valueChanged("rackLocalMapsPercentage");
-    }
-
-    public int getTotalLaunchedMaps() {
-        return totalLaunchedMaps;
-    }
-
-    public void setTotalLaunchedMaps(int totalLaunchedMaps) {
-        this.totalLaunchedMaps = totalLaunchedMaps;
-        valueChanged("totalLaunchedMaps");
-    }
-
-    public long getSubmissionTime() {
-        return submissionTime;
-    }
-
-    public void setSubmissionTime(long submissionTime) {
-        this.submissionTime = submissionTime;
-        valueChanged("submissionTime");
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/d6ec142d/eagle-jpm/eagle-jpm-mr-running/src/main/java/org/apache/eagle/jpm/mr/running/entities/TaskAttemptExecutionAPIEntity.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-mr-running/src/main/java/org/apache/eagle/jpm/mr/running/entities/TaskAttemptExecutionAPIEntity.java b/eagle-jpm/eagle-jpm-mr-running/src/main/java/org/apache/eagle/jpm/mr/running/entities/TaskAttemptExecutionAPIEntity.java
deleted file mode 100644
index c74563b..0000000
--- a/eagle-jpm/eagle-jpm-mr-running/src/main/java/org/apache/eagle/jpm/mr/running/entities/TaskAttemptExecutionAPIEntity.java
+++ /dev/null
@@ -1,135 +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.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.map.annotate.JsonSerialize;
-
-@JsonSerialize(include=JsonSerialize.Inclusion.NON_NULL)
-@Table("eagleMRRunningTasks")
-@ColumnFamily("f")
-@Prefix("tasks_exec_attempt")
-@Service(Constants.JPA_RUNNING_TASK_ATTEMPT_EXECUTION_SERVICE_NAME)
-@TimeSeries(true)
-@Partition({"site"})
-
-@Tags({"site", "jobId", "JobName", "jobDefId", "jobType", "taskType", "taskId", "user", "queue", "host", "rack"})
-public class TaskAttemptExecutionAPIEntity extends TaggedLogAPIEntity {
-    @Column("a")
-    private long startTime;
-    @Column("b")
-    private long finishTime;
-    @Column("c")
-    private long elapsedTime;
-    @Column("d")
-    private double progress;
-    @Column("e")
-    private String id;
-    @Column("f")
-    private String status;
-    @Column("g")
-    private String diagnostics;
-    @Column("h")
-    private String type;
-    @Column("i")
-    private String assignedContainerId;
-
-    public long getStartTime() {
-        return startTime;
-    }
-
-    public void setStartTime(long startTime) {
-        this.startTime = startTime;
-        valueChanged("startTime");
-    }
-
-    public long getFinishTime() {
-        return finishTime;
-    }
-
-    public void setFinishTime(long finishTime) {
-        this.finishTime = finishTime;
-        valueChanged("finishTime");
-    }
-
-    public long getElapsedTime() {
-        return elapsedTime;
-    }
-
-    public void setElapsedTime(long elapsedTime) {
-        this.elapsedTime = elapsedTime;
-        valueChanged("elapsedTime");
-    }
-
-    public double getProgress() {
-        return progress;
-    }
-
-    public void setProgress(double progress) {
-        this.progress = progress;
-        valueChanged("progress");
-    }
-
-    public String getId() {
-        return id;
-    }
-
-    public void setId(String id) {
-        this.id = id;
-        valueChanged("id");
-    }
-
-    public String getStatus() {
-        return status;
-    }
-
-    public void setStatus(String status) {
-        this.status = status;
-        valueChanged("status");
-    }
-
-    public String getDiagnostics() {
-        return diagnostics;
-    }
-
-    public void setDiagnostics(String diagnostics) {
-        this.diagnostics = diagnostics;
-        valueChanged("diagnostics");
-    }
-
-    public String getType() {
-        return type;
-    }
-
-    public void setType(String type) {
-        this.type = type;
-        valueChanged("type");
-    }
-
-    public String getAssignedContainerId() {
-        return assignedContainerId;
-    }
-
-    public void setAssignedContainerId(String assignedContainerId) {
-        this.assignedContainerId = assignedContainerId;
-        valueChanged("assignedContainerId");
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/d6ec142d/eagle-jpm/eagle-jpm-mr-running/src/main/java/org/apache/eagle/jpm/mr/running/entities/TaskExecutionAPIEntity.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-mr-running/src/main/java/org/apache/eagle/jpm/mr/running/entities/TaskExecutionAPIEntity.java b/eagle-jpm/eagle-jpm-mr-running/src/main/java/org/apache/eagle/jpm/mr/running/entities/TaskExecutionAPIEntity.java
deleted file mode 100644
index 7677f45..0000000
--- a/eagle-jpm/eagle-jpm-mr-running/src/main/java/org/apache/eagle/jpm/mr/running/entities/TaskExecutionAPIEntity.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.mr.running.entities;
-
-import org.apache.eagle.jpm.util.Constants;
-import org.apache.eagle.jpm.util.jobcounter.JobCounters;
-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("eagleMRRunningTasks")
-@ColumnFamily("f")
-@Prefix("tasks_exec")
-@Service(Constants.JPA_RUNNING_TASK_EXECUTION_SERVICE_NAME)
-@TimeSeries(true)
-@Partition({"site"})
-
-@Tags({"site", "jobId", "JobName", "jobDefId", "jobType", "taskType", "taskId", "user", "queue"})
-public class TaskExecutionAPIEntity extends TaggedLogAPIEntity {
-    @Column("a")
-    private long startTime;
-    @Column("b")
-    private long finishTime;
-    @Column("c")
-    private long elapsedTime;
-    @Column("d")
-    private double progress;
-    @Column("e")
-    private String status;
-    @Column("f")
-    private String successfulAttempt;
-    @Column("g")
-    private String statusDesc;
-    @Column("h")
-    private JobCounters jobCounters;
-    @Column("i")
-    private String host;
-
-    public long getStartTime() {
-        return startTime;
-    }
-
-    public void setStartTime(long startTime) {
-        this.startTime = startTime;
-        valueChanged("startTime");
-    }
-
-    public long getFinishTime() {
-        return finishTime;
-    }
-
-    public void setFinishTime(long finishTime) {
-        this.finishTime = finishTime;
-        valueChanged("finishTime");
-    }
-
-    public long getElapsedTime() {
-        return elapsedTime;
-    }
-
-    public void setElapsedTime(long elapsedTime) {
-        this.elapsedTime = elapsedTime;
-        valueChanged("elapsedTime");
-    }
-
-    public double getProgress() {
-        return progress;
-    }
-
-    public void setProgress(double progress) {
-        this.progress = progress;
-        valueChanged("progress");
-    }
-
-    public String getStatus() {
-        return status;
-    }
-
-    public void setStatus(String status) {
-        this.status = status;
-        valueChanged("status");
-    }
-
-    public String getSuccessfulAttempt() {
-        return successfulAttempt;
-    }
-
-    public void setSuccessfulAttempt(String successfulAttempt) {
-        this.successfulAttempt = successfulAttempt;
-        valueChanged("successfulAttempt");
-    }
-
-    public String getStatusDesc() {
-        return statusDesc;
-    }
-
-    public void setStatusDesc(String statusDesc) {
-        this.statusDesc = statusDesc;
-        valueChanged("statusDesc");
-    }
-
-    public JobCounters getJobCounters() {
-        return jobCounters;
-    }
-
-    public void setJobCounters(JobCounters jobCounters) {
-        this.jobCounters = jobCounters;
-        valueChanged("jobCounters");
-    }
-
-    public String getHost() {
-        return host;
-    }
-
-    public void setHost(String host) {
-        this.host = host;
-        valueChanged("host");
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/d6ec142d/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 aae0dd8..0dc53c3 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
@@ -19,11 +19,10 @@
 package org.apache.eagle.jpm.mr.running.parser;
 
 import org.apache.eagle.jpm.mr.running.config.MRRunningConfigManager;
-import org.apache.eagle.jpm.mr.running.entities.JobExecutionAPIEntity;
-import org.apache.eagle.jpm.mr.running.entities.TaskExecutionAPIEntity;
 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.util.Utils;
+import org.apache.eagle.jpm.mr.runningentity.JobExecutionAPIEntity;
+import org.apache.eagle.jpm.mr.runningentity.TaskExecutionAPIEntity;
 import org.apache.eagle.log.base.taggedlog.TaggedLogAPIEntity;
 import org.apache.eagle.log.entity.GenericMetricEntity;
 import org.apache.eagle.service.client.IEagleServiceClient;
@@ -32,8 +31,6 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import java.util.ArrayList;
-import java.util.Deque;
-import java.util.LinkedList;
 import java.util.List;
 
 public class MRJobEntityCreationHandler {
@@ -53,12 +50,16 @@ public class MRJobEntityCreationHandler {
     public void add(TaggedLogAPIEntity entity) {
         entities.add(entity);
         List<GenericMetricEntity> metricEntities;
-        if (entity instanceof TaskExecutionAPIEntity) {
+        /*if (entity instanceof TaskExecutionAPIEntity) {
             metricEntities = taskMetricsListener.generateMetrics((TaskExecutionAPIEntity) entity);
             entities.addAll(metricEntities);
         } else if (entity instanceof JobExecutionAPIEntity) {
             metricEntities = jobMetricsListener.generateMetrics((JobExecutionAPIEntity) entity);
             entities.addAll(metricEntities);
+        }*/
+        if (entity instanceof JobExecutionAPIEntity) {
+            metricEntities = jobMetricsListener.generateMetrics((JobExecutionAPIEntity) entity);
+            entities.addAll(metricEntities);
         }
         if (entities.size() >= eagleServiceConfig.maxFlushNum) {
             this.flush();


[4/4] incubator-eagle git commit: [EAGLE-467] Job list apis for querying jobs regardless of the status

Posted by qi...@apache.org.
[EAGLE-467] Job list apis for querying jobs regardless of the status

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

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

Closes #348 from qingwen220/jobAPI.


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

Branch: refs/heads/develop
Commit: d6ec142d3b1a6de28787e5fcda94f8f28cf02a3e
Parents: acee5cb
Author: Qingwen Zhao <qi...@gmail.com>
Authored: Fri Aug 19 17:46:42 2016 +0800
Committer: Qingwen Zhao <qi...@gmail.com>
Committed: Fri Aug 19 17:49:50 2016 +0800

----------------------------------------------------------------------
 .../eagle/jpm/entity/JPMEntityRepository.java   |  30 --
 .../org/apache/eagle/jpm/entity/JobConfig.java  |  38 --
 .../org/apache/eagle/jpm/entity/SparkApp.java   | 428 ------------------
 .../apache/eagle/jpm/entity/SparkExecutor.java  | 233 ----------
 .../org/apache/eagle/jpm/entity/SparkJob.java   | 178 --------
 .../org/apache/eagle/jpm/entity/SparkStage.java | 299 -------------
 .../org/apache/eagle/jpm/entity/SparkTask.java  | 290 ------------
 .../mr/historyentity/JPAEntityRepository.java   |  40 ++
 .../jpm/mr/historyentity/JobBaseAPIEntity.java  |  24 +
 .../eagle/jpm/mr/historyentity/JobConfig.java   |  38 ++
 .../jpm/mr/historyentity/JobConfigSerDeser.java |  62 +++
 .../JobConfigurationAPIEntity.java              |  66 +++
 .../jpm/mr/historyentity/JobEventAPIEntity.java |  44 ++
 .../mr/historyentity/JobExecutionAPIEntity.java | 230 ++++++++++
 .../JobProcessTimeStampEntity.java              |  44 ++
 .../TaskAttemptCounterAPIEntity.java            |  61 +++
 .../TaskAttemptExecutionAPIEntity.java          | 101 +++++
 .../historyentity/TaskExecutionAPIEntity.java   |  92 ++++
 .../TaskFailureCountAPIEntity.java              |  67 +++
 .../mr/runningentity/JPMEntityRepository.java   |  33 ++
 .../eagle/jpm/mr/runningentity/JobConfig.java   |  26 ++
 .../jpm/mr/runningentity/JobConfigSerDeser.java |  46 ++
 .../mr/runningentity/JobExecutionAPIEntity.java | 437 +++++++++++++++++++
 .../TaskAttemptExecutionAPIEntity.java          | 137 ++++++
 .../runningentity/TaskExecutionAPIEntity.java   | 127 ++++++
 .../jpm/spark/crawl/JHFSparkEventReader.java    |   6 +-
 .../jpm/spark/entity/JPMEntityRepository.java   |  32 ++
 .../eagle/jpm/spark/entity/JobConfig.java       |  39 ++
 .../jpm/spark/entity/JobConfigSerDeser.java     |  46 ++
 .../apache/eagle/jpm/spark/entity/SparkApp.java | 429 ++++++++++++++++++
 .../eagle/jpm/spark/entity/SparkExecutor.java   | 234 ++++++++++
 .../apache/eagle/jpm/spark/entity/SparkJob.java | 179 ++++++++
 .../eagle/jpm/spark/entity/SparkStage.java      | 300 +++++++++++++
 .../eagle/jpm/spark/entity/SparkTask.java       | 291 ++++++++++++
 eagle-jpm/eagle-jpm-mr-history/pom.xml          |   5 +
 .../eagle/jpm/mr/history/MRHistoryJobMain.java  |   5 +-
 .../crawler/JobHistoryContentFilter.java        |   6 +-
 .../crawler/JobHistoryContentFilterBuilder.java |   8 +
 .../crawler/JobHistoryContentFilterImpl.java    |  11 +
 .../history/entities/JPAEntityRepository.java   |  40 --
 .../mr/history/entities/JobBaseAPIEntity.java   |  24 -
 .../jpm/mr/history/entities/JobConfig.java      |  38 --
 .../mr/history/entities/JobConfigSerDeser.java  |  62 ---
 .../entities/JobConfigurationAPIEntity.java     |  66 ---
 .../mr/history/entities/JobEventAPIEntity.java  |  44 --
 .../history/entities/JobExecutionAPIEntity.java | 220 ----------
 .../entities/JobProcessTimeStampEntity.java     |  44 --
 .../entities/TaskAttemptCounterAPIEntity.java   |  61 ---
 .../entities/TaskAttemptExecutionAPIEntity.java | 101 -----
 .../entities/TaskExecutionAPIEntity.java        |  89 ----
 .../entities/TaskFailureCountAPIEntity.java     |  67 ---
 .../HistoryJobEntityCreationListener.java       |   2 +-
 .../HistoryJobEntityLifecycleListener.java      |   2 +-
 .../mr/history/parser/JHFEventReaderBase.java   |  14 +-
 .../mr/history/parser/JHFMRVer1EventReader.java |   2 +-
 ...JobConfigurationCreationServiceListener.java |   4 +-
 .../JobEntityCreationEagleServiceListener.java  |   2 +-
 .../parser/JobEntityCreationPublisher.java      |   2 +-
 .../parser/JobEntityLifecycleAggregator.java    |   6 +-
 .../parser/TaskAttemptCounterListener.java      |   6 +-
 .../mr/history/parser/TaskFailureListener.java  |   6 +-
 .../jpm/mr/history/storm/JobHistorySpout.java   |   2 +-
 .../src/main/resources/application.conf         |  35 +-
 eagle-jpm/eagle-jpm-mr-running/pom.xml          |   5 +
 .../eagle/jpm/mr/running/MRRunningJobMain.java  |   1 +
 .../running/entities/JPMEntityRepository.java   |  32 --
 .../jpm/mr/running/entities/JobConfig.java      |  25 --
 .../running/entities/JobExecutionAPIEntity.java | 437 -------------------
 .../entities/TaskAttemptExecutionAPIEntity.java | 135 ------
 .../entities/TaskExecutionAPIEntity.java        | 136 ------
 .../parser/MRJobEntityCreationHandler.java      |  13 +-
 .../jpm/mr/running/parser/MRJobParser.java      |  37 +-
 .../JobExecutionMetricsCreationListener.java    |   4 +-
 .../TaskExecutionMetricsCreationListener.java   |   6 +-
 .../mr/running/recover/MRRunningJobManager.java |   2 +-
 .../running/storm/MRRunningJobFetchSpout.java   |   2 +-
 .../mr/running/storm/MRRunningJobParseBolt.java |   2 +-
 .../src/main/resources/application.conf         |   1 +
 eagle-jpm/eagle-jpm-service/pom.xml             |  48 ++
 .../service/jpm/MRJobExecutionResource.java     | 306 +++++++++++++
 .../service/jpm/MRJobTaskGroupResponse.java     |  41 ++
 .../service/jpm/TestMRJobExecutionResource.java |  99 +++++
 eagle-jpm/eagle-jpm-spark-history/pom.xml       | 199 +++++----
 eagle-jpm/eagle-jpm-util/pom.xml                |  20 -
 .../org/apache/eagle/jpm/util/Constants.java    |   7 +-
 .../java/org/apache/eagle/jpm/util/Utils.java   |   3 +-
 .../util/resourceFetch/RMResourceFetcher.java   | 118 ++---
 .../resourceFetch/ha/HAURLSelectorImpl.java     |   4 +-
 eagle-jpm/pom.xml                               |   1 +
 eagle-security/eagle-security-hive/pom.xml      | 154 +++----
 90 files changed, 4091 insertions(+), 3448 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/d6ec142d/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/entity/JPMEntityRepository.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/entity/JPMEntityRepository.java b/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/entity/JPMEntityRepository.java
deleted file mode 100644
index f54688b..0000000
--- a/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/entity/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.entity;
-
-import org.apache.eagle.log.entity.repo.EntityRepository;
-
-public class JPMEntityRepository extends EntityRepository {
-    public JPMEntityRepository() {
-        entitySet.add(SparkApp.class);
-        entitySet.add(SparkJob.class);
-        entitySet.add(SparkStage.class);
-        entitySet.add(SparkTask.class);
-        entitySet.add(SparkExecutor.class);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/d6ec142d/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/entity/JobConfig.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/entity/JobConfig.java b/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/entity/JobConfig.java
deleted file mode 100644
index de3bd7a..0000000
--- a/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/entity/JobConfig.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.entity;
-
-import java.io.Serializable;
-import java.util.Map;
-import java.util.TreeMap;
-
-public class JobConfig implements Serializable {
-    private Map<String, String> config = new TreeMap<>();
-
-    public Map<String, String> getConfig() {
-        return config;
-    }
-
-    public void setConfig(Map<String, String> config) {
-        this.config = config;
-    }
-    @Override
-    public String toString(){
-        return config.toString();
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/d6ec142d/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/entity/SparkApp.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/entity/SparkApp.java b/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/entity/SparkApp.java
deleted file mode 100644
index 1760753..0000000
--- a/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/entity/SparkApp.java
+++ /dev/null
@@ -1,428 +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.entity;
-
-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")
-@Prefix("sprkapp")
-@Service(Constants.SPARK_APP_SERVICE_ENDPOINT_NAME)
-@JsonIgnoreProperties(ignoreUnknown = true)
-@TimeSeries(true)
-@Tags({"site","sprkAppId", "sprkAppAttemptId", "sprkAppName", "normSprkAppName","user", "queue"})
-@Partition({"site"})
-public class SparkApp 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;
-
-    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/d6ec142d/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/entity/SparkExecutor.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/entity/SparkExecutor.java b/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/entity/SparkExecutor.java
deleted file mode 100644
index 92cb130..0000000
--- a/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/entity/SparkExecutor.java
+++ /dev/null
@@ -1,233 +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.entity;
-
-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")
-@Prefix("sprkexcutr")
-@Service(Constants.SPARK_EXECUTOR_SERVICE_ENDPOINT_NAME)
-@JsonIgnoreProperties(ignoreUnknown = true)
-@TimeSeries(true)
-@Tags({"site","sprkAppId", "sprkAppAttemptId", "sprkAppName", "normSprkAppName", "executorId","user", "queue"})
-@Partition({"site"})
-public class SparkExecutor 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/d6ec142d/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/entity/SparkJob.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/entity/SparkJob.java b/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/entity/SparkJob.java
deleted file mode 100644
index a641440..0000000
--- a/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/entity/SparkJob.java
+++ /dev/null
@@ -1,178 +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.entity;
-
-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")
-@Prefix("sprkjob")
-@Service(Constants.SPARK_JOB_SERVICE_ENDPOINT_NAME)
-@JsonIgnoreProperties(ignoreUnknown = true)
-@TimeSeries(true)
-@Tags({"site","sprkAppId", "sprkAppAttemptId", "sprkAppName", "normSprkAppName", "jobId","user", "queue"})
-@Partition({"site"})
-public class SparkJob 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;
-
-    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/d6ec142d/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/entity/SparkStage.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/entity/SparkStage.java b/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/entity/SparkStage.java
deleted file mode 100644
index 92714bf..0000000
--- a/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/entity/SparkStage.java
+++ /dev/null
@@ -1,299 +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.entity;
-
-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")
-@Prefix("sprkstage")
-@Service(Constants.SPARK_STAGE_SERVICE_ENDPOINT_NAME)
-@JsonIgnoreProperties(ignoreUnknown = true)
-@TimeSeries(true)
-@Tags({"site","sprkAppId", "sprkAppAttemptId", "sprkAppName", "normSprkAppName", "jobId", "stageId","stageAttemptId","user", "queue"})
-@Partition({"site"})
-public class SparkStage 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/d6ec142d/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/entity/SparkTask.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/entity/SparkTask.java b/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/entity/SparkTask.java
deleted file mode 100644
index af9ed21..0000000
--- a/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/entity/SparkTask.java
+++ /dev/null
@@ -1,290 +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.entity;
-
-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")
-@Prefix("sprktask")
-@Service(Constants.SPARK_TASK_SERVICE_ENDPOINT_NAME)
-@JsonIgnoreProperties(ignoreUnknown = true)
-@TimeSeries(true)
-@Tags({"site","sprkAppId", "sprkAppAttemptId", "sprkAppName", "normSprkAppName", "jobId", "jobName", "stageId","stageAttemptId","taskIndex","taskAttemptId","user", "queue"})
-@Partition({"site"})
-public class SparkTask 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/d6ec142d/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
new file mode 100644
index 0000000..49095ed
--- /dev/null
+++ b/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/mr/historyentity/JPAEntityRepository.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.mr.historyentity;
+
+import org.apache.eagle.jpm.util.jobcounter.JobCounters;
+import org.apache.eagle.jpm.util.jobcounter.JobCountersSerDeser;
+import org.apache.eagle.log.entity.repo.EntityRepository;
+
+public class JPAEntityRepository extends EntityRepository {
+
+    public JPAEntityRepository() {
+        serDeserMap.put(JobCounters.class, new JobCountersSerDeser());
+        serDeserMap.put(JobConfig.class, new JobConfigSerDeser());
+        entitySet.add(JobConfigurationAPIEntity.class);
+        entitySet.add(JobEventAPIEntity.class);
+        entitySet.add(JobExecutionAPIEntity.class);
+
+        entitySet.add(TaskAttemptExecutionAPIEntity.class);
+        entitySet.add(TaskExecutionAPIEntity.class);
+        entitySet.add(TaskFailureCountAPIEntity.class);
+        entitySet.add(TaskAttemptCounterAPIEntity.class);
+        entitySet.add(JobProcessTimeStampEntity.class);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/d6ec142d/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/mr/historyentity/JobBaseAPIEntity.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/mr/historyentity/JobBaseAPIEntity.java b/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/mr/historyentity/JobBaseAPIEntity.java
new file mode 100644
index 0000000..a49fb3d
--- /dev/null
+++ b/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/mr/historyentity/JobBaseAPIEntity.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.mr.historyentity;
+
+import org.apache.eagle.log.base.taggedlog.TaggedLogAPIEntity;
+
+public class JobBaseAPIEntity extends TaggedLogAPIEntity {
+}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/d6ec142d/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
new file mode 100644
index 0000000..97ebd50
--- /dev/null
+++ b/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/mr/historyentity/JobConfig.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.mr.historyentity;
+
+import java.util.Map;
+import java.util.TreeMap;
+
+public final class JobConfig {
+    private Map<String, String> config = new TreeMap<>();
+
+    public Map<String, String> getConfig() {
+        return config;
+    }
+
+    public void setConfig(Map<String, String> config) {
+        this.config = config;
+    }
+    
+    public String toString(){
+        return config.toString();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/d6ec142d/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
new file mode 100644
index 0000000..5af4377
--- /dev/null
+++ b/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/mr/historyentity/JobConfigSerDeser.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.mr.historyentity;
+
+import org.apache.eagle.log.entity.meta.EntitySerDeser;
+import org.apache.hadoop.hbase.util.Bytes;
+
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.TreeMap;
+
+public class JobConfigSerDeser implements EntitySerDeser<JobConfig> {
+
+    @Override
+    public JobConfig deserialize(byte[] bytes) {
+        JobConfig jc = new JobConfig();
+        Map<String, String> map = new TreeMap<String, String>();
+        jc.setConfig(map);
+        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 key = str[0];
+            String value = "";
+            if (str.length == 2) value = str[1];
+            map.put(key, value);
+        }
+        return jc;
+    }
+    
+    @Override
+    public byte[] serialize(JobConfig conf) {
+        Map<String, String> map = conf.getConfig();
+        StringBuilder sb = new StringBuilder();
+        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(){
+        return JobConfig.class;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/d6ec142d/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
new file mode 100644
index 0000000..3a09c5f
--- /dev/null
+++ b/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/mr/historyentity/JobConfigurationAPIEntity.java
@@ -0,0 +1,66 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.entity.meta.*;
+import org.codehaus.jackson.map.annotate.JsonSerialize;
+
+@JsonSerialize(include=JsonSerialize.Inclusion.NON_NULL)
+@Table("eaglejpa")
+@ColumnFamily("f")
+@Prefix("jconf")
+@Service(Constants.JPA_JOB_CONFIG_SERVICE_NAME)
+@TimeSeries(true)
+@Partition({"site"})
+@Indexes({
+        @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")
+    private String configJobName;
+    @Column("b")
+    private JobConfig jobConfig;
+    @Column("c")
+    private String alertEmailList;
+    
+    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/d6ec142d/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
new file mode 100644
index 0000000..b289a9c
--- /dev/null
+++ b/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/mr/historyentity/JobEventAPIEntity.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.mr.historyentity;
+
+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)
+@Table("eaglejpa")
+@ColumnFamily("f")
+@Prefix("jevent")
+@Service(Constants.JPA_JOB_EVENT_SERVICE_NAME)
+@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/d6ec142d/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
new file mode 100644
index 0000000..db8f0d6
--- /dev/null
+++ b/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/mr/historyentity/JobExecutionAPIEntity.java
@@ -0,0 +1,230 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.jpm.util.jobcounter.JobCounters;
+import org.apache.eagle.log.entity.meta.*;
+import org.codehaus.jackson.map.annotate.JsonSerialize;
+
+@JsonSerialize(include=JsonSerialize.Inclusion.NON_NULL)
+@Table("eaglejpa")
+@ColumnFamily("f")
+@Prefix("jexec")
+@Service(Constants.JPA_JOB_EXECUTION_SERVICE_NAME)
+@TimeSeries(true)
+@Partition({"site"})
+@Indexes({
+    @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")
+    private String currentState;
+    @Column("b")
+    private long startTime;
+    @Column("c")
+    private long endTime;
+    @Column("d")
+    private int numTotalMaps;
+    @Column("e")
+    private int numFailedMaps;
+    @Column("f")
+    private int numFinishedMaps;
+    @Column("g")
+    private int numTotalReduces;
+    @Column("h")
+    private int numFailedReduces;
+    @Column("i")
+    private int numFinishedReduces;
+    @Column("j")
+    private JobCounters jobCounters;
+    @Column("k")
+    private int dataLocalMaps;
+    @Column("l")
+    private double dataLocalMapsPercentage;
+    @Column("m")
+    private int rackLocalMaps;
+    @Column("n")
+    private double rackLocalMapsPercentage;
+    @Column("o")
+    private int totalLaunchedMaps;
+    @Column("p")
+    private long submissionTime;
+    @Column("q")
+    private long lastMapDuration;
+    @Column("r")
+    private long lastReduceDuration;
+    @Column("s")
+    private long durationTime;
+
+    public long getDurationTime() {
+        return durationTime;
+    }
+    public void setDurationTime(long durationTime) {
+        this.durationTime = durationTime;
+        valueChanged("durationTime");
+    }
+
+    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;
+    }
+
+    public void setNumFinishedReduces(int numFinishedReduces) {
+        this.numFinishedReduces = numFinishedReduces;
+        _pcs.firePropertyChange("numFinishedReduces", null, null);
+    }
+
+    public JobCounters getJobCounters() {
+        return jobCounters;
+    }
+
+    public void setJobCounters(JobCounters jobCounters) {
+        this.jobCounters = jobCounters;
+        _pcs.firePropertyChange("jobCounters", null, null);
+    }
+
+    public int getDataLocalMaps() {
+        return dataLocalMaps;
+    }
+
+    public void setDataLocalMaps(int dataLocalMaps) {
+        this.dataLocalMaps = dataLocalMaps;
+        valueChanged("dataLocalMaps");
+    }
+
+    public double getDataLocalMapsPercentage() {
+        return dataLocalMapsPercentage;
+    }
+
+    public void setDataLocalMapsPercentage(double dataLocalMapsPercentage) {
+        this.dataLocalMapsPercentage = dataLocalMapsPercentage;
+        valueChanged("dataLocalMapsPercentage");
+    }
+
+    public int getRackLocalMaps() {
+        return rackLocalMaps;
+    }
+
+    public void setRackLocalMaps(int rackLocalMaps) {
+        this.rackLocalMaps = rackLocalMaps;
+        valueChanged("rackLocalMaps");
+    }
+
+    public double getRackLocalMapsPercentage() {
+        return rackLocalMapsPercentage;
+    }
+
+    public void setRackLocalMapsPercentage(double rackLocalMapsPercentage) {
+        this.rackLocalMapsPercentage = rackLocalMapsPercentage;
+        valueChanged("rackLocalMapsPercentage");
+    }
+
+    public int getTotalLaunchedMaps() {
+        return totalLaunchedMaps;
+    }
+
+    public void setTotalLaunchedMaps(int totalLaunchedMaps) {
+        this.totalLaunchedMaps = totalLaunchedMaps;
+        valueChanged("totalLaunchedMaps");
+    }
+
+    public long getSubmissionTime() {
+        return submissionTime;
+    }
+
+    public void setSubmissionTime(long submissionTime) {
+        this.submissionTime = submissionTime;
+        valueChanged("submissionTime");
+    }
+
+    public long getLastMapDuration() {
+        return lastMapDuration;
+    }
+
+    public void setLastMapDuration(long lastMapDuration) {
+        this.lastMapDuration = lastMapDuration;
+        valueChanged("lastMapDuration");
+    }
+
+    public long getLastReduceDuration() {
+        return lastReduceDuration;
+    }
+
+    public void setLastReduceDuration(long lastReduceDuration) {
+        this.lastReduceDuration = lastReduceDuration;
+        valueChanged("lastReduceDuration");
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/d6ec142d/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
new file mode 100644
index 0000000..df57657
--- /dev/null
+++ b/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/mr/historyentity/JobProcessTimeStampEntity.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.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("eaglejpa_process")
+@ColumnFamily("f")
+@Prefix("process")
+@Service(Constants.JPA_JOB_PROCESS_TIME_STAMP_NAME)
+@TimeSeries(true)
+@Partition({"site"})
+public class JobProcessTimeStampEntity extends TaggedLogAPIEntity {
+    @Column("a")
+    private long currentTimeStamp;
+
+    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/d6ec142d/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
new file mode 100644
index 0000000..89272bf
--- /dev/null
+++ b/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/mr/historyentity/TaskAttemptCounterAPIEntity.java
@@ -0,0 +1,61 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.entity.meta.*;
+import org.codehaus.jackson.map.annotate.JsonSerialize;
+
+@JsonSerialize(include=JsonSerialize.Inclusion.NON_NULL)
+@Table("eaglejpa_anomaly")
+@ColumnFamily("f")
+@Prefix("tacount")
+@Service(Constants.JPA_TASK_ATTEMPT_COUNTER_SERVICE_NAME)
+@TimeSeries(true)
+@Partition({"site"})
+public class TaskAttemptCounterAPIEntity extends JobBaseAPIEntity {
+    @Column("a")
+    private int totalCount;
+    @Column("b")
+    private int failedCount;
+    @Column("c")
+    private int killedCount;
+    
+    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/d6ec142d/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
new file mode 100644
index 0000000..be5566b
--- /dev/null
+++ b/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/mr/historyentity/TaskAttemptExecutionAPIEntity.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.mr.historyentity;
+
+import org.apache.eagle.jpm.util.Constants;
+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)
+@Table("eaglejpa_task")
+@ColumnFamily("f")
+@Prefix("taexec")
+@Service(Constants.JPA_TASK_ATTEMPT_EXECUTION_SERVICE_NAME)
+@TimeSeries(true)
+@Partition({"site"})
+@Indexes({
+    @Index(name="Index_1_jobId", columns = { "jobID" }, unique = false)
+    })
+public class TaskAttemptExecutionAPIEntity extends JobBaseAPIEntity {
+    @Column("a")
+    private String taskStatus;
+    @Column("b")
+    private long startTime;
+    @Column("c")
+    private long endTime;
+    @Column("d")
+    private long duration;
+    @Column("e")
+    private String error;
+    @Column("f")
+    private JobCounters jobCounters;
+    @Column("g")
+    private String taskAttemptID;
+
+    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/d6ec142d/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
new file mode 100644
index 0000000..9de8b05
--- /dev/null
+++ b/eagle-jpm/eagle-jpm-entity/src/main/java/org/apache/eagle/jpm/mr/historyentity/TaskExecutionAPIEntity.java
@@ -0,0 +1,92 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+*/
+
+package org.apache.eagle.jpm.mr.historyentity;
+
+import org.apache.eagle.jpm.util.Constants;
+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)
+@Table("eaglejpa_task")
+@ColumnFamily("f")
+@Prefix("texec")
+@Service(Constants.JPA_TASK_EXECUTION_SERVICE_NAME)
+@TimeSeries(true)
+@Partition({"site"})
+@Indexes({
+        @Index(name="Index_1_jobId", columns = { "jobId" }, unique = false)
+})
+public class TaskExecutionAPIEntity extends JobBaseAPIEntity {
+    @Column("a")
+    private String taskStatus;
+    @Column("b")
+    private long startTime;
+    @Column("c")
+    private long endTime;
+    @Column("d")
+    private long duration;
+    @Column("e")
+    private String error;
+    @Column("f")
+    private JobCounters jobCounters;
+
+    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);
+    }
+}