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);
+ }
+}