diff --git a/.gitignore b/.gitignore
index 9e140102f..f6b110e89 100644
--- a/.gitignore
+++ b/.gitignore
@@ -1,3 +1,19 @@
+#
+# Copyright 2016 LinkedIn Corp.
+#
+# Licensed under the Apache License, Version 2.0 (the "License"); you may not
+# use this file except in compliance with the License. You may obtain a copy of
+# the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+# WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+# License for the specific language governing permissions and limitations under
+# the License.
+#
+
# general
*~
*.log
@@ -32,5 +48,13 @@ target
logs
RUNNING_PID
+# macOS
+.DS_Store
*.jar
+
+#web
+public/assets/ember/
+public/assets/fonts/
+web/bower_components/
+web/node_modules/
diff --git a/.travis.yml b/.travis.yml
new file mode 100644
index 000000000..1f2e642ad
--- /dev/null
+++ b/.travis.yml
@@ -0,0 +1,11 @@
+language: scala
+sudo: false
+jdk:
+ - oraclejdk8
+ - oraclejdk7
+
+# only build PRs and master (not all branch pushes)
+branches:
+ only:
+ - master
+
diff --git a/LICENSE b/LICENSE
new file mode 100644
index 000000000..7a4a3ea24
--- /dev/null
+++ b/LICENSE
@@ -0,0 +1,202 @@
+
+ Apache License
+ Version 2.0, January 2004
+ http://www.apache.org/licenses/
+
+ TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION
+
+ 1. Definitions.
+
+ "License" shall mean the terms and conditions for use, reproduction,
+ and distribution as defined by Sections 1 through 9 of this document.
+
+ "Licensor" shall mean the copyright owner or entity authorized by
+ the copyright owner that is granting the License.
+
+ "Legal Entity" shall mean the union of the acting entity and all
+ other entities that control, are controlled by, or are under common
+ control with that entity. For the purposes of this definition,
+ "control" means (i) the power, direct or indirect, to cause the
+ direction or management of such entity, whether by contract or
+ otherwise, or (ii) ownership of fifty percent (50%) or more of the
+ outstanding shares, or (iii) beneficial ownership of such entity.
+
+ "You" (or "Your") shall mean an individual or Legal Entity
+ exercising permissions granted by this License.
+
+ "Source" form shall mean the preferred form for making modifications,
+ including but not limited to software source code, documentation
+ source, and configuration files.
+
+ "Object" form shall mean any form resulting from mechanical
+ transformation or translation of a Source form, including but
+ not limited to compiled object code, generated documentation,
+ and conversions to other media types.
+
+ "Work" shall mean the work of authorship, whether in Source or
+ Object form, made available under the License, as indicated by a
+ copyright notice that is included in or attached to the work
+ (an example is provided in the Appendix below).
+
+ "Derivative Works" shall mean any work, whether in Source or Object
+ form, that is based on (or derived from) the Work and for which the
+ editorial revisions, annotations, elaborations, or other modifications
+ represent, as a whole, an original work of authorship. For the purposes
+ of this License, Derivative Works shall not include works that remain
+ separable from, or merely link (or bind by name) to the interfaces of,
+ the Work and Derivative Works thereof.
+
+ "Contribution" shall mean any work of authorship, including
+ the original version of the Work and any modifications or additions
+ to that Work or Derivative Works thereof, that is intentionally
+ submitted to Licensor for inclusion in the Work by the copyright owner
+ or by an individual or Legal Entity authorized to submit on behalf of
+ the copyright owner. For the purposes of this definition, "submitted"
+ means any form of electronic, verbal, or written communication sent
+ to the Licensor or its representatives, including but not limited to
+ communication on electronic mailing lists, source code control systems,
+ and issue tracking systems that are managed by, or on behalf of, the
+ Licensor for the purpose of discussing and improving the Work, but
+ excluding communication that is conspicuously marked or otherwise
+ designated in writing by the copyright owner as "Not a Contribution."
+
+ "Contributor" shall mean Licensor and any individual or Legal Entity
+ on behalf of whom a Contribution has been received by Licensor and
+ subsequently incorporated within the Work.
+
+ 2. Grant of Copyright License. Subject to the terms and conditions of
+ this License, each Contributor hereby grants to You a perpetual,
+ worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+ copyright license to reproduce, prepare Derivative Works of,
+ publicly display, publicly perform, sublicense, and distribute the
+ Work and such Derivative Works in Source or Object form.
+
+ 3. Grant of Patent License. Subject to the terms and conditions of
+ this License, each Contributor hereby grants to You a perpetual,
+ worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+ (except as stated in this section) patent license to make, have made,
+ use, offer to sell, sell, import, and otherwise transfer the Work,
+ where such license applies only to those patent claims licensable
+ by such Contributor that are necessarily infringed by their
+ Contribution(s) alone or by combination of their Contribution(s)
+ with the Work to which such Contribution(s) was submitted. If You
+ institute patent litigation against any entity (including a
+ cross-claim or counterclaim in a lawsuit) alleging that the Work
+ or a Contribution incorporated within the Work constitutes direct
+ or contributory patent infringement, then any patent licenses
+ granted to You under this License for that Work shall terminate
+ as of the date such litigation is filed.
+
+ 4. Redistribution. You may reproduce and distribute copies of the
+ Work or Derivative Works thereof in any medium, with or without
+ modifications, and in Source or Object form, provided that You
+ meet the following conditions:
+
+ (a) You must give any other recipients of the Work or
+ Derivative Works a copy of this License; and
+
+ (b) You must cause any modified files to carry prominent notices
+ stating that You changed the files; and
+
+ (c) You must retain, in the Source form of any Derivative Works
+ that You distribute, all copyright, patent, trademark, and
+ attribution notices from the Source form of the Work,
+ excluding those notices that do not pertain to any part of
+ the Derivative Works; and
+
+ (d) If the Work includes a "NOTICE" text file as part of its
+ distribution, then any Derivative Works that You distribute must
+ include a readable copy of the attribution notices contained
+ within such NOTICE file, excluding those notices that do not
+ pertain to any part of the Derivative Works, in at least one
+ of the following places: within a NOTICE text file distributed
+ as part of the Derivative Works; within the Source form or
+ documentation, if provided along with the Derivative Works; or,
+ within a display generated by the Derivative Works, if and
+ wherever such third-party notices normally appear. The contents
+ of the NOTICE file are for informational purposes only and
+ do not modify the License. You may add Your own attribution
+ notices within Derivative Works that You distribute, alongside
+ or as an addendum to the NOTICE text from the Work, provided
+ that such additional attribution notices cannot be construed
+ as modifying the License.
+
+ You may add Your own copyright statement to Your modifications and
+ may provide additional or different license terms and conditions
+ for use, reproduction, or distribution of Your modifications, or
+ for any such Derivative Works as a whole, provided Your use,
+ reproduction, and distribution of the Work otherwise complies with
+ the conditions stated in this License.
+
+ 5. Submission of Contributions. Unless You explicitly state otherwise,
+ any Contribution intentionally submitted for inclusion in the Work
+ by You to the Licensor shall be under the terms and conditions of
+ this License, without any additional terms or conditions.
+ Notwithstanding the above, nothing herein shall supersede or modify
+ the terms of any separate license agreement you may have executed
+ with Licensor regarding such Contributions.
+
+ 6. Trademarks. This License does not grant permission to use the trade
+ names, trademarks, service marks, or product names of the Licensor,
+ except as required for reasonable and customary use in describing the
+ origin of the Work and reproducing the content of the NOTICE file.
+
+ 7. Disclaimer of Warranty. Unless required by applicable law or
+ agreed to in writing, Licensor provides the Work (and each
+ Contributor provides its Contributions) on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
+ implied, including, without limitation, any warranties or conditions
+ of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A
+ PARTICULAR PURPOSE. You are solely responsible for determining the
+ appropriateness of using or redistributing the Work and assume any
+ risks associated with Your exercise of permissions under this License.
+
+ 8. Limitation of Liability. In no event and under no legal theory,
+ whether in tort (including negligence), contract, or otherwise,
+ unless required by applicable law (such as deliberate and grossly
+ negligent acts) or agreed to in writing, shall any Contributor be
+ liable to You for damages, including any direct, indirect, special,
+ incidental, or consequential damages of any character arising as a
+ result of this License or out of the use or inability to use the
+ Work (including but not limited to damages for loss of goodwill,
+ work stoppage, computer failure or malfunction, or any and all
+ other commercial damages or losses), even if such Contributor
+ has been advised of the possibility of such damages.
+
+ 9. Accepting Warranty or Additional Liability. While redistributing
+ the Work or Derivative Works thereof, You may choose to offer,
+ and charge a fee for, acceptance of support, warranty, indemnity,
+ or other liability obligations and/or rights consistent with this
+ License. However, in accepting such obligations, You may act only
+ on Your own behalf and on Your sole responsibility, not on behalf
+ of any other Contributor, and only if You agree to indemnify,
+ defend, and hold each Contributor harmless for any liability
+ incurred by, or claims asserted against, such Contributor by reason
+ of your accepting any such warranty or additional liability.
+
+ END OF TERMS AND CONDITIONS
+
+ APPENDIX: How to apply the Apache License to your work.
+
+ To apply the Apache License to your work, attach the following
+ boilerplate notice, with the fields enclosed by brackets "[]"
+ replaced with your own identifying information. (Don't include
+ the brackets!) The text should be enclosed in the appropriate
+ comment syntax for the file format. We also recommend that a
+ file or class name and description of purpose be included on the
+ same "printed page" as the copyright notice for easier
+ identification within third-party archives.
+
+ Copyright [yyyy] [name of copyright owner]
+
+ Licensed under the Apache License, Version 2.0 (the "License");
+ you may not use this file except in compliance with the License.
+ You may obtain a copy of the License at
+
+ http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
\ No newline at end of file
diff --git a/NOTICE b/NOTICE
new file mode 100644
index 000000000..fc2cea466
--- /dev/null
+++ b/NOTICE
@@ -0,0 +1,115 @@
+Copyright 2016 LinkedIn Corp.
+
+Licensed under the Apache License, Version 2.0 (the "License"); you may not
+use this file except in compliance with the License. You may obtain a copy of
+the License at
+
+http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+License for the specific language governing permissions and limitations under
+the License.
+------------------------------------------------------------------------------
+
+This product automatically loads third party code from an external repository(configurable) using Play's build system.
+Such third party code is subject to other license terms than as set forth above. Please review the complete list of
+dependencies for applicable license terms.
+
+In addition, such third party code may also depend on and load multiple tiers of dependencies. Please review the
+applicable licenses of the additional dependencies.
+
+------------------------------------------------------------------------------
+
+This product includes software developed at
+The Apache Software Foundation (http://www.apache.org/)
+
+This product includes/uses Apache Commons IO (https://commons.apache.org/proper/commons-io/)
+Copyright (c) 2002-2016 The Apache Software Foundation
+License: Apache 2.0
+
+This product includes/uses Gson (https://github.com/google/gson)
+Copyright (C) 2008 Google Inc.
+License: Apache 2.0
+
+This product includes/uses Jsoup (https://github.com/jhy/jsoup)
+Copyright (c) 2009 - 2015 Jonathan Hedley (jonathan@hedley.net)
+License: The MIT License (https://github.com/jhy/jsoup/blob/master/LICENSE)
+
+This product includes/uses Guava (https://github.com/google/guava)
+License: Apache 2.0
+
+This product includes/uses Jackson Databind (http://github.com/FasterXML/jackson)
+License: Apache 2.0
+
+This product includes/uses Apache Hadoop (http://hadoop.apache.org/)
+Notice: https://github.com/apache/hadoop/blob/trunk/NOTICE.txt
+License: Apache 2.0
+
+This product includes/uses Apache Spark (http://spark.apache.org/)
+Notice: https://github.com/apache/spark/blob/master/NOTICE
+License: Apache 2.0 (https://github.com/apache/spark/blob/master/LICENSE)
+
+This product includes/uses Play Framework (https://www.playframework.com/)
+Copyright (C) 2009-2016 Typesafe Inc. (http://www.typesafe.com)
+License: Apache 2.0
+
+This product requires the following play dependencies in addition to the core Play library
+javaEbean, the Ebean plugin for Java.
+javaJdbc, the Java database API.
+
+This product includes/uses JMockit (http://jmockit.org/)
+Notice: https://github.com/jmockit/jmockit1/blob/master/NOTICE.txt
+License: The MIT License (https://github.com/jmockit/jmockit1/blob/master/LICENSE.txt)
+
+This product includes/uses Mockito (http://mockito.org)
+Copyright (c) 2007 Mockito contributors
+License: The MIT License (https://github.com/mockito/mockito/blob/master/LICENSE)
+
+This product includes/uses Metrics (http://metrics.dropwizard.io/)
+Copyright 2010-2014, Coda Hale, Yammer Inc.. Created using Sphinx 1.4.1.
+License: Apache 2.0 (https://github.com/dropwizard/metrics/blob/master/LICENSE)
+
+This product includes/uses ember-cli (https://ember-cli.com/)
+Copyright (c) 2013-2016 Stefan Penner, Robert Jackson and ember-cli contributors
+License: The MIT License (MIT) (https://github.com/ember-cli/ember-cli/blob/master/LICENSE)
+
+This product includes/uses bower (https://bower.io/)
+Copyright (c) 2016 Twitter and other contributors (https://github.com/bower/bower/blob/master/LICENSE)
+
+This product includes/uses broccoli-asset-rev
+Copyright (c) 2014 Rick Harrison
+The MIT License (MIT) (https://github.com/rickharrison/broccoli-asset-rev/blob/master/LICENSE)
+
+------------------------------------------------------------------------------
+Attribution for JavaScript Libraries
+------------------------------------------------------------------------------
+
+jQuery (http://jquery.com/)
+Copyright 2005, 2014 jQuery Foundation, Inc. and other contributors
+License: MIT License (http://jquery.org/license)
+
+jQuery Deserialize (https://github.com/kflorence/jquery-deserialize/)
+Copyright (c) 2015 Kyle Florence
+License: Dual licensed under the MIT and GPLv2 licenses.
+
+Bootstrap (http://getbootstrap.com/)
+Copyright 2011-2016 Twitter, Inc.
+License: MIT License (https://github.com/twbs/bootstrap/blob/master/LICENSE)
+
+Bootstrap Datepicker (https://github.com/eternicode/bootstrap-datepicker)
+Copyright 2012 Stefan Petre, Improvements by Andrew Rowls
+License: Apache 2.0
+
+D3.js (http://d3js.org)
+Copyright (c) 2010-2016, Michael Bostock
+License: BSD 3-Clause (https://github.com/mbostock/d3/blob/master/LICENSE)
+
+ember.js ( http://emberjs.com/)
+Copyright (c) 2016 Yehuda Katz, Tom Dale and Ember.js contributors
+License: (https://github.com/emberjs/ember.js/blob/master/LICENSE)
+
+------------------------------------------------------------------------------
+
+
diff --git a/README.md b/README.md
index 172604f50..7283dbba7 100644
--- a/README.md
+++ b/README.md
@@ -1,61 +1,52 @@
-## Dr Elephant
+# Dr. Elephant
-### Compiling & testing locally
+[![Build Status](https://api.travis-ci.org/linkedin/dr-elephant.svg)](https://travis-ci.org/linkedin/dr-elephant/)
+[![Join the chat at https://gitter.im/linkedin/dr-elephant](https://badges.gitter.im/linkedin/dr-elephant.svg)](https://gitter.im/linkedin/dr-elephant?utm_source=badge&utm_medium=badge&utm_campaign=pr-badge&utm_content=badge)
-* To be able to build & run the application, download and install [Play framework 2.2.2](http://downloads.typesafe.com/play/2.2.2/play-2.2.2.zip).
-* The pre-installed play command on our boxes will not work as it is configured to look at LinkedIns repos
-* If this is your first time working with Dr. Elephant, take the deployed Hadoop jars and put them in the /lib directory:
- scp eat1-magicgw01.grid.linkedin.com:/export/apps/hadoop/latest/hadoop-core-1.2.1-p3.jar ./lib/.
+
-* To build and run the application in dev mode, run from command line "play run" in the project directory.
-* There is need to investigate the framework to see how one can add parameters to the classpath in dev mode.
+**Dr. Elephant** is a performance monitoring and tuning tool for Hadoop and Spark. It automatically gathers all the metrics, runs analysis on them, and presents them in a simple way for easy consumption. Its goal is to improve developer productivity and increase cluster efficiency by making it easier to tune the jobs. It analyzes the Hadoop and Spark jobs using a set of pluggable, configurable, rule-based heuristics that provide insights on how a job performed, and then uses the results to make suggestions about how to tune the job to make it perform more efficiently.
-### Deployment
+## Documentation
-* To create a deployment package, use "play dist" to create a zip package, or use "play universal:package-zip-tarball" to create a tarball
-* To run the deployed package with Hadoop properly, some changes needs to be added to the startup script located at ./bin/dr-elephant
+For more information on Dr. Elephant, check the wiki pages [here](https://github.com/linkedin/dr-elephant/wiki).
-* in the classpath ("declare -r app\_classpath=...") , add to the end of the string, before the end quotes
+For quick setup instructions: [Click here](https://github.com/linkedin/dr-elephant/wiki/Quick-Setup-Instructions)
- :$HADOOP_HOME/*:$HADOOP_HOME/lib/*:$HADOOP_HOME/conf
+Developer guide: [Click here](https://github.com/linkedin/dr-elephant/wiki/Developer-Guide)
-* after the next line ("addJava ... ;"), add new line
+Administrator guide: [Click here](https://github.com/linkedin/dr-elephant/wiki/Administrator-Guide)
- addJava "-Djava.library.path=$HADOOP_HOME/lib/native/Linux-amd64-64"
+User guide: [Click here](https://github.com/linkedin/dr-elephant/wiki/User-Guide)
-### New Deployment (All previous instructions are deprecated!)
+Engineering Blog: [Click here](https://engineering.linkedin.com/blog/2016/04/dr-elephant-open-source-self-serve-performance-tuning-hadoop-spark)
-* ./compile.sh will create two zips under 'dist' dir which can deploy with h1 and h2 directly without changing classpath
-* When test dr.e in hadoop2.x locally, HADOOP_HOME and HADOOP_CONF_DIR need to be set properly
-* Upon deployment on cluster, we can specify keytab and database location at runtime: ./bin/dr-elephant -Dhttp.port=xxxx -Dkeytab.user="xxxx" -Dkeytab.location="xxxx" -Ddb.default.url="jdbc:mysql://xxxx" -Ddb.default.user=xxxx -Ddb.default.password=xxxx so that we don't have to change application.conf at compile time
+## Mailing-list & Github Issues
+Google groups mailing list: [Click here](https://groups.google.com/forum/#!forum/dr-elephant-users)
+Github issues: [click here](https://github.com/linkedin/dr-elephant/issues)
-### DB Schema evolutions
+## Meetings
-When the schema in the model package changes, play will need to be ran to automatically apply the evolution.
+We have scheduled a weekly Dr. Elephant meeting for the interested developers and users to discuss future plans for Dr. Elephant. Please [click here](https://github.com/linkedin/dr-elephant/issues/209) for details.
-* There is a problem with Ebean where it does not support something like @Index to generate indices for columns of interest
-* So what we did to work around this is to manually add indices into the sql script.
-* To do this, we needed to prevent the automatically generated sql to overwrite our modified sql.
-* The evolution sql file must be changed (by moving or removing the header "To stop Ebean DDL generation, remove this comment and start using Evolutions") to make sure it does not automatically generate new sql.
-* To re-create the sql file from a new schema in code:
- * Backup the file at ./conf/evolutions/default/1.sql
- * Remove the file
- * Run play in debug mode and browse the page. This causes EBean to generate the new sql file, and automatically apply the evolution.
- * Copy over the indices from the old 1.sql file
- * Remove the header in the sql file so it does not get overwritten
- * Browse the page again to refresh the schema to add the indices.
+## How to Contribute?
-### Running on the cluster
+Check this [link](https://github.com/linkedin/dr-elephant/wiki/How-to-Contribute%3F).
-* SSH into the machine
-* sudo as elephant
-* go to /export/apps/elephant/
-* To start: ./run.sh
-* To kill: ./kill.sh
-* To deploy new version:
- * scp machine:location-to-drelephant.zip /export/apps/elephant/
- * ./kill.sh
- * unzip dr-elephant-0.1-SNAPSHOT.zip
- * ./run.sh
+## License
+
+ Copyright 2016 LinkedIn Corp.
+
+ Licensed under the Apache License, Version 2.0 (the "License"); you may not
+ use this file except in compliance with the License. You may obtain a copy of
+ the License at
+
+ http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ License for the specific language governing permissions and limitations under
+ the License.
diff --git a/app-conf/AggregatorConf.xml b/app-conf/AggregatorConf.xml
new file mode 100644
index 000000000..23586d587
--- /dev/null
+++ b/app-conf/AggregatorConf.xml
@@ -0,0 +1,43 @@
+
+
+
+
+
+
+ mapreduce
+ com.linkedin.drelephant.mapreduce.MapReduceMetricsAggregator
+
+
+ spark
+ com.linkedin.drelephant.spark.SparkMetricsAggregator
+
+ 0.5
+
+
+
diff --git a/app-conf/FetcherConf.xml b/app-conf/FetcherConf.xml
new file mode 100644
index 000000000..d06ce8bf7
--- /dev/null
+++ b/app-conf/FetcherConf.xml
@@ -0,0 +1,68 @@
+
+
+
+
+
+
+ mapreduce
+ com.linkedin.drelephant.mapreduce.fetchers.MapReduceFetcherHadoop2
+
+ false
+
+
+
+
+
+
+ spark
+ com.linkedin.drelephant.spark.fetchers.SparkFetcher
+
+
diff --git a/app-conf/GeneralConf.xml b/app-conf/GeneralConf.xml
new file mode 100644
index 000000000..944f10d0e
--- /dev/null
+++ b/app-conf/GeneralConf.xml
@@ -0,0 +1,48 @@
+
+
+
+
+
+
+ drelephant.analysis.thread.count
+ 3
+ Number of threads to analyze the completed jobs
+
+
+ drelephant.analysis.fetch.interval
+ 60000
+ Interval between fetches in milliseconds
+
+
+ drelephant.analysis.retry.interval
+ 60000
+ Interval between retries in milliseconds
+
+
+ drelephant.application.search.match.partial
+ true
+ If this property is "false", search will only make exact matches
+
+
+
+
diff --git a/app-conf/HeuristicConf.xml b/app-conf/HeuristicConf.xml
new file mode 100644
index 000000000..21a00168a
--- /dev/null
+++ b/app-conf/HeuristicConf.xml
@@ -0,0 +1,197 @@
+
+
+
+
+
+
+
+
+ mapreduce
+ Mapper Data Skew
+ com.linkedin.drelephant.mapreduce.heuristics.MapperDataSkewHeuristic
+ views.html.help.mapreduce.helpMapperDataSkew
+
+
+
+
+ mapreduce
+ Mapper GC
+ com.linkedin.drelephant.mapreduce.heuristics.MapperGCHeuristic
+ views.html.help.mapreduce.helpGC
+
+
+
+
+ mapreduce
+ Mapper Time
+ com.linkedin.drelephant.mapreduce.heuristics.MapperTimeHeuristic
+ views.html.help.mapreduce.helpMapperTime
+
+
+
+
+ mapreduce
+ Mapper Speed
+ com.linkedin.drelephant.mapreduce.heuristics.MapperSpeedHeuristic
+ views.html.help.mapreduce.helpMapperSpeed
+
+
+
+
+ mapreduce
+ Mapper Spill
+ com.linkedin.drelephant.mapreduce.heuristics.MapperSpillHeuristic
+ views.html.help.mapreduce.helpMapperSpill
+
+
+
+
+ mapreduce
+ Mapper Memory
+ com.linkedin.drelephant.mapreduce.heuristics.MapperMemoryHeuristic
+ views.html.help.mapreduce.helpMapperMemory
+
+
+
+
+ mapreduce
+ Reducer Data Skew
+ com.linkedin.drelephant.mapreduce.heuristics.ReducerDataSkewHeuristic
+ views.html.help.mapreduce.helpReducerDataSkew
+
+
+
+
+ mapreduce
+ Reducer GC
+ com.linkedin.drelephant.mapreduce.heuristics.ReducerGCHeuristic
+ views.html.help.mapreduce.helpGC
+
+
+
+
+ mapreduce
+ Reducer Time
+ com.linkedin.drelephant.mapreduce.heuristics.ReducerTimeHeuristic
+ views.html.help.mapreduce.helpReducerTime
+
+
+
+
+ mapreduce
+ Reducer Memory
+ com.linkedin.drelephant.mapreduce.heuristics.ReducerMemoryHeuristic
+ views.html.help.mapreduce.helpReducerMemory
+
+
+
+
+ mapreduce
+ Shuffle & Sort
+ com.linkedin.drelephant.mapreduce.heuristics.ShuffleSortHeuristic
+ views.html.help.mapreduce.helpShuffleSort
+
+
+
+
+ mapreduce
+ Exception
+ com.linkedin.drelephant.mapreduce.heuristics.ExceptionHeuristic
+ views.html.help.mapreduce.helpException
+
+
+
+ mapreduce
+ Distributed Cache Limit
+ com.linkedin.drelephant.mapreduce.heuristics.DistributedCacheLimitHeuristic
+ views.html.help.mapreduce.helpDistributedCacheLimit
+
+ 500000000
+
+
+
+
+
+
+
+ spark
+ Spark Configuration
+ com.linkedin.drelephant.spark.heuristics.ConfigurationHeuristic
+ views.html.help.spark.helpConfigurationHeuristic
+
+
+ spark
+ Spark Executor Metrics
+ com.linkedin.drelephant.spark.heuristics.ExecutorsHeuristic
+ views.html.help.spark.helpExecutorsHeuristic
+
+
+ spark
+ Spark Job Metrics
+ com.linkedin.drelephant.spark.heuristics.JobsHeuristic
+ views.html.help.spark.helpJobsHeuristic
+
+
+ spark
+ Spark Stage Metrics
+ com.linkedin.drelephant.spark.heuristics.StagesHeuristic
+ views.html.help.spark.helpStagesHeuristic
+
+
+
diff --git a/app-conf/JobTypeConf.xml b/app-conf/JobTypeConf.xml
new file mode 100644
index 000000000..8a4cae3eb
--- /dev/null
+++ b/app-conf/JobTypeConf.xml
@@ -0,0 +1,77 @@
+
+
+
+
+
+
+ Spark
+ spark
+ spark.app.id
+
+
+
+ Pig
+ mapreduce
+ pig.script
+
+
+ Hive
+ mapreduce
+ hive.mapred.mode
+
+
+ OozieLauncher
+ mapreduce
+ oozie.launcher.action.main.class
+
+
+ Cascading
+ mapreduce
+ cascading.app.frameworks
+
+
+ Voldemort
+ mapreduce
+ mapred.reducer.class
+ voldemort.store.readonly.mr.*
+
+
+ Kafka
+ mapreduce
+ kafka.url
+
+
+ HadoopJava
+ mapreduce
+ mapred.child.java.opts
+
+
+
diff --git a/app-conf/SchedulerConf.xml b/app-conf/SchedulerConf.xml
new file mode 100644
index 000000000..3bfd26cd7
--- /dev/null
+++ b/app-conf/SchedulerConf.xml
@@ -0,0 +1,72 @@
+
+
+
+
+
+
+ airflow
+ com.linkedin.drelephant.schedulers.AirflowScheduler
+
+ http://localhost:8000
+
+
+
+
+ azkaban
+ com.linkedin.drelephant.schedulers.AzkabanScheduler
+
+
+
+
+
+
+
+
+
+
+
+ oozie
+ com.linkedin.drelephant.schedulers.OozieScheduler
+
+
+ http://localhost:11000/oozie
+
+
+
+
+
+
+ no_scheduler
+ com.linkedin.drelephant.schedulers.NoScheduler
+
+
+
diff --git a/app-conf/elephant.conf b/app-conf/elephant.conf
new file mode 100644
index 000000000..2ab7852c2
--- /dev/null
+++ b/app-conf/elephant.conf
@@ -0,0 +1,55 @@
+# Play application server port
+port=8080
+
+# Secret key
+# The secret key is used to secure cryptographics functions.
+# If you deploy your application to several instances be sure to use the same key!
+# You can set the key using the env variable APPLICATION_SECRET or set it here
+# Setting it here has higher precedence than env variable.
+# application_secret="changeme"
+
+# Database configuration
+db_url=localhost
+db_name=drelephant
+db_user=root
+db_password=""
+
+# Enable web analytics for the application.
+# By default analytics is not turned on. Set this property
+# to true and paste the javascript snippet into 'public/analytics/track.js' for
+# enabling web analytics for the application. You may configure an analytics application
+# like piwik. More information on piwik at piwik.org
+enable_analytics=false
+
+# Set the keytab user and the path to the keytab file if security is enabled.
+# keytab_user=""
+# keytab_location=""
+
+# Additional Configuration
+# Check https://www.playframework.com/documentation/2.2.x/ProductionConfiguration
+jvm_args="-Devolutionplugin=enabled -DapplyEvolutions.default=true"
+
+# Property enables dropwizard metrics for the application.
+# More info on Dropwizard metrics at http://metrics.dropwizard.io
+# By default metrics are turned which provides several useful stats for
+# the application. The following endpoints can be queried once the application is up.
+# /ping
+# /healthcheck
+# /metrics
+metrics=true
+
+#
+# Property enables an agent jar to be loaded along with the Dr. Elephant application.
+# The intention with this property is to have the agent publish metrics to other
+# applications, although it can do anything else. This property is disabled by default
+# and users wishing to make use of it should provide their own implementation of the agent.
+#
+# More information on writing a jvm agent can be found under the following.
+# https://docs.oracle.com/javase/8/docs/api/java/lang/instrument/package-summary.html
+#
+# Sample configuration of this property is shown below.
+# metrics_agent_jar="-javaagent:lib/your_agent.jar"
+#
+# Sample configuration of the agent with additional options.
+# metrics_agent_jar="-javaagent:lib/your_agent.jar=app-name=dr-elephant,app-host=foo"
+
diff --git a/app/Global.java b/app/Global.java
index 02b04175d..9b346e0b5 100644
--- a/app/Global.java
+++ b/app/Global.java
@@ -1,3 +1,19 @@
+/*
+ * Copyright 2016 LinkedIn Corp.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License"); you may not
+ * use this file except in compliance with the License. You may obtain a copy of
+ * the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
import com.linkedin.drelephant.DrElephant;
import com.sun.security.sasl.util.AbstractSaslImpl;
@@ -11,27 +27,30 @@
import java.util.logging.Level;
+/**
+ * This class manages all the global settings
+ */
public class Global extends GlobalSettings {
- DrElephant drElephant;
+ DrElephant _drElephant;
public void onStart(Application app) {
- Logger.info("Application has started");
+ Logger.info("Starting Application...");
fixJavaKerberos();
try {
- drElephant = new DrElephant();
- drElephant.start();
+ _drElephant = new DrElephant();
+ _drElephant.start();
} catch (IOException e) {
Logger.error("Application start failed...", e);
}
}
public void onStop(Application app) {
- Logger.info("Application shutdown...");
- if (drElephant != null) {
- drElephant.kill();
+ Logger.info("Stopping application...");
+ if (_drElephant != null) {
+ _drElephant.kill();
}
}
diff --git a/app/com/linkedin/drelephant/DrElephant.java b/app/com/linkedin/drelephant/DrElephant.java
index c471029cf..f0de35a02 100644
--- a/app/com/linkedin/drelephant/DrElephant.java
+++ b/app/com/linkedin/drelephant/DrElephant.java
@@ -1,23 +1,42 @@
+/*
+ * Copyright 2016 LinkedIn Corp.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License"); you may not
+ * use this file except in compliance with the License. You may obtain a copy of
+ * the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
package com.linkedin.drelephant;
import java.io.IOException;
+/**
+ * The main class which starts Dr. Elephant
+ */
public class DrElephant extends Thread {
- private ElephantRunner elephant;
+ private ElephantRunner _elephant;
public DrElephant() throws IOException {
- elephant = new ElephantRunner();
+ _elephant = new ElephantRunner();
}
@Override
public void run() {
- elephant.run();
+ _elephant.run();
}
public void kill() {
- if (elephant != null) {
- elephant.kill();
+ if (_elephant != null) {
+ _elephant.kill();
}
}
}
diff --git a/app/com/linkedin/drelephant/ElephantAnalyser.java b/app/com/linkedin/drelephant/ElephantAnalyser.java
deleted file mode 100644
index 6ad090dfa..000000000
--- a/app/com/linkedin/drelephant/ElephantAnalyser.java
+++ /dev/null
@@ -1,83 +0,0 @@
-package com.linkedin.drelephant;
-
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Properties;
-
-import com.linkedin.drelephant.analysis.Heuristic;
-import com.linkedin.drelephant.analysis.HeuristicResult;
-import com.linkedin.drelephant.analysis.Severity;
-import com.linkedin.drelephant.analysis.heuristics.*;
-import com.linkedin.drelephant.hadoop.HadoopJobData;
-
-import model.JobType;
-
-public class ElephantAnalyser {
- public static final String NO_DATA = "No Data Received";
- private static final ElephantAnalyser instance = new ElephantAnalyser();
-
- private HeuristicResult nodata;
- private List heuristics = new ArrayList();
- public List heuristicNames = new ArrayList();
-
- public ElephantAnalyser() {
- nodata = new HeuristicResult(NO_DATA, Severity.LOW);
- addHeuristic(new MapperDataSkewHeuristic());
- addHeuristic(new ReducerDataSkewHeuristic());
- addHeuristic(new MapperInputSizeHeuristic());
- addHeuristic(new MapperSpeedHeuristic());
- addHeuristic(new ReducerTimeHeuristic());
- addHeuristic(new ShuffleSortHeuristic());
- }
-
- public void addHeuristic(Heuristic heuristic) {
- heuristics.add(heuristic);
- heuristicNames.add(heuristic.getHeuristicName());
- }
-
- public HeuristicResult[] analyse(HadoopJobData data) {
- if (data.getMapperData().length == 0 && data.getReducerData().length == 0) {
- return new HeuristicResult[]{nodata};
- }
-
- List results = new ArrayList();
- for (Heuristic heuristic : heuristics) {
- results.add(heuristic.apply(data));
- }
- return results.toArray(new HeuristicResult[results.size()]);
- }
-
- public JobType getJobType(HadoopJobData data) {
- String pigVersion = data.getJobConf().getProperty("pig.version");
- if (pigVersion != null && !pigVersion.isEmpty()) {
- return JobType.PIG;
- }
- String hiveMapredMode = data.getJobConf().getProperty("hive.mapred.mode");
- if (hiveMapredMode != null && !hiveMapredMode.isEmpty()) {
- return JobType.HIVE;
- }
-
- return JobType.HADOOPJAVA;
- }
-
-
- public Map getMetaUrls(HadoopJobData data) {
- Map result = new HashMap();
- final String prefix = "meta.url.";
- Properties jobConf = data.getJobConf();
- for (Map.Entry entry : jobConf.entrySet()) {
- if (entry.getKey().toString().startsWith(prefix)) {
- String key = entry.getKey().toString();
- String value = jobConf.getProperty(key);
- result.put(key.substring(prefix.length()), value);
- }
- }
- return result;
- }
-
- public static ElephantAnalyser instance() {
- return instance;
- }
-}
diff --git a/app/com/linkedin/drelephant/ElephantContext.java b/app/com/linkedin/drelephant/ElephantContext.java
new file mode 100644
index 000000000..fe6ff3026
--- /dev/null
+++ b/app/com/linkedin/drelephant/ElephantContext.java
@@ -0,0 +1,412 @@
+/*
+ * Copyright 2016 LinkedIn Corp.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License"); you may not
+ * use this file except in compliance with the License. You may obtain a copy of
+ * the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package com.linkedin.drelephant;
+
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.Sets;
+import com.linkedin.drelephant.analysis.ApplicationType;
+import com.linkedin.drelephant.analysis.ElephantFetcher;
+import com.linkedin.drelephant.analysis.HadoopApplicationData;
+import com.linkedin.drelephant.analysis.HadoopMetricsAggregator;
+import com.linkedin.drelephant.analysis.Heuristic;
+import com.linkedin.drelephant.analysis.HeuristicResult;
+import com.linkedin.drelephant.analysis.JobType;
+import com.linkedin.drelephant.configurations.aggregator.AggregatorConfiguration;
+import com.linkedin.drelephant.configurations.aggregator.AggregatorConfigurationData;
+import com.linkedin.drelephant.configurations.fetcher.FetcherConfiguration;
+import com.linkedin.drelephant.configurations.fetcher.FetcherConfigurationData;
+import com.linkedin.drelephant.configurations.heuristic.HeuristicConfiguration;
+import com.linkedin.drelephant.configurations.heuristic.HeuristicConfigurationData;
+import com.linkedin.drelephant.configurations.jobtype.JobTypeConfiguration;
+import com.linkedin.drelephant.mapreduce.MapReduceMetricsAggregator;
+import com.linkedin.drelephant.util.Utils;
+import java.lang.reflect.InvocationTargetException;
+import java.lang.reflect.Method;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.Set;
+
+import org.apache.commons.lang.StringUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.log4j.Logger;
+import org.w3c.dom.Document;
+import play.api.templates.Html;
+
+
+/**
+ * This is a general singleton instance that provides globally accessible resources.
+ *
+ * It is not mandatory that an AnalysisPromise implementation must leverage this instance, but this context provides
+ * a way for Promises to access shared objects (singletons, thread-local variables and etc.).
+ */
+public class ElephantContext {
+ private static final Logger logger = Logger.getLogger(ElephantContext.class);
+ private static ElephantContext INSTANCE;
+
+ private static final String AGGREGATORS_CONF = "AggregatorConf.xml";
+ private static final String FETCHERS_CONF = "FetcherConf.xml";
+ private static final String HEURISTICS_CONF = "HeuristicConf.xml";
+ private static final String JOB_TYPES_CONF = "JobTypeConf.xml";
+ private static final String GENERAL_CONF = "GeneralConf.xml";
+
+ private final Map> _heuristicGroupedNames = new HashMap>();
+ private List _heuristicsConfData;
+ private List _fetchersConfData;
+ private Configuration _generalConf;
+ private List _aggregatorConfData;
+
+ private final Map _nameToType = new HashMap();
+ private final Map> _typeToHeuristics = new HashMap>();
+ private final Map _typeToAggregator = new HashMap();
+ private final Map _typeToFetcher = new HashMap();
+ private final Map _heuristicToView = new HashMap();
+ private Map> _appTypeToJobTypes = new HashMap>();
+
+ public static void init() {
+ INSTANCE = new ElephantContext();
+ }
+
+ public static ElephantContext instance() {
+ if (INSTANCE == null) {
+ INSTANCE = new ElephantContext();
+ }
+ return INSTANCE;
+ }
+
+ // private on purpose
+ private ElephantContext() {
+ loadConfiguration();
+ }
+
+ private void loadConfiguration() {
+ loadAggregators();
+ loadFetchers();
+ loadHeuristics();
+ loadJobTypes();
+
+ loadGeneralConf();
+
+ // It is important to configure supported types in the LAST step so that we could have information from all
+ // configurable components.
+ configureSupportedApplicationTypes();
+ }
+
+
+ private void loadAggregators() {
+ Document document = Utils.loadXMLDoc(AGGREGATORS_CONF);
+
+ _aggregatorConfData = new AggregatorConfiguration(document.getDocumentElement()).getAggregatorsConfigurationData();
+ for (AggregatorConfigurationData data : _aggregatorConfData) {
+ try {
+ Class> aggregatorClass = Class.forName(data.getClassName());
+ Object instance = aggregatorClass.getConstructor(AggregatorConfigurationData.class).newInstance(data);
+ if (!(instance instanceof HadoopMetricsAggregator)) {
+ throw new IllegalArgumentException(
+ "Class " + aggregatorClass.getName() + " is not an implementation of " + HadoopMetricsAggregator.class.getName());
+ }
+
+ ApplicationType type = data.getAppType();
+ if (_typeToAggregator.get(type) == null) {
+ _typeToAggregator.put(type, (HadoopMetricsAggregator) instance);
+ }
+
+ logger.info("Load Aggregator : " + data.getClassName());
+ } catch (ClassNotFoundException e) {
+ throw new RuntimeException("Could not find class " + data.getClassName(), e);
+ } catch (InstantiationException e) {
+ throw new RuntimeException("Could not instantiate class " + data.getClassName(), e);
+ } catch (IllegalAccessException e) {
+ throw new RuntimeException("Could not access constructor for class" + data.getClassName(), e);
+ } catch (RuntimeException e) {
+ throw new RuntimeException(data.getClassName() + " is not a valid Aggregator class.", e);
+ } catch (InvocationTargetException e) {
+ throw new RuntimeException("Could not invoke class " + data.getClassName(), e);
+ } catch (NoSuchMethodException e) {
+ throw new RuntimeException("Could not find constructor for class " + data.getClassName(), e);
+ }
+ }
+
+ }
+ /**
+ * Load all the fetchers configured in FetcherConf.xml
+ */
+ private void loadFetchers() {
+ Document document = Utils.loadXMLDoc(FETCHERS_CONF);
+
+ _fetchersConfData = new FetcherConfiguration(document.getDocumentElement()).getFetchersConfigurationData();
+ for (FetcherConfigurationData data : _fetchersConfData) {
+ try {
+ Class> fetcherClass = Class.forName(data.getClassName());
+ Object instance = fetcherClass.getConstructor(FetcherConfigurationData.class).newInstance(data);
+ if (!(instance instanceof ElephantFetcher)) {
+ throw new IllegalArgumentException(
+ "Class " + fetcherClass.getName() + " is not an implementation of " + ElephantFetcher.class.getName());
+ }
+
+ ApplicationType type = data.getAppType();
+ if (_typeToFetcher.get(type) == null) {
+ _typeToFetcher.put(type, (ElephantFetcher) instance);
+ }
+
+ logger.info("Load Fetcher : " + data.getClassName());
+ } catch (ClassNotFoundException e) {
+ throw new RuntimeException("Could not find class " + data.getClassName(), e);
+ } catch (InstantiationException e) {
+ throw new RuntimeException("Could not instantiate class " + data.getClassName(), e);
+ } catch (IllegalAccessException e) {
+ throw new RuntimeException("Could not access constructor for class" + data.getClassName(), e);
+ } catch (RuntimeException e) {
+ throw new RuntimeException(data.getClassName() + " is not a valid Fetcher class.", e);
+ } catch (InvocationTargetException e) {
+ throw new RuntimeException("Could not invoke class " + data.getClassName(), e);
+ } catch (NoSuchMethodException e) {
+ throw new RuntimeException("Could not find constructor for class " + data.getClassName(), e);
+ }
+ }
+ }
+
+ /**
+ * Load all the heuristics and their views configured in HeuristicConf.xml
+ */
+ private void loadHeuristics() {
+ Document document = Utils.loadXMLDoc(HEURISTICS_CONF);
+
+ _heuristicsConfData = new HeuristicConfiguration(document.getDocumentElement()).getHeuristicsConfigurationData();
+ for (HeuristicConfigurationData data : _heuristicsConfData) {
+
+ // Load all the heuristic classes
+ try {
+ Class> heuristicClass = Class.forName(data.getClassName());
+
+ Object instance = heuristicClass.getConstructor(HeuristicConfigurationData.class).newInstance(data);
+ if (!(instance instanceof Heuristic)) {
+ throw new IllegalArgumentException(
+ "Class " + heuristicClass.getName() + " is not an implementation of " + Heuristic.class.getName());
+ }
+ ApplicationType type = data.getAppType();
+ List heuristics = _typeToHeuristics.get(type);
+ if (heuristics == null) {
+ heuristics = new ArrayList();
+ _typeToHeuristics.put(type, heuristics);
+ }
+ heuristics.add((Heuristic) instance);
+
+ logger.info("Load Heuristic : " + data.getClassName());
+ } catch (ClassNotFoundException e) {
+ throw new RuntimeException("Could not find class " + data.getClassName(), e);
+ } catch (InstantiationException e) {
+ throw new RuntimeException("Could not instantiate class " + data.getClassName(), e);
+ } catch (IllegalAccessException e) {
+ throw new RuntimeException("Could not access constructor for class" + data.getClassName(), e);
+ } catch (RuntimeException e) {
+ // More descriptive on other runtime exception such as ClassCastException
+ throw new RuntimeException(data.getClassName() + " is not a valid Heuristic class.", e);
+ } catch (InvocationTargetException e) {
+ throw new RuntimeException("Could not invoke class " + data.getClassName(), e);
+ } catch (NoSuchMethodException e) {
+ throw new RuntimeException("Could not find constructor for class " + data.getClassName(), e);
+ }
+
+ // Load all the heuristic views
+ try {
+ Class> viewClass = Class.forName(data.getViewName());
+
+ Method render = viewClass.getDeclaredMethod("render");
+ Html page = (Html) render.invoke(null);
+ _heuristicToView.put(data.getHeuristicName(), page);
+
+ logger.info("Load View : " + data.getViewName());
+ } catch (ClassNotFoundException e) {
+ throw new RuntimeException("Could not find view " + data.getViewName(), e);
+ } catch (IllegalAccessException e) {
+ throw new RuntimeException("Could not access render on view" + data.getViewName(), e);
+ } catch (RuntimeException e) {
+ // More descriptive on other runtime exception such as ClassCastException
+ throw new RuntimeException(data.getViewName() + " is not a valid view class.", e);
+ } catch (InvocationTargetException e) {
+ throw new RuntimeException("Could not invoke view " + data.getViewName(), e);
+ } catch (NoSuchMethodException e) {
+ throw new RuntimeException("Could not find method render for view " + data.getViewName(), e);
+ }
+ }
+
+ // Bind No_DATA heuristic to its helper pages, no need to add any real configurations
+ _heuristicsConfData.add(
+ new HeuristicConfigurationData(HeuristicResult.NO_DATA.getHeuristicName(),
+ HeuristicResult.NO_DATA.getHeuristicClassName(), "views.html.help.helpNoData", null, null));
+ }
+
+ /**
+ * Decides what application types can be supported.
+ *
+ * An application type is supported if all the below are true.
+ * 1. A Fetcher is defined in FetcherConf.xml for the application type.
+ * 2. At least one Heuristic is configured in HeuristicConf.xml for the application type.
+ * 3. At least one job type is configured in JobTypeConf.xml for the application type.
+ */
+ private void configureSupportedApplicationTypes() {
+ Set supportedTypes = Sets.intersection(_typeToFetcher.keySet(), _typeToHeuristics.keySet());
+ supportedTypes = Sets.intersection(supportedTypes, _appTypeToJobTypes.keySet());
+ supportedTypes = Sets.intersection(supportedTypes, _typeToAggregator.keySet());
+
+ _typeToAggregator.keySet().retainAll(supportedTypes);
+ _typeToFetcher.keySet().retainAll(supportedTypes);
+ _typeToHeuristics.keySet().retainAll(supportedTypes);
+ _appTypeToJobTypes.keySet().retainAll(supportedTypes);
+
+ logger.info("Configuring ElephantContext...");
+ for (ApplicationType type : supportedTypes) {
+ _nameToType.put(type.getName(), type);
+
+ List classes = new ArrayList();
+ List heuristics = _typeToHeuristics.get(type);
+ for (Heuristic heuristic : heuristics) {
+ classes.add(heuristic.getClass().getName());
+ }
+
+ List jobTypes = _appTypeToJobTypes.get(type);
+ logger.info("Supports " + type.getName() + " application type, using " + _typeToFetcher.get(type).toString()
+ + " fetcher class with Heuristics [" + StringUtils.join(classes, ", ") + "] and following JobTypes ["
+ + StringUtils.join(jobTypes, ", ") + "].");
+ }
+ }
+
+ /**
+ * Load all the job types configured in JobTypeConf.xml
+ */
+ private void loadJobTypes() {
+ Document document = Utils.loadXMLDoc(JOB_TYPES_CONF);
+ JobTypeConfiguration conf = new JobTypeConfiguration(document.getDocumentElement());
+ _appTypeToJobTypes = conf.getAppTypeToJobTypeList();
+ }
+
+ /**
+ * Load in the GeneralConf.xml file as a configuration object for other objects to access
+ */
+ private void loadGeneralConf() {
+ logger.info("Loading configuration file " + GENERAL_CONF);
+
+ _generalConf = new Configuration();
+ _generalConf.addResource(this.getClass().getClassLoader().getResourceAsStream(GENERAL_CONF));
+ }
+
+ /**
+ * Given an application type, return the currently bound heuristics
+ *
+ * @param type The application type
+ * @return The corresponding heuristics
+ */
+ public List getHeuristicsForApplicationType(ApplicationType type) {
+ return _typeToHeuristics.get(type);
+ }
+
+ /**
+ * Return the heuristic names available grouped by application type.
+ *
+ * @return A map of application type name -> a list of heuristic names
+ */
+ public Map> getAllHeuristicNames() {
+ if (_heuristicGroupedNames.isEmpty()) {
+ for (Map.Entry> entry : _typeToHeuristics.entrySet()) {
+ ApplicationType type = entry.getKey();
+ List list = entry.getValue();
+
+ List nameList = new ArrayList();
+ for (Heuristic heuristic : list) {
+ nameList.add(heuristic.getHeuristicConfData().getHeuristicName());
+ }
+
+ Collections.sort(nameList);
+ _heuristicGroupedNames.put(type.getName(), nameList);
+ }
+ }
+
+ return _heuristicGroupedNames;
+ }
+
+ /**
+ * Get the heuristic configuration data
+ *
+ * @return The configuration data of heuristics
+ */
+ public List getHeuristicsConfigurationData() {
+ return _heuristicsConfData;
+ }
+
+ /**
+ * Given an application type, return the currently ElephantFetcher that binds with the type.
+ *
+ * @param type The application type
+ * @return The corresponding fetcher
+ */
+ public ElephantFetcher getFetcherForApplicationType(ApplicationType type) {
+ return _typeToFetcher.get(type);
+ }
+
+ public HadoopMetricsAggregator getAggregatorForApplicationType(ApplicationType type) {
+ return _typeToAggregator.get(type);
+ }
+
+ /**
+ * Get the application type given a type name.
+ *
+ * @return The corresponding application type, null if not found
+ */
+ public ApplicationType getApplicationTypeForName(String typeName) {
+ return _nameToType.get(typeName.toUpperCase());
+ }
+
+ /**
+ * Get the general configuration object.
+ *
+ * @return the genral configuration object.
+ */
+ public Configuration getGeneralConf() {
+ return _generalConf;
+ }
+
+ /**
+ * Get the matched job type given a
+ *
+ * @param data The HadoopApplicationData to check
+ * @return The matched job type
+ */
+ public JobType matchJobType(HadoopApplicationData data) {
+ if (data != null) {
+ List jobTypeList = _appTypeToJobTypes.get(data.getApplicationType());
+ Properties jobProp = data.getConf();
+ for (JobType type : jobTypeList) {
+ if (type.matchType(jobProp)) {
+ return type;
+ }
+ }
+ }
+ return null;
+ }
+
+ public Map> getAppTypeToJobTypes() {
+ return ImmutableMap.copyOf(_appTypeToJobTypes);
+ }
+
+ public Map getHeuristicToView() {
+ return ImmutableMap.copyOf(_heuristicToView);
+ }
+}
diff --git a/app/com/linkedin/drelephant/ElephantFetcher.java b/app/com/linkedin/drelephant/ElephantFetcher.java
deleted file mode 100644
index 24f5afe62..000000000
--- a/app/com/linkedin/drelephant/ElephantFetcher.java
+++ /dev/null
@@ -1,18 +0,0 @@
-package com.linkedin.drelephant;
-
-import com.linkedin.drelephant.hadoop.HadoopJobData;
-import org.apache.hadoop.security.authentication.client.AuthenticationException;
-
-import java.io.IOException;
-import java.util.List;
-
-
-public interface ElephantFetcher {
-
- public List fetchJobList() throws IOException, AuthenticationException;
-
- public void fetchJobData(HadoopJobData job_data) throws IOException, AuthenticationException;
-
- public void finishJob(HadoopJobData job_data, boolean success);
-
-}
diff --git a/app/com/linkedin/drelephant/ElephantFetcherClassic.java b/app/com/linkedin/drelephant/ElephantFetcherClassic.java
deleted file mode 100644
index de5bd3783..000000000
--- a/app/com/linkedin/drelephant/ElephantFetcherClassic.java
+++ /dev/null
@@ -1,310 +0,0 @@
-package com.linkedin.drelephant;
-
-import com.linkedin.drelephant.analysis.Constants;
-import com.linkedin.drelephant.hadoop.HadoopCounterHolder;
-import com.linkedin.drelephant.hadoop.HadoopCounterHolder.CounterName;
-import com.linkedin.drelephant.hadoop.HadoopJobData;
-import com.linkedin.drelephant.hadoop.HadoopTaskData;
-import com.linkedin.drelephant.math.Statistics;
-
-import model.JobResult;
-
-import org.apache.commons.io.IOUtils;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.mapred.*;
-import org.apache.hadoop.security.authentication.client.AuthenticatedURL;
-import org.apache.hadoop.security.authentication.client.AuthenticationException;
-import org.apache.log4j.Logger;
-import org.jsoup.Jsoup;
-import org.jsoup.nodes.Document;
-import org.jsoup.nodes.Element;
-import org.jsoup.select.Elements;
-
-import java.io.IOException;
-import java.net.HttpURLConnection;
-import java.net.URL;
-import java.text.SimpleDateFormat;
-import java.text.ParseException;
-import java.util.ArrayList;
-import java.util.EnumMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Properties;
-import java.util.Set;
-
-
-public class ElephantFetcherClassic implements ElephantFetcher {
- private static final Logger logger = Logger.getLogger(ElephantFetcher.class);
- private static SimpleDateFormat dateFormat = new SimpleDateFormat("d-MMM-yyyy HH:mm:ss");
-
- private JobClient jobClient;
- private Set previousJobs = new HashSet();
- private boolean firstRun = true;
-
- public ElephantFetcherClassic(Configuration hadoopConf) throws IOException {
- init(hadoopConf);
- }
-
- private void init(Configuration hadoopConf) throws IOException {
- logger.info("Connecting to the jobtracker");
- jobClient = new JobClient(new JobConf(hadoopConf));
- }
-
- public List fetchJobList() throws IOException {
- JobStatus[] result = null;
-
- result = jobClient.getAllJobs();
- if (result == null) {
- throw new IOException("Error fetching joblist from jobtracker.");
- }
-
- Set successJobs = filterSuccessfulJobs(result);
- successJobs = filterPreviousJobs(successJobs, previousJobs);
-
- List jobList = new ArrayList();
- for (String jobId : successJobs) {
- jobList.add(new HadoopJobData().setJobId(jobId));
- }
- return jobList;
- }
-
- public void finishJob(HadoopJobData jobData, boolean success) {
- if (success) {
- previousJobs.add(jobData.getJobId());
- }
- }
-
- public void fetchJobData(HadoopJobData jobData) throws IOException, AuthenticationException {
- JobID job_id = JobID.forName(jobData.getJobId());
-
- RunningJob job = getJob(job_id);
- if (job == null) {
- throw new IOException("Unable to fetch job data from Jobtracker, job id = " + job_id);
- }
-
- JobStatus status = job.getJobStatus();
- String username = status.getUsername();
- long startTime = status.getStartTime();
- String jobUrl = job.getTrackingURL();
- String jobName = job.getJobName();
-
- HadoopCounterHolder counterHolder = fetchCounter(job.getCounters());
-
- TaskReport[] mapperTasks = getMapTaskReports(job_id);
- TaskReport[] reducerTasks = getReduceTaskReports(job_id);
- String jobTrackingUrl = job.getTrackingURL();
- int sampleSize = Constants.SHUFFLE_SORT_MAX_SAMPLE_SIZE;
-
- HadoopTaskData[] mappers = new HadoopTaskData[mapperTasks.length];
- Statistics.shuffleArraySample(mapperTasks, sampleSize);
- for (int i = 0; i < mapperTasks.length; i++) {
- mappers[i] = fetchTaskData(jobTrackingUrl, mapperTasks[i], false, (i < sampleSize));
- }
-
- HadoopTaskData[] reducers = new HadoopTaskData[reducerTasks.length];
- Statistics.shuffleArraySample(reducerTasks, sampleSize);
- for (int i = 0; i < reducerTasks.length; i++) {
- reducers[i] = fetchTaskData(jobTrackingUrl, reducerTasks[i], true, (i < sampleSize));
- }
-
- Properties jobConf = getJobConf(job);
-
- jobData.setUsername(username).setStartTime(startTime).setUrl(jobUrl).setJobName(jobName).setCounters(counterHolder)
- .setMapperData(mappers).setReducerData(reducers).setJobConf(jobConf);
-
- }
-
- private RunningJob getJob(JobID job_id) throws IOException {
- return jobClient.getJob(job_id);
- }
-
- private TaskReport[] getMapTaskReports(JobID job_id) throws IOException {
- return jobClient.getMapTaskReports(job_id);
- }
-
- private TaskReport[] getReduceTaskReports(JobID job_id) throws IOException {
- return jobClient.getReduceTaskReports(job_id);
- }
-
- private Properties getJobConf(RunningJob job) throws IOException, AuthenticationException {
- Properties properties = new Properties();
- String jobconfUrl = getJobconfUrl(job);
- if (jobconfUrl == null) {
- return properties;
- }
-
- URL url = new URL(jobconfUrl);
- AuthenticatedURL.Token token = new AuthenticatedURL.Token();
- HttpURLConnection conn = new AuthenticatedURL().openConnection(url, token);
- String data = IOUtils.toString(conn.getInputStream());
- Document doc = Jsoup.parse(data);
- Elements rows = doc.select("table").select("tr");
- for (int i = 1; i < rows.size(); i++) {
- Element row = rows.get(i);
- Elements cells = row.select("> td");
- if (cells.size() == 2) {
- String key = cells.get(0).text().trim();
- String value = cells.get(1).text().trim();
- properties.put(key, value);
- }
- }
- return properties;
- }
-
- private String getJobconfUrl(RunningJob job) {
- String jobDetails = job.getTrackingURL();
- String root = jobDetails.substring(0, jobDetails.indexOf("jobdetails.jsp"));
- return root + "jobconf.jsp?jobid=" + job.getID().toString();
- }
-
- private Set filterSuccessfulJobs(JobStatus[] jobs) {
- Set successJobs = new HashSet();
- for (JobStatus job : jobs) {
- if (job.getRunState() == JobStatus.SUCCEEDED && job.isJobComplete()) {
- successJobs.add(job.getJobID().toString());
- }
- }
- return successJobs;
- }
-
- private Set filterPreviousJobs(Set jobs, Set previousJobs) {
- logger.info("Cleaning up previous runs.");
- // On first run, check against DB
- if (firstRun) {
- Set newJobs = new HashSet();
- for (String jobId : jobs) {
- JobResult prevResult = JobResult.find.byId(jobId);
- if (prevResult == null) {
- // Job not found, add to new jobs list
- newJobs.add(jobId);
- } else {
- // Job found, add to old jobs list
- previousJobs.add(jobId);
- }
- }
- jobs = newJobs;
- firstRun = false;
- } else {
- // Remove untracked jobs
- previousJobs.retainAll(jobs);
- // Remove previously analysed jobs
- jobs.removeAll(previousJobs);
- }
- return jobs;
- }
-
- private HadoopTaskData fetchTaskData(String jobDetailUrl, TaskReport task, boolean isReducer, boolean sampled)
- throws IOException, AuthenticationException {
-
- HadoopCounterHolder taskCounter = fetchCounter(task.getCounters());
-
- if (!sampled) {
- return new HadoopTaskData(taskCounter);
- }
-
- String taskDetailsUrl = getTaskDetailsPage(jobDetailUrl, task.getTaskID().toString());
- long[] time = fetchTaskDetails(taskDetailsUrl, isReducer);
-
- return new HadoopTaskData(taskCounter, time);
- }
-
- private String getTaskDetailsPage(String jobDetails, String taskId) {
- String root = jobDetails.substring(0, jobDetails.indexOf("jobdetails.jsp"));
- return root + "taskdetails.jsp?tipid=" + taskId.toString();
- }
-
- private long[] fetchTaskDetails(String taskDetailUrl, boolean isReducer) throws IOException, AuthenticationException {
-
- URL url = new URL(taskDetailUrl);
- AuthenticatedURL.Token token = new AuthenticatedURL.Token();
- HttpURLConnection conn = new AuthenticatedURL().openConnection(url, token);
- String data = IOUtils.toString(conn.getInputStream());
- Document doc = Jsoup.parse(data);
- Elements rows = doc.select("table").select("tr");
- long[] time = null;
- for (int i = 1; i < rows.size(); i++) {
- Element row = rows.get(i);
- try {
- time = tryExtractDetailFromRow(row, isReducer);
- if (time != null) {
- return time;
- }
- } catch (Exception e) {
- throw new IOException("Error in fetch task data from task detail page. TASK URL=" + taskDetailUrl, e);
- }
- }
- throw new IOException("No valid time data found from task detail page. TASK URL=" + taskDetailUrl);
- }
-
- //Return shuffle sort time if successfully extracted data from row
- private long[] tryExtractDetailFromRow(Element row, boolean isReducer) throws ParseException {
- Elements cells = row.select("> td");
-
- // For rows( ) in reducer task page with other than 12 cols( ),or 10 cols in mapper page,
- // they are not rows that contains time data
- if ((isReducer && cells.size() != 12) || (!isReducer && cells.size() != 10)) {
- return null;
- }
-
- boolean succeeded = cells.get(2).html().trim().equals("SUCCEEDED");
- if (succeeded) {
- if (isReducer) {
- // Fetch time info from reducer task page
- String startTime = cells.get(4).html().trim();
- String shuffleTime = cells.get(5).html().trim();
- String sortTime = cells.get(6).html().trim();
- String finishTime = cells.get(7).html().trim();
- if (shuffleTime.contains("(")) {
- shuffleTime = shuffleTime.substring(0, shuffleTime.indexOf("(") - 1);
- }
- if (sortTime.contains("(")) {
- sortTime = sortTime.substring(0, sortTime.indexOf("(") - 1);
- }
- if (finishTime.contains("(")) {
- finishTime = finishTime.substring(0, finishTime.indexOf("(") - 1);
- }
- long start = dateFormat.parse(startTime).getTime();
- long shuffle = dateFormat.parse(shuffleTime).getTime();
- long sort = dateFormat.parse(sortTime).getTime();
- long finish = dateFormat.parse(finishTime).getTime();
-
- long shuffleDuration = (shuffle - start);
- long sortDuration = (sort - shuffle);
- return new long[] { start, finish, shuffleDuration, sortDuration };
- } else {
- // Fetch time info from mapper task page
- String startTime = cells.get(4).html().trim();
- String finishTime = cells.get(5).html().trim();
- if (finishTime.contains("(")) {
- finishTime = finishTime.substring(0, finishTime.indexOf("(") - 1);
- }
- long start = dateFormat.parse(startTime).getTime();
- long finish = dateFormat.parse(finishTime).getTime();
- return new long[] { start, finish, 0, 0 };
- }
- }
- return null;
- }
-
- private HadoopCounterHolder fetchCounter(Counters counters) {
- Map counterMap = new EnumMap(CounterName.class);
- for (CounterName counterName : CounterName.values()) {
- counterMap.put(counterName, readCounter(counterName, counters));
- }
- return new HadoopCounterHolder(counterMap);
- }
-
- private long readCounter(CounterName counterName, Counters counters) {
- String groupName = counterName.getGroup().getName();
- Counters.Group group = counters.getGroup(groupName);
- if (group == null) {
- return 0;
- }
- Counters.Counter counter = group.getCounterForName(counterName.getName());
- if (counter == null) {
- return 0;
- }
- return counter.getValue();
- }
-}
diff --git a/app/com/linkedin/drelephant/ElephantFetcherYarn.java b/app/com/linkedin/drelephant/ElephantFetcherYarn.java
deleted file mode 100644
index 3814d1fb7..000000000
--- a/app/com/linkedin/drelephant/ElephantFetcherYarn.java
+++ /dev/null
@@ -1,364 +0,0 @@
-package com.linkedin.drelephant;
-
-import com.google.common.collect.Lists;
-import com.linkedin.drelephant.hadoop.HadoopCounterHolder;
-import com.linkedin.drelephant.hadoop.HadoopCounterHolder.CounterName;
-import com.linkedin.drelephant.hadoop.HadoopJobData;
-import com.linkedin.drelephant.hadoop.HadoopTaskData;
-
-import model.JobResult;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.security.authentication.client.AuthenticatedURL;
-import org.apache.hadoop.security.authentication.client.AuthenticationException;
-import org.apache.log4j.Logger;
-import org.codehaus.jackson.JsonNode;
-import org.codehaus.jackson.map.ObjectMapper;
-
-import java.io.IOException;
-import java.net.HttpURLConnection;
-import java.net.MalformedURLException;
-import java.net.URL;
-import java.net.URLConnection;
-import java.util.ArrayList;
-import java.util.EnumMap;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Properties;
-import java.util.Set;
-
-
-public class ElephantFetcherYarn implements ElephantFetcher {
- private static final Logger logger = Logger.getLogger(ElephantFetcher.class);
-
- private RetryFactory retryFactory;
- private URLFactory urlFactory;
- private JSONFactory jsonFactory;
- private boolean firstTime = true;
- private long lastTime = 0;
- private long currentTime = 0;
-
- public ElephantFetcherYarn(Configuration hadoopConf) throws IOException {
- init(hadoopConf);
- }
-
- private void init(Configuration hadoopConf) throws IOException {
- logger.info("Connecting to the job history server...");
- String jhistoryAddr = hadoopConf.get("mapreduce.jobhistory.webapp.address");
- urlFactory = new URLFactory(jhistoryAddr);
- jsonFactory = new JSONFactory();
- retryFactory = new RetryFactory();
- logger.info("Connection success.");
- }
-
- /*
- * Fetch job list to analyze
- * If first time, search time span from 0 to now, check database for each job
- * If not first time, search time span since last fetch, also re-fetch failed jobs
- * Return list on success, throw Exception on error
- */
- public List fetchJobList() throws IOException, AuthenticationException {
-
- List jobList;
-
- currentTime = System.currentTimeMillis();
- URL joblistURL = urlFactory.fetchJobListURL(lastTime, currentTime);
-
- jobList = jsonFactory.getJobData(joblistURL, firstTime);
- if (firstTime) {
- firstTime = false;
- } else {
- // If not first time, also fetch jobs that need to retry
- jobList.addAll(retryFactory.getJobs());
- }
-
- lastTime = currentTime;
-
- return jobList;
- }
-
- // Check database to see if a job is already analyzed
- private boolean checkDBforJob(String jobId) {
- JobResult result = JobResult.find.byId(jobId);
- return (result != null);
- }
-
- // Clear all data stored on the job object
- private void clearJobData(HadoopJobData jobData) {
- jobData.setCounters(null).setJobConf(null).setMapperData(null).setReducerData(null);
- }
-
- // OnJobFinish Add to retry list upon failure
- public void finishJob(HadoopJobData jobData, boolean success) {
- if (!success) {
- clearJobData(jobData);
- // Add to retry list
- retryFactory.addJob(jobData);
- }
- }
-
- // Fetch job detailed data. Return true on success
- public void fetchJobData(HadoopJobData jobData) throws IOException, AuthenticationException {
- String jobId = jobData.getJobId();
-
- // Fetch job counter
- URL jobCounterURL = urlFactory.getJobCounterURL(jobId);
- HadoopCounterHolder jobCounter = jsonFactory.getJobCounter(jobCounterURL);
-
- // Fetch job config
- URL jobConfigURL = urlFactory.getJobConfigURL(jobId);
- Properties jobConf = jsonFactory.getProperties(jobConfigURL);
-
- // Fetch task data
- URL taskListURL = urlFactory.getTaskListURL(jobId);
- List mapperList = new ArrayList();
- List reducerList = new ArrayList();
- jsonFactory.getTaskDataAll(taskListURL, jobId, mapperList, reducerList);
-
- HadoopTaskData[] mapperData = mapperList.toArray(new HadoopTaskData[mapperList.size()]);
- HadoopTaskData[] reducerData = reducerList.toArray(new HadoopTaskData[reducerList.size()]);
-
- jobData.setCounters(jobCounter).setMapperData(mapperData).setReducerData(reducerData).setJobConf(jobConf);
- }
-
- private String getJobDetailURL(String jobId) {
- return urlFactory.getJobDetailURLString(jobId);
- }
-
- private URL getTaskCounterURL(String jobId, String taskId) throws MalformedURLException {
- return urlFactory.getTaskCounterURL(jobId, taskId);
- }
-
- private URL getTaskAttemptURL(String jobId, String taskId, String attemptId) throws MalformedURLException {
- return urlFactory.getTaskAttemptURL(jobId, taskId, attemptId);
- }
-
- private class URLFactory {
-
- private String root;
- private String restRoot;
-
- public URLFactory(String hserverAddr) throws IOException {
- root = "http://" + hserverAddr;
- restRoot = "http://" + hserverAddr + "/ws/v1/history/mapreduce/jobs";
- verifyURL(restRoot);
- }
-
- private void verifyURL(String url) throws IOException {
- final URLConnection connection = new URL(url).openConnection();
- // Check service availability
- connection.connect();
- return;
- }
-
- private String getJobDetailURLString(String jobId) {
- return root + "/jobhistory/job/" + jobId;
- }
-
- private URL fetchJobListURL(long startTime, long endTime) throws MalformedURLException {
- return new URL(restRoot + "?finishedTimeBegin=" + startTime + "&finishedTimeEnd=" + endTime + "&state=SUCCEEDED");
- }
-
- private URL getJobConfigURL(String jobId) throws MalformedURLException {
- return new URL(restRoot + "/" + jobId + "/conf");
- }
-
- private URL getJobCounterURL(String jobId) throws MalformedURLException {
- return new URL(restRoot + "/" + jobId + "/counters");
- }
-
- private URL getTaskListURL(String jobId) throws MalformedURLException {
- return new URL(restRoot + "/" + jobId + "/tasks");
- }
-
- private URL getTaskCounterURL(String jobId, String taskId) throws MalformedURLException {
- return new URL(restRoot + "/" + jobId + "/tasks/" + taskId + "/counters");
- }
-
- private URL getTaskAttemptURL(String jobId, String taskId, String attemptId) throws MalformedURLException {
- return new URL(restRoot + "/" + jobId + "/tasks/" + taskId + "/attempts/" + attemptId);
- }
- }
-
- private class JSONFactory {
- private ObjectMapper mapper = new ObjectMapper();
- private AuthenticatedURL.Token token = new AuthenticatedURL.Token();
- private AuthenticatedURL authenticatedURL = new AuthenticatedURL();
- private Set counterSet = new HashSet();;
-
- public JSONFactory() {
- // Store the set of counters we want to fetch
- for (CounterName counter : CounterName.values()) {
- counterSet.add(counter.getName());
- }
- }
-
- private List getJobData(URL url, boolean checkDB) throws IOException, AuthenticationException {
- List jobList = new ArrayList();
-
- HttpURLConnection conn = authenticatedURL.openConnection(url, token);
- JsonNode rootNode = mapper.readTree(conn.getInputStream());
- JsonNode jobs = rootNode.path("jobs").path("job");
-
- for (JsonNode job : jobs) {
- String jobId = job.get("id").getValueAsText();
-
- // On first time, for every job, we check database
- if (checkDB && checkDBforJob(jobId)) {
- continue;
- }
-
- // New job
- HadoopJobData jobData = new HadoopJobData();
- jobData.setJobId(jobId).setUsername(job.get("user").getValueAsText())
- .setJobName(job.get("name").getValueAsText()).setUrl(getJobDetailURL(jobId));
-
- jobList.add(jobData);
- }
- return jobList;
- }
-
- private Properties getProperties(URL url) throws IOException, AuthenticationException {
- Properties jobConf = new Properties();
-
- HttpURLConnection conn = authenticatedURL.openConnection(url, token);
- JsonNode rootNode = mapper.readTree(conn.getInputStream());
- JsonNode configs = rootNode.path("conf").path("property");
-
- for (JsonNode conf : configs) {
- String key = conf.get("name").getValueAsText();
- String val = conf.get("value").getValueAsText();
- jobConf.setProperty(key, val);
- }
- return jobConf;
- }
-
- private HadoopCounterHolder getJobCounter(URL url) throws IOException, AuthenticationException {
- Map counterMap = new EnumMap(CounterName.class);
-
- HttpURLConnection conn = authenticatedURL.openConnection(url, token);
- JsonNode rootNode = mapper.readTree(conn.getInputStream());
- JsonNode groups = rootNode.path("jobCounters").path("counterGroup");
-
- for (JsonNode group : groups) {
- for (JsonNode counter : group.path("counter")) {
- String name = counter.get("name").getValueAsText();
- if (counterSet.contains(name)) {
- // This is a counter we want to fetch
- long val = counter.get("totalCounterValue").getLongValue();
- counterMap.put(CounterName.valueOf(name), val);
- }
- }
- }
- // For every missing counters in the job, set with default value 0
- for (CounterName name : CounterName.values()) {
- if (!counterMap.containsKey(name)) {
- counterMap.put(name, 0L);
- }
- }
- return new HadoopCounterHolder(counterMap);
- }
-
- private HadoopCounterHolder getTaskCounter(URL url) throws IOException, AuthenticationException {
- Map counterMap = new EnumMap(CounterName.class);
-
- HttpURLConnection conn = authenticatedURL.openConnection(url, token);
- JsonNode rootNode = mapper.readTree(conn.getInputStream());
- JsonNode groups = rootNode.path("jobTaskCounters").path("taskCounterGroup");
-
- for (JsonNode group : groups) {
- for (JsonNode counter : group.path("counter")) {
- String name = counter.get("name").getValueAsText();
- if (counterSet.contains(name)) {
- long val = counter.get("value").getLongValue();
- counterMap.put(CounterName.valueOf(name), val);
- }
- }
- }
-
- for (CounterName name : CounterName.values()) {
- if (!counterMap.containsKey(name)) {
- counterMap.put(name, 0L);
- }
- }
- return new HadoopCounterHolder(counterMap);
- }
-
- private long[] getTaskExecTime(URL url) throws IOException, AuthenticationException {
- HttpURLConnection conn = authenticatedURL.openConnection(url, token);
- JsonNode rootNode = mapper.readTree(conn.getInputStream());
- JsonNode taskAttempt = rootNode.path("taskAttempt");
-
- long startTime = taskAttempt.get("startTime").getLongValue();
- long finishTime = taskAttempt.get("finishTime").getLongValue();
- boolean isMapper = taskAttempt.get("type").getValueAsText().equals("MAP");
-
- long[] time;
- if (isMapper) {
- // No shuffle sore time in Mapper
- time = new long[] { startTime, finishTime, 0, 0 };
- } else {
- long shuffleTime = taskAttempt.get("elapsedShuffleTime").getLongValue();
- long sortTime = taskAttempt.get("elapsedMergeTime").getLongValue();
- time = new long[] { startTime, finishTime, shuffleTime, sortTime };
- }
-
- return time;
- }
-
- private void getTaskDataAll(URL url, String jobId, List mapperList, List reducerList)
- throws IOException, AuthenticationException {
- HttpURLConnection conn = authenticatedURL.openConnection(url, token);
- JsonNode rootNode = mapper.readTree(conn.getInputStream());
- JsonNode tasks = rootNode.path("tasks").path("task");
-
- for (JsonNode task : tasks) {
- String taskId = task.get("id").getValueAsText();
- String attemptId = task.get("successfulAttempt").getValueAsText();
- boolean isMapper = task.get("type").getValueAsText().equals("MAP");
-
- URL taskCounterURL = getTaskCounterURL(jobId, taskId);
- HadoopCounterHolder taskCounter = getTaskCounter(taskCounterURL);
-
- URL taskAttemptURL = getTaskAttemptURL(jobId, taskId, attemptId);
- long[] taskExecTime = getTaskExecTime(taskAttemptURL);
-
- HadoopTaskData taskData = new HadoopTaskData(taskCounter, taskExecTime);
- if (isMapper) {
- mapperList.add(taskData);
- } else {
- reducerList.add(taskData);
- }
- }
- }
- }
-
- private class RetryFactory {
- private static final int DEFAULT_RETRY = 3;
- private Map retryMap = new HashMap();
-
- private void addJob(HadoopJobData job) {
- if (retryMap.containsKey(job)) {
- // This is old retry job
- int retryLeft = retryMap.get(job);
- if (retryLeft == 1) {
- // Drop job on max retries
- logger.error("Drop job. Reason: reach max retry for job id=" + job.getJobId());
- retryMap.remove(job);
- } else {
- retryMap.put(job, retryLeft - 1);
- }
- } else {
- // This is new retry job
- retryMap.put(job, DEFAULT_RETRY);
- }
- }
-
- private List getJobs() {
- return Lists.newArrayList(retryMap.keySet());
- }
- }
-
-}
diff --git a/app/com/linkedin/drelephant/ElephantRunner.java b/app/com/linkedin/drelephant/ElephantRunner.java
index 66d540758..2bebd00c7 100644
--- a/app/com/linkedin/drelephant/ElephantRunner.java
+++ b/app/com/linkedin/drelephant/ElephantRunner.java
@@ -1,180 +1,230 @@
+/*
+ * Copyright 2016 LinkedIn Corp.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License"); you may not
+ * use this file except in compliance with the License. You may obtain a copy of
+ * the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
package com.linkedin.drelephant;
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+
+import com.linkedin.drelephant.analysis.AnalyticJob;
+import com.linkedin.drelephant.analysis.AnalyticJobGenerator;
+import com.linkedin.drelephant.analysis.HDFSContext;
+import com.linkedin.drelephant.analysis.HadoopSystemContext;
+import com.linkedin.drelephant.analysis.AnalyticJobGeneratorHadoop2;
+
+import com.linkedin.drelephant.security.HadoopSecurity;
+
+import controllers.MetricsController;
import java.io.IOException;
import java.security.PrivilegedAction;
-import java.util.ArrayList;
import java.util.List;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.ThreadFactory;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
-import model.JobHeuristicResult;
-import model.JobResult;
-import model.JobType;
+import com.linkedin.drelephant.util.Utils;
+import models.AppResult;
+import org.apache.commons.lang.exception.ExceptionUtils;
import org.apache.hadoop.conf.Configuration;
import org.apache.log4j.Logger;
-import com.linkedin.drelephant.analysis.Constants;
-import com.linkedin.drelephant.analysis.HeuristicResult;
-import com.linkedin.drelephant.analysis.Severity;
-import com.linkedin.drelephant.hadoop.HadoopJobData;
-import com.linkedin.drelephant.hadoop.HadoopSecurity;
-import com.linkedin.drelephant.notifications.EmailThread;
-
+/**
+ * The class that runs the Dr. Elephant daemon
+ */
public class ElephantRunner implements Runnable {
- private static final long WAIT_INTERVAL = 10 * 1000;
private static final Logger logger = Logger.getLogger(ElephantRunner.class);
- private AtomicBoolean running = new AtomicBoolean(true);
- private EmailThread emailer = new EmailThread();
- private HadoopSecurity hadoopSecurity;
- private InfoExtractor urlRetriever = new InfoExtractor();
+
+ private static final long FETCH_INTERVAL = 60 * 1000; // Interval between fetches
+ private static final long RETRY_INTERVAL = 60 * 1000; // Interval between retries
+ private static final int EXECUTOR_NUM = 5; // The number of executor threads to analyse the jobs
+
+ private static final String FETCH_INTERVAL_KEY = "drelephant.analysis.fetch.interval";
+ private static final String RETRY_INTERVAL_KEY = "drelephant.analysis.retry.interval";
+ private static final String EXECUTOR_NUM_KEY = "drelephant.analysis.thread.count";
+
+ private AtomicBoolean _running = new AtomicBoolean(true);
+ private long lastRun;
+ private long _fetchInterval;
+ private long _retryInterval;
+ private int _executorNum;
+ private HadoopSecurity _hadoopSecurity;
+ private ThreadPoolExecutor _threadPoolExecutor;
+ private AnalyticJobGenerator _analyticJobGenerator;
+
+ private void loadGeneralConfiguration() {
+ Configuration configuration = ElephantContext.instance().getGeneralConf();
+
+ _executorNum = Utils.getNonNegativeInt(configuration, EXECUTOR_NUM_KEY, EXECUTOR_NUM);
+ _fetchInterval = Utils.getNonNegativeLong(configuration, FETCH_INTERVAL_KEY, FETCH_INTERVAL);
+ _retryInterval = Utils.getNonNegativeLong(configuration, RETRY_INTERVAL_KEY, RETRY_INTERVAL);
+ }
+
+ private void loadAnalyticJobGenerator() {
+ if (HadoopSystemContext.isHadoop2Env()) {
+ _analyticJobGenerator = new AnalyticJobGeneratorHadoop2();
+ } else {
+ throw new RuntimeException("Unsupported Hadoop major version detected. It is not 2.x.");
+ }
+
+ try {
+ _analyticJobGenerator.configure(ElephantContext.instance().getGeneralConf());
+ } catch (Exception e) {
+ logger.error("Error occurred when configuring the analysis provider.", e);
+ throw new RuntimeException(e);
+ }
+ }
@Override
public void run() {
logger.info("Dr.elephant has started");
try {
- hadoopSecurity = new HadoopSecurity();
- hadoopSecurity.doAs(new PrivilegedAction() {
+ _hadoopSecurity = HadoopSecurity.getInstance();
+ _hadoopSecurity.doAs(new PrivilegedAction() {
@Override
public Void run() {
- Constants.load();
- emailer.start();
- long lastRun;
- ElephantFetcher fetcher = null;
-
- try {
- // Tell which hadoop version from hadoop configuration,
- // and start fetcher accordingly
- Configuration hadoopConf = new Configuration();
- String framework = hadoopConf.get("mapreduce.framework.name");
-
- if (framework != null) {
- if (framework.equals("yarn")) {
- fetcher = new ElephantFetcherYarn(hadoopConf);
- } else if (framework.equals("classic")) {
- fetcher = new ElephantFetcherClassic(hadoopConf);
- } else {
- logger.error("mapreduce.framework.name must be either 'classic' or 'yarn'. Current value: "+framework);
- return null;
- }
- } else {
- if (hadoopConf.get("mapred.job.tracker.http.address") != null) {
- fetcher = new ElephantFetcherClassic(hadoopConf);
- } else {
- logger.error("Either mapreduce.framework.name or mapred.job.tracker.http.address must be set. Plseae check your configuration.");
- return null;
- }
- }
+ HDFSContext.load();
+ loadGeneralConfiguration();
+ loadAnalyticJobGenerator();
+ ElephantContext.init();
- } catch (IOException e) {
- logger.error("Error initializing dr elephant fetcher! ", e);
- return null;
+ // Initialize the metrics registries.
+ MetricsController.init();
+
+ logger.info("executor num is " + _executorNum);
+ if (_executorNum < 1) {
+ throw new RuntimeException("Must have at least 1 worker thread.");
}
+ ThreadFactory factory = new ThreadFactoryBuilder().setNameFormat("dr-el-executor-thread-%d").build();
+ _threadPoolExecutor = new ThreadPoolExecutor(_executorNum, _executorNum, 0L, TimeUnit.MILLISECONDS,
+ new LinkedBlockingQueue(), factory);
- while (running.get()) {
+ while (_running.get() && !Thread.currentThread().isInterrupted()) {
+ _analyticJobGenerator.updateResourceManagerAddresses();
lastRun = System.currentTimeMillis();
- logger.info("Fetching job list.....");
+ logger.info("Fetching analytic job list...");
try {
- hadoopSecurity.checkLogin();
+ _hadoopSecurity.checkLogin();
} catch (IOException e) {
logger.info("Error with hadoop kerberos login", e);
+ //Wait for a while before retry
+ waitInterval(_retryInterval);
continue;
}
- List successJobs;
+ List todos;
try {
- successJobs = fetcher.fetchJobList();
+ todos = _analyticJobGenerator.fetchAnalyticJobs();
} catch (Exception e) {
logger.error("Error fetching job list. Try again later...", e);
+ //Wait for a while before retry
+ waitInterval(_retryInterval);
continue;
}
- logger.info(successJobs.size() + " jobs to analyse.");
-
- // Analyse all ready jobs
- for (HadoopJobData jobData : successJobs) {
- try {
- fetcher.fetchJobData(jobData);
- analyzeJob(jobData);
- fetcher.finishJob(jobData, true);
- } catch (Exception e) {
- logger.error("Error fetching job data. job id=" + jobData.getJobId(), e);
- fetcher.finishJob(jobData, false);
- }
- }
- logger.info("Finished all jobs. Waiting for refresh.");
-
- // Wait for long enough
- long nextRun = lastRun + WAIT_INTERVAL;
- long waitTime = nextRun - System.currentTimeMillis();
- while (running.get() && waitTime > 0) {
- try {
- Thread.sleep(waitTime);
- } catch (InterruptedException e) {
- logger.error("Thread interrupted", e);
- }
- waitTime = nextRun - System.currentTimeMillis();
+ for (AnalyticJob analyticJob : todos) {
+ _threadPoolExecutor.submit(new ExecutorJob(analyticJob));
}
+
+ int queueSize = _threadPoolExecutor.getQueue().size();
+ MetricsController.setQueueSize(queueSize);
+ logger.info("Job queue size is " + queueSize);
+
+ //Wait for a while before next fetch
+ waitInterval(_fetchInterval);
}
+ logger.info("Main thread is terminated.");
return null;
}
});
- } catch (IOException e) {
- logger.error("Error on Hadoop Security setup. Failed to login with Kerberos");
+ } catch (Exception e) {
+ logger.error(e.getMessage());
+ logger.error(ExceptionUtils.getStackTrace(e));
}
}
- private void analyzeJob(HadoopJobData jobData) {
- ElephantAnalyser analyser = ElephantAnalyser.instance();
+ private class ExecutorJob implements Runnable {
- logger.info("Analyze job " + jobData.getJobId());
+ private AnalyticJob _analyticJob;
- HeuristicResult[] analysisResults = analyser.analyse(jobData);
- JobType jobType = analyser.getJobType(jobData);
-
- // Save to DB
- JobResult result = new JobResult();
- result.job_id = jobData.getJobId();
- result.url = jobData.getUrl();
- result.username = jobData.getUsername();
- result.startTime = jobData.getStartTime();
- result.analysisTime = System.currentTimeMillis();
- result.jobName = jobData.getJobName();
- result.jobType = jobType;
-
- // Truncate long names
- if (result.jobName.length() > 100) {
- result.jobName = result.jobName.substring(0, 97) + "...";
+ ExecutorJob(AnalyticJob analyticJob) {
+ _analyticJob = analyticJob;
}
- result.heuristicResults = new ArrayList();
-
- Severity worstSeverity = Severity.NONE;
-
- for (HeuristicResult heuristicResult : analysisResults) {
- JobHeuristicResult detail = new JobHeuristicResult();
- detail.analysisName = heuristicResult.getAnalysis();
- detail.data = heuristicResult.getDetailsCSV();
- detail.dataColumns = heuristicResult.getDetailsColumns();
- detail.severity = heuristicResult.getSeverity();
- if (detail.dataColumns < 1) {
- detail.dataColumns = 1;
+
+ @Override
+ public void run() {
+ try {
+ String analysisName = String.format("%s %s", _analyticJob.getAppType().getName(), _analyticJob.getAppId());
+ long analysisStartTimeMillis = System.currentTimeMillis();
+ logger.info(String.format("Analyzing %s", analysisName));
+ AppResult result = _analyticJob.getAnalysis();
+ result.save();
+ long processingTime = System.currentTimeMillis() - analysisStartTimeMillis;
+ logger.info(String.format("Analysis of %s took %sms", analysisName, processingTime));
+ MetricsController.setJobProcessingTime(processingTime);
+ MetricsController.markProcessedJobs();
+
+ } catch (InterruptedException e) {
+ logger.info("Thread interrupted");
+ logger.info(e.getMessage());
+ logger.info(ExceptionUtils.getStackTrace(e));
+
+ Thread.currentThread().interrupt();
+ } catch (Exception e) {
+ logger.error(e.getMessage());
+ logger.error(ExceptionUtils.getStackTrace(e));
+
+ if (_analyticJob != null && _analyticJob.retry()) {
+ logger.error("Add analytic job id [" + _analyticJob.getAppId() + "] into the retry list.");
+ _analyticJobGenerator.addIntoRetries(_analyticJob);
+ } else {
+ if (_analyticJob != null) {
+ MetricsController.markSkippedJob();
+ logger.error("Drop the analytic job. Reason: reached the max retries for application id = ["
+ + _analyticJob.getAppId() + "].");
+ }
+ }
}
- result.heuristicResults.add(detail);
- worstSeverity = Severity.max(worstSeverity, detail.severity);
}
+ }
- result.severity = worstSeverity;
- urlRetriever.retrieveURLs(result, jobData);
+ private void waitInterval(long interval) {
+ // Wait for long enough
+ long nextRun = lastRun + interval;
+ long waitTime = nextRun - System.currentTimeMillis();
- result.save();
+ if (waitTime <= 0) {
+ return;
+ }
- emailer.enqueue(result);
+ try {
+ Thread.sleep(waitTime);
+ } catch (InterruptedException e) {
+ Thread.currentThread().interrupt();
+ }
}
public void kill() {
- running.set(false);
- emailer.kill();
+ _running.set(false);
+ if (_threadPoolExecutor != null) {
+ _threadPoolExecutor.shutdownNow();
+ }
}
}
diff --git a/app/com/linkedin/drelephant/InfoExtractor.java b/app/com/linkedin/drelephant/InfoExtractor.java
deleted file mode 100644
index 9cab589ce..000000000
--- a/app/com/linkedin/drelephant/InfoExtractor.java
+++ /dev/null
@@ -1,44 +0,0 @@
-package com.linkedin.drelephant;
-
-import java.util.Properties;
-
-import org.apache.log4j.Logger;
-
-import model.JobResult;
-
-import com.linkedin.drelephant.hadoop.HadoopJobData;
-
-/**
- * InfoExtractor is responsible for retrieving information and context about a
- * job from the job's configuration which will be leveraged by the UI
- */
-public class InfoExtractor {
- private static final Logger logger = Logger.getLogger(InfoExtractor.class);
- private static final String AZK_URL_PREFIX = "azkaban.link";
- private static final String AZK_WORKFLOW_URL = "azkaban.link.workflow.url";
- private static final String AZK_JOB_URL = "azkaban.link.job.url";
- private static final String AZK_JOB_EXECUTION_URL =
- "azkaban.link.jobexec.url";
- private static final String AZK_EXECUTION_URL = "azkaban.link.execution.url";
- private static final String AZK_ATTEMPT_URL = "azkaban.link.attempt.url";
- private static final String AZK_URN_KEY = "azk.urn";
-
- void retrieveURLs(JobResult result, HadoopJobData jobData) {
- Properties jobConf = jobData.getJobConf();
- String jobId = jobData.getJobId();
- result.jobExecUrl = truncate(jobConf.getProperty(AZK_ATTEMPT_URL), jobId);
- // For jobs launched by Azkaban, we consider different attempts to be
- // different jobs
- result.jobUrl = truncate(jobConf.getProperty(AZK_JOB_URL), jobId);
- result.flowExecUrl = truncate(jobConf.getProperty(AZK_EXECUTION_URL), jobId);
- result.flowUrl = truncate(jobConf.getProperty(AZK_WORKFLOW_URL), jobId);
- }
-
- String truncate(String value, String jobId) {
- if (value != null && value.length() > JobResult.URL_LEN_LIMIT) {
- logger.info("Truncate long URL in job result for job : "+jobId+ ". Original Url : "+value);
- value = value.substring(0, JobResult.URL_LEN_LIMIT);
- }
- return value;
- }
-}
diff --git a/app/com/linkedin/drelephant/analysis/AnalyticJob.java b/app/com/linkedin/drelephant/analysis/AnalyticJob.java
new file mode 100644
index 000000000..029f429f8
--- /dev/null
+++ b/app/com/linkedin/drelephant/analysis/AnalyticJob.java
@@ -0,0 +1,326 @@
+/*
+ * Copyright 2016 LinkedIn Corp.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License"); you may not
+ * use this file except in compliance with the License. You may obtain a copy of
+ * the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package com.linkedin.drelephant.analysis;
+
+import com.linkedin.drelephant.ElephantContext;
+import com.linkedin.drelephant.util.InfoExtractor;
+import com.linkedin.drelephant.util.Utils;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import models.AppHeuristicResult;
+import models.AppHeuristicResultDetails;
+import models.AppResult;
+import org.apache.log4j.Logger;
+
+
+/**
+ * This class wraps some basic meta data of a completed application run (notice that the information is generally the
+ * same regardless of hadoop versions and application types), and then promises to return the analyzed result later.
+ */
+public class AnalyticJob {
+ private static final Logger logger = Logger.getLogger(AnalyticJob.class);
+
+ private static final String UNKNOWN_JOB_TYPE = "Unknown"; // The default job type when the data matches nothing.
+ private static final int _RETRY_LIMIT = 3; // Number of times a job needs to be tried before dropping
+ private static final String EXCLUDE_JOBTYPE = "exclude_jobtypes_filter"; // excluded Job Types for heuristic
+
+ private int _retries = 0;
+ private ApplicationType _type;
+ private String _appId;
+ private String _name;
+ private String _queueName;
+ private String _user;
+ private String _trackingUrl;
+ private long _startTime;
+ private long _finishTime;
+
+ /**
+ * Returns the application type
+ * E.g., Mapreduce or Spark
+ *
+ * @return The application type
+ */
+ public ApplicationType getAppType() {
+ return _type;
+ }
+
+ /**
+ * Set the application type of this job.
+ *
+ * @param type The Application type
+ * @return The analytic job
+ */
+ public AnalyticJob setAppType(ApplicationType type) {
+ _type = type;
+ return this;
+ }
+
+ /**
+ * Set the application id of this job
+ *
+ * @param appId The application id of the job obtained resource manager
+ * @return The analytic job
+ */
+ public AnalyticJob setAppId(String appId) {
+ _appId = appId;
+ return this;
+ }
+
+ /**
+ * Set the name of the analytic job
+ *
+ * @param name
+ * @return The analytic job
+ */
+ public AnalyticJob setName(String name) {
+ _name = name;
+ return this;
+ }
+
+ /**
+ * Set the queue name in which the analytic jobs was submitted
+ *
+ * @param name the name of the queue
+ * @return The analytic job
+ */
+ public AnalyticJob setQueueName(String name) {
+ _queueName = name;
+ return this;
+ }
+
+ /**
+ * Sets the user who ran the job
+ *
+ * @param user The username of the user
+ * @return The analytic job
+ */
+ public AnalyticJob setUser(String user) {
+ _user = user;
+ return this;
+ }
+
+ /**
+ * Sets the start time of the job
+ * Start time is the time at which the job was submitted by the resource manager
+ *
+ * @param startTime
+ * @return The analytic job
+ */
+ public AnalyticJob setStartTime(long startTime) {
+ // TIMESTAMP range starts from FROM_UNIXTIME(1) = 1970-01-01 00:00:01
+ if (startTime <= 0) {
+ startTime = 1000; // 1 sec
+ }
+ _startTime = startTime;
+ return this;
+ }
+
+ /**
+ * Sets the finish time of the job
+ *
+ * @param finishTime
+ * @return The analytic job
+ */
+ public AnalyticJob setFinishTime(long finishTime) {
+ // TIMESTAMP range starts from FROM_UNIXTIME(1) = 1970-01-01 00:00:01
+ if (finishTime <= 0) {
+ finishTime = 1000; // 1 sec
+ }
+ _finishTime = finishTime;
+ return this;
+ }
+
+ /**
+ * Returns the application id
+ *
+ * @return The analytic job
+ */
+ public String getAppId() {
+ return _appId;
+ }
+
+ /**
+ * Returns the name of the analytic job
+ *
+ * @return the analytic job's name
+ */
+ public String getName() {
+ return _name;
+ }
+
+ /**
+ * Returns the user who ran the job
+ *
+ * @return The user who ran the analytic job
+ */
+ public String getUser() {
+ return _user;
+ }
+
+ /**
+ * Returns the time at which the job was submitted by the resource manager
+ *
+ * @return The start time
+ */
+ public long getStartTime() {
+ return _startTime;
+ }
+
+ /**
+ * Returns the finish time of the job.
+ *
+ * @return The finish time
+ */
+ public long getFinishTime() {
+ return _finishTime;
+ }
+
+ /**
+ * Returns the tracking url of the job
+ *
+ * @return The tracking url in resource manager
+ */
+ public String getTrackingUrl() {
+ return _trackingUrl;
+ }
+
+ /**
+ * Returns the queue in which the application was submitted
+ *
+ * @return The queue name
+ */
+ public String getQueueName() {
+ return _queueName;
+ }
+
+ /**
+ * Sets the tracking url for the job
+ *
+ * @param trackingUrl The url to track the job
+ * @return The analytic job
+ */
+ public AnalyticJob setTrackingUrl(String trackingUrl) {
+ _trackingUrl = trackingUrl;
+ return this;
+ }
+
+ /**
+ * Returns the analysed AppResult that could be directly serialized into DB.
+ *
+ * This method fetches the data using the appropriate application fetcher, runs all the heuristics on them and
+ * loads it into the AppResult model.
+ *
+ * @throws Exception if the analysis process encountered a problem.
+ * @return the analysed AppResult
+ */
+ public AppResult getAnalysis() throws Exception {
+ ElephantFetcher fetcher = ElephantContext.instance().getFetcherForApplicationType(getAppType());
+ HadoopApplicationData data = fetcher.fetchData(this);
+
+ JobType jobType = ElephantContext.instance().matchJobType(data);
+ String jobTypeName = jobType == null ? UNKNOWN_JOB_TYPE : jobType.getName();
+
+ // Run all heuristics over the fetched data
+ List analysisResults = new ArrayList();
+ if (data == null || data.isEmpty()) {
+ // Example: a MR job has 0 mappers and 0 reducers
+ logger.info("No Data Received for analytic job: " + getAppId());
+ analysisResults.add(HeuristicResult.NO_DATA);
+ } else {
+ List heuristics = ElephantContext.instance().getHeuristicsForApplicationType(getAppType());
+ for (Heuristic heuristic : heuristics) {
+ String confExcludedApps = heuristic.getHeuristicConfData().getParamMap().get(EXCLUDE_JOBTYPE);
+
+ if (confExcludedApps == null || confExcludedApps.length() == 0 ||
+ !Arrays.asList(confExcludedApps.split(",")).contains(jobTypeName)) {
+ HeuristicResult result = heuristic.apply(data);
+ if (result != null) {
+ analysisResults.add(result);
+ }
+ }
+ }
+ }
+
+
+ HadoopMetricsAggregator hadoopMetricsAggregator = ElephantContext.instance().getAggregatorForApplicationType(getAppType());
+ hadoopMetricsAggregator.aggregate(data);
+ HadoopAggregatedData hadoopAggregatedData = hadoopMetricsAggregator.getResult();
+
+ // Load app information
+ AppResult result = new AppResult();
+ result.id = Utils.truncateField(getAppId(), AppResult.ID_LIMIT, getAppId());
+ result.trackingUrl = Utils.truncateField(getTrackingUrl(), AppResult.TRACKING_URL_LIMIT, getAppId());
+ result.queueName = Utils.truncateField(getQueueName(), AppResult.QUEUE_NAME_LIMIT, getAppId());
+ result.username = Utils.truncateField(getUser(), AppResult.USERNAME_LIMIT, getAppId());
+ result.startTime = getStartTime();
+ result.finishTime = getFinishTime();
+ result.name = Utils.truncateField(getName(), AppResult.APP_NAME_LIMIT, getAppId());
+ result.jobType = Utils.truncateField(jobTypeName, AppResult.JOBTYPE_LIMIT, getAppId());
+ result.resourceUsed = hadoopAggregatedData.getResourceUsed();
+ result.totalDelay = hadoopAggregatedData.getTotalDelay();
+ result.resourceWasted = hadoopAggregatedData.getResourceWasted();
+
+ // Load App Heuristic information
+ int jobScore = 0;
+ result.yarnAppHeuristicResults = new ArrayList();
+ Severity worstSeverity = Severity.NONE;
+ for (HeuristicResult heuristicResult : analysisResults) {
+ AppHeuristicResult detail = new AppHeuristicResult();
+ detail.heuristicClass = Utils.truncateField(heuristicResult.getHeuristicClassName(),
+ AppHeuristicResult.HEURISTIC_CLASS_LIMIT, getAppId());
+ detail.heuristicName = Utils.truncateField(heuristicResult.getHeuristicName(),
+ AppHeuristicResult.HEURISTIC_NAME_LIMIT, getAppId());
+ detail.severity = heuristicResult.getSeverity();
+ detail.score = heuristicResult.getScore();
+
+ // Load Heuristic Details
+ for (HeuristicResultDetails heuristicResultDetails : heuristicResult.getHeuristicResultDetails()) {
+ AppHeuristicResultDetails heuristicDetail = new AppHeuristicResultDetails();
+ heuristicDetail.yarnAppHeuristicResult = detail;
+ heuristicDetail.name = Utils.truncateField(heuristicResultDetails.getName(),
+ AppHeuristicResultDetails.NAME_LIMIT, getAppId());
+ heuristicDetail.value = Utils.truncateField(heuristicResultDetails.getValue(),
+ AppHeuristicResultDetails.VALUE_LIMIT, getAppId());
+ heuristicDetail.details = Utils.truncateField(heuristicResultDetails.getDetails(),
+ AppHeuristicResultDetails.DETAILS_LIMIT, getAppId());
+ // This was added for AnalyticTest. Commenting this out to fix a bug. Also disabling AnalyticJobTest.
+ //detail.yarnAppHeuristicResultDetails = new ArrayList();
+ detail.yarnAppHeuristicResultDetails.add(heuristicDetail);
+ }
+ result.yarnAppHeuristicResults.add(detail);
+ worstSeverity = Severity.max(worstSeverity, detail.severity);
+ jobScore += detail.score;
+ }
+ result.severity = worstSeverity;
+ result.score = jobScore;
+
+ // Retrieve information from job configuration like scheduler information and store them into result.
+ InfoExtractor.loadInfo(result, data);
+
+ return result;
+ }
+
+ /**
+ * Indicate this promise should retry itself again.
+ *
+ * @return true if should retry, else false
+ */
+ public boolean retry() {
+ return (_retries++) < _RETRY_LIMIT;
+ }
+}
diff --git a/app/com/linkedin/drelephant/analysis/AnalyticJobGenerator.java b/app/com/linkedin/drelephant/analysis/AnalyticJobGenerator.java
new file mode 100644
index 000000000..4015f0408
--- /dev/null
+++ b/app/com/linkedin/drelephant/analysis/AnalyticJobGenerator.java
@@ -0,0 +1,62 @@
+/*
+ * Copyright 2016 LinkedIn Corp.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License"); you may not
+ * use this file except in compliance with the License. You may obtain a copy of
+ * the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package com.linkedin.drelephant.analysis;
+
+import java.io.IOException;
+import java.util.List;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.security.authentication.client.AuthenticationException;
+
+
+/**
+ * Provides AnalyticJobs that will yield to analysis results later. This class basically generates to-dos that could be
+ * executed later.
+ */
+public interface AnalyticJobGenerator {
+
+ /**
+ * Configures the provider instance
+ *
+ * @param configuration The Hadoop configuration object
+ * @throws Exception
+ */
+ public void configure(Configuration configuration)
+ throws IOException;
+
+ /**
+ * Configures the resource manager addresses considering HA
+ */
+ public void updateResourceManagerAddresses();
+
+ /**
+ * Provides a list of AnalyticJobs that should be calculated
+ *
+ * @return a list of AnalyticJobs
+ * @throws IOException
+ * @throws AuthenticationException
+ */
+ public List fetchAnalyticJobs()
+ throws IOException, AuthenticationException;
+
+ /**
+ * Add an AnalyticJob into retry list. Those jobs will be provided again via #fetchAnalyticJobs under
+ * the generator's decision.
+ *
+ * @param job The job to add
+ */
+ public void addIntoRetries(AnalyticJob job);
+}
diff --git a/app/com/linkedin/drelephant/analysis/AnalyticJobGeneratorHadoop2.java b/app/com/linkedin/drelephant/analysis/AnalyticJobGeneratorHadoop2.java
new file mode 100644
index 000000000..1d49ec84d
--- /dev/null
+++ b/app/com/linkedin/drelephant/analysis/AnalyticJobGeneratorHadoop2.java
@@ -0,0 +1,247 @@
+/*
+ * Copyright 2016 LinkedIn Corp.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License"); you may not
+ * use this file except in compliance with the License. You may obtain a copy of
+ * the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package com.linkedin.drelephant.analysis;
+
+import com.linkedin.drelephant.ElephantContext;
+import com.linkedin.drelephant.math.Statistics;
+import controllers.MetricsController;
+import java.io.IOException;
+import java.net.HttpURLConnection;
+import java.net.URL;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Queue;
+import java.util.Random;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import models.AppResult;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.security.authentication.client.AuthenticatedURL;
+import org.apache.hadoop.security.authentication.client.AuthenticationException;
+import org.apache.log4j.Logger;
+import org.codehaus.jackson.JsonNode;
+import org.codehaus.jackson.map.ObjectMapper;
+
+
+/**
+ * This class provides a list of analysis promises to be generated under Hadoop YARN environment
+ */
+public class AnalyticJobGeneratorHadoop2 implements AnalyticJobGenerator {
+ private static final Logger logger = Logger.getLogger(AnalyticJobGeneratorHadoop2.class);
+ private static final String RESOURCE_MANAGER_ADDRESS = "yarn.resourcemanager.webapp.address";
+ private static final String IS_RM_HA_ENABLED = "yarn.resourcemanager.ha.enabled";
+ private static final String RESOURCE_MANAGER_IDS = "yarn.resourcemanager.ha.rm-ids";
+ private static final String RM_NODE_STATE_URL = "http://%s/ws/v1/cluster/info";
+ private static final String FETCH_INITIAL_WINDOW_MS = "drelephant.analysis.fetch.initial.windowMillis";
+
+ private static Configuration configuration;
+
+ // We provide one minute job fetch delay due to the job sending lag from AM/NM to JobHistoryServer HDFS
+ private static final long FETCH_DELAY = 60000;
+
+ // Generate a token update interval with a random deviation so that it does not update the token exactly at the same
+ // time with other token updaters (e.g. ElephantFetchers).
+ private static final long TOKEN_UPDATE_INTERVAL =
+ Statistics.MINUTE_IN_MS * 30 + new Random().nextLong() % (3 * Statistics.MINUTE_IN_MS);
+
+ private String _resourceManagerAddress;
+ private long _lastTime = 0;
+ private long _fetchStartTime = 0;
+ private long _currentTime = 0;
+ private long _tokenUpdatedTime = 0;
+ private AuthenticatedURL.Token _token;
+ private AuthenticatedURL _authenticatedURL;
+ private final ObjectMapper _objectMapper = new ObjectMapper();
+
+ private final Queue _retryQueue = new ConcurrentLinkedQueue();
+
+ public void updateResourceManagerAddresses() {
+ if (Boolean.valueOf(configuration.get(IS_RM_HA_ENABLED))) {
+ String resourceManagers = configuration.get(RESOURCE_MANAGER_IDS);
+ if (resourceManagers != null) {
+ logger.info("The list of RM IDs are " + resourceManagers);
+ List ids = Arrays.asList(resourceManagers.split(","));
+ _currentTime = System.currentTimeMillis();
+ updateAuthToken();
+ for (String id : ids) {
+ try {
+ String resourceManager = configuration.get(RESOURCE_MANAGER_ADDRESS + "." + id);
+ String resourceManagerURL = String.format(RM_NODE_STATE_URL, resourceManager);
+ logger.info("Checking RM URL: " + resourceManagerURL);
+ JsonNode rootNode = readJsonNode(new URL(resourceManagerURL));
+ String status = rootNode.path("clusterInfo").path("haState").getValueAsText();
+ if (status.equals("ACTIVE")) {
+ logger.info(resourceManager + " is ACTIVE");
+ _resourceManagerAddress = resourceManager;
+ break;
+ } else {
+ logger.info(resourceManager + " is STANDBY");
+ }
+ } catch (AuthenticationException e) {
+ logger.info("Error fetching resource manager " + id + " state " + e.getMessage());
+ } catch (IOException e) {
+ logger.info("Error fetching Json for resource manager "+ id + " status " + e.getMessage());
+ }
+ }
+ }
+ } else {
+ _resourceManagerAddress = configuration.get(RESOURCE_MANAGER_ADDRESS);
+ }
+ if (_resourceManagerAddress == null) {
+ throw new RuntimeException(
+ "Cannot get YARN resource manager address from Hadoop Configuration property: [" + RESOURCE_MANAGER_ADDRESS
+ + "].");
+ }
+ }
+
+ @Override
+ public void configure(Configuration configuration)
+ throws IOException {
+ this.configuration = configuration;
+ String initialFetchWindowString = configuration.get(FETCH_INITIAL_WINDOW_MS);
+ if (initialFetchWindowString != null) {
+ long initialFetchWindow = Long.getLong(initialFetchWindowString);
+ _lastTime = System.currentTimeMillis() - FETCH_DELAY - initialFetchWindow;
+ _fetchStartTime = _lastTime;
+ }
+ updateResourceManagerAddresses();
+ }
+
+ /**
+ * Fetch all the succeeded and failed applications/analytic jobs from the resource manager.
+ *
+ * @return
+ * @throws IOException
+ * @throws AuthenticationException
+ */
+ @Override
+ public List fetchAnalyticJobs()
+ throws IOException, AuthenticationException {
+ List appList = new ArrayList();
+
+ // There is a lag of job data from AM/NM to JobHistoryServer HDFS, we shouldn't use the current time, since there
+ // might be new jobs arriving after we fetch jobs. We provide one minute delay to address this lag.
+ _currentTime = System.currentTimeMillis() - FETCH_DELAY;
+ updateAuthToken();
+
+ logger.info("Fetching recent finished application runs between last time: " + (_lastTime + 1)
+ + ", and current time: " + _currentTime);
+
+ // Fetch all succeeded apps
+ URL succeededAppsURL = new URL(new URL("http://" + _resourceManagerAddress), String.format(
+ "/ws/v1/cluster/apps?finalStatus=SUCCEEDED&finishedTimeBegin=%s&finishedTimeEnd=%s",
+ String.valueOf(_lastTime + 1), String.valueOf(_currentTime)));
+ logger.info("The succeeded apps URL is " + succeededAppsURL);
+ List succeededApps = readApps(succeededAppsURL);
+ appList.addAll(succeededApps);
+
+ // Fetch all failed apps
+ // state: Application Master State
+ // finalStatus: Status of the Application as reported by the Application Master
+ URL failedAppsURL = new URL(new URL("http://" + _resourceManagerAddress), String.format(
+ "/ws/v1/cluster/apps?finalStatus=FAILED&state=FINISHED&finishedTimeBegin=%s&finishedTimeEnd=%s",
+ String.valueOf(_lastTime + 1), String.valueOf(_currentTime)));
+ List failedApps = readApps(failedAppsURL);
+ logger.info("The failed apps URL is " + failedAppsURL);
+ appList.addAll(failedApps);
+
+ // Append promises from the retry queue at the end of the list
+ while (!_retryQueue.isEmpty()) {
+ appList.add(_retryQueue.poll());
+ }
+
+ _lastTime = _currentTime;
+ return appList;
+ }
+
+ @Override
+ public void addIntoRetries(AnalyticJob promise) {
+ _retryQueue.add(promise);
+ int retryQueueSize = _retryQueue.size();
+ MetricsController.setRetryQueueSize(retryQueueSize);
+ logger.info("Retry queue size is " + retryQueueSize);
+ }
+
+ /**
+ * Authenticate and update the token
+ */
+ private void updateAuthToken() {
+ if (_currentTime - _tokenUpdatedTime > TOKEN_UPDATE_INTERVAL) {
+ logger.info("AnalysisProvider updating its Authenticate Token...");
+ _token = new AuthenticatedURL.Token();
+ _authenticatedURL = new AuthenticatedURL();
+ _tokenUpdatedTime = _currentTime;
+ }
+ }
+
+ /**
+ * Connect to url using token and return the JsonNode
+ *
+ * @param url The url to connect to
+ * @return
+ * @throws IOException Unable to get the stream
+ * @throws AuthenticationException Authencation problem
+ */
+ private JsonNode readJsonNode(URL url)
+ throws IOException, AuthenticationException {
+ HttpURLConnection conn = _authenticatedURL.openConnection(url, _token);
+ return _objectMapper.readTree(conn.getInputStream());
+ }
+
+ /**
+ * Parse the returned json from Resource manager
+ *
+ * @param url The REST call
+ * @return
+ * @throws IOException
+ * @throws AuthenticationException Problem authenticating to resource manager
+ */
+ private List readApps(URL url) throws IOException, AuthenticationException{
+ List appList = new ArrayList();
+
+ JsonNode rootNode = readJsonNode(url);
+ JsonNode apps = rootNode.path("apps").path("app");
+
+ for (JsonNode app : apps) {
+ String appId = app.get("id").getValueAsText();
+
+ // When called first time after launch, hit the DB and avoid duplicated analytic jobs that have been analyzed
+ // before.
+ if (_lastTime > _fetchStartTime || (_lastTime == _fetchStartTime && AppResult.find.byId(appId) == null)) {
+ String user = app.get("user").getValueAsText();
+ String name = app.get("name").getValueAsText();
+ String queueName = app.get("queue").getValueAsText();
+ String trackingUrl = app.get("trackingUrl") != null? app.get("trackingUrl").getValueAsText() : null;
+ long startTime = app.get("startedTime").getLongValue();
+ long finishTime = app.get("finishedTime").getLongValue();
+
+ ApplicationType type =
+ ElephantContext.instance().getApplicationTypeForName(app.get("applicationType").getValueAsText());
+
+ // If the application type is supported
+ if (type != null) {
+ AnalyticJob analyticJob = new AnalyticJob();
+ analyticJob.setAppId(appId).setAppType(type).setUser(user).setName(name).setQueueName(queueName)
+ .setTrackingUrl(trackingUrl).setStartTime(startTime).setFinishTime(finishTime);
+
+ appList.add(analyticJob);
+ }
+ }
+ }
+ return appList;
+ }
+}
diff --git a/app/com/linkedin/drelephant/analysis/ApplicationType.java b/app/com/linkedin/drelephant/analysis/ApplicationType.java
new file mode 100644
index 000000000..6280cddf9
--- /dev/null
+++ b/app/com/linkedin/drelephant/analysis/ApplicationType.java
@@ -0,0 +1,50 @@
+/*
+ * Copyright 2016 LinkedIn Corp.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License"); you may not
+ * use this file except in compliance with the License. You may obtain a copy of
+ * the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package com.linkedin.drelephant.analysis;
+
+/**
+ * Manages and represents supported application types.
+ */
+public class ApplicationType {
+ private final String _name;
+
+ public ApplicationType(String name) {
+ _name = name.toUpperCase();
+ }
+
+ @Override
+ public int hashCode() {
+ return _name.hashCode();
+ }
+
+ @Override
+ public boolean equals(Object other) {
+ if (other instanceof ApplicationType) {
+ return ((ApplicationType) other).getName().equals(getName());
+ }
+ return false;
+ }
+
+ /**
+ * Get the name
+ *
+ * @return the name of the application type
+ */
+ public String getName() {
+ return _name;
+ }
+}
diff --git a/app/com/linkedin/drelephant/analysis/Constants.java b/app/com/linkedin/drelephant/analysis/Constants.java
deleted file mode 100644
index 9e481cceb..000000000
--- a/app/com/linkedin/drelephant/analysis/Constants.java
+++ /dev/null
@@ -1,25 +0,0 @@
-package com.linkedin.drelephant.analysis;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.log4j.Logger;
-
-import java.io.IOException;
-
-public class Constants {
- private static final Logger logger = Logger.getLogger(Constants.class);
- public static long HDFS_BLOCK_SIZE = 64 * 1024 * 1024;
- public static final long DISK_READ_SPEED = 100 * 1024 * 1024;
- public static final int SHUFFLE_SORT_MAX_SAMPLE_SIZE = 50;
-
- public static void load() {
- try {
- HDFS_BLOCK_SIZE = FileSystem.get(new Configuration()).getDefaultBlockSize(new Path("/"));
- } catch (IOException e) {
- logger.error("Error getting FS Block Size!", e);
- }
-
- logger.info("HDFS BLock size: " + HDFS_BLOCK_SIZE);
- }
-}
diff --git a/app/com/linkedin/drelephant/analysis/ElephantFetcher.java b/app/com/linkedin/drelephant/analysis/ElephantFetcher.java
new file mode 100644
index 000000000..d83ff90ad
--- /dev/null
+++ b/app/com/linkedin/drelephant/analysis/ElephantFetcher.java
@@ -0,0 +1,35 @@
+/*
+ * Copyright 2016 LinkedIn Corp.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License"); you may not
+ * use this file except in compliance with the License. You may obtain a copy of
+ * the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package com.linkedin.drelephant.analysis;
+
+/**
+ * The interface to define common methods for each fetcher.
+ *
+ * There would be a different fetcher implementation given a different Hadoop version and a different application type.
+ */
+public interface ElephantFetcher {
+
+ /**
+ * Given an application id, fetches the data object
+ *
+ * @param job The job being analysed
+ * @return the fetched data
+ * @throws Exception
+ */
+ public T fetchData(AnalyticJob job)
+ throws Exception;
+}
\ No newline at end of file
diff --git a/app/com/linkedin/drelephant/analysis/HDFSContext.java b/app/com/linkedin/drelephant/analysis/HDFSContext.java
new file mode 100644
index 000000000..345751d67
--- /dev/null
+++ b/app/com/linkedin/drelephant/analysis/HDFSContext.java
@@ -0,0 +1,51 @@
+/*
+ * Copyright 2016 LinkedIn Corp.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License"); you may not
+ * use this file except in compliance with the License. You may obtain a copy of
+ * the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package com.linkedin.drelephant.analysis;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.log4j.Logger;
+import java.io.IOException;
+
+
+/**
+ * The HDFS Information
+ */
+public final class HDFSContext {
+ private static final Logger logger = Logger.getLogger(HDFSContext.class);
+
+ public static long HDFS_BLOCK_SIZE = 64 * 1024 * 1024;
+ public static final long DISK_READ_SPEED = 100 * 1024 * 1024;
+
+ private HDFSContext() {
+ // Empty on purpose
+ }
+
+ /**
+ * Captures the HDFS Block Size
+ */
+ public static void load() {
+ try {
+ HDFS_BLOCK_SIZE = FileSystem.get(new Configuration()).getDefaultBlockSize(new Path("/"));
+ } catch (IOException e) {
+ logger.error("Error getting FS Block Size!", e);
+ }
+
+ logger.info("HDFS BLock size: " + HDFS_BLOCK_SIZE);
+ }
+}
diff --git a/app/com/linkedin/drelephant/analysis/HadoopAggregatedData.java b/app/com/linkedin/drelephant/analysis/HadoopAggregatedData.java
new file mode 100644
index 000000000..2c88f3d7b
--- /dev/null
+++ b/app/com/linkedin/drelephant/analysis/HadoopAggregatedData.java
@@ -0,0 +1,76 @@
+/*
+ * Copyright 2016 LinkedIn Corp.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License"); you may not
+ * use this file except in compliance with the License. You may obtain a copy of
+ * the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package com.linkedin.drelephant.analysis;
+
+/**
+ * This class contains the aggregated data of a job
+ */
+public class HadoopAggregatedData {
+
+ private long resourceUsed = 0;
+ private long resourceWasted = 0;
+ private long totalDelay = 0;
+
+ /**
+ * Returns the resource usage (in MBSeconds) of the job
+ * @return The resource usage (in MBSeconds) of the job
+ */
+ public long getResourceUsed() {
+ return resourceUsed;
+ }
+
+ /**
+ * Setter for the resource usage (in MBSeconds) of the job
+ * @param resourceUsed The resource usage (in MBSeconds) of the job
+ */
+ public void setResourceUsed(long resourceUsed) {
+ this.resourceUsed = resourceUsed;
+ }
+
+ /**
+ * Returns the wasted resources (in MBSeconds) of the job
+ * @return The wasted resources (in MBSeconds) of the job
+ */
+ public long getResourceWasted() {
+ return resourceWasted;
+ }
+
+ /**
+ * Setter for the wasted resources (in MBSeconds)
+ * @param resourceWasted The wasted resources (in MBSeconds) of the job
+ */
+ public void setResourceWasted(long resourceWasted) {
+ this.resourceWasted = resourceWasted;
+ }
+
+ /**
+ * returns the total delay of the job
+ * @return The total delay of the job
+ */
+ public long getTotalDelay() {
+ return totalDelay;
+ }
+
+ /**
+ * Setter for the total delay of the job
+ * @param totalDelay The total delay of the job
+ */
+ public void setTotalDelay(long totalDelay) {
+ this.totalDelay = totalDelay;
+ }
+
+}
diff --git a/app/com/linkedin/drelephant/analysis/HadoopApplicationData.java b/app/com/linkedin/drelephant/analysis/HadoopApplicationData.java
new file mode 100644
index 000000000..35dc5457f
--- /dev/null
+++ b/app/com/linkedin/drelephant/analysis/HadoopApplicationData.java
@@ -0,0 +1,54 @@
+/*
+ * Copyright 2016 LinkedIn Corp.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License"); you may not
+ * use this file except in compliance with the License. You may obtain a copy of
+ * the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package com.linkedin.drelephant.analysis;
+
+import java.util.Properties;
+
+
+/**
+ * This interface indicates that a class is holding the information of a Hadoop application
+ */
+public interface HadoopApplicationData {
+
+ /**
+ * Returns the unique id to identify an application run.
+ *
+ * @return the id
+ */
+ public String getAppId();
+
+ /**
+ * Returns the configuration of an application.
+ *
+ * @return A java Properties that contains the application configuration
+ */
+ public Properties getConf();
+
+ /**
+ * Returns the application type this data is for
+ *
+ * @return the application type
+ */
+ public ApplicationType getApplicationType();
+
+ /**
+ * Indicate if the data holder is actually empty (nothing is set up).
+ *
+ * @return true if the data holder is empty else false
+ */
+ public boolean isEmpty();
+}
diff --git a/app/com/linkedin/drelephant/analysis/HadoopMetricsAggregator.java b/app/com/linkedin/drelephant/analysis/HadoopMetricsAggregator.java
new file mode 100644
index 000000000..3830fe008
--- /dev/null
+++ b/app/com/linkedin/drelephant/analysis/HadoopMetricsAggregator.java
@@ -0,0 +1,22 @@
+/*
+ * Copyright 2016 LinkedIn Corp.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License"); you may not
+ * use this file except in compliance with the License. You may obtain a copy of
+ * the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package com.linkedin.drelephant.analysis;
+
+public interface HadoopMetricsAggregator {
+ public void aggregate(HadoopApplicationData data);
+ public HadoopAggregatedData getResult();
+}
diff --git a/app/com/linkedin/drelephant/analysis/HadoopSystemContext.java b/app/com/linkedin/drelephant/analysis/HadoopSystemContext.java
new file mode 100644
index 000000000..0d1a37f1c
--- /dev/null
+++ b/app/com/linkedin/drelephant/analysis/HadoopSystemContext.java
@@ -0,0 +1,50 @@
+/*
+ * Copyright 2016 LinkedIn Corp.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License"); you may not
+ * use this file except in compliance with the License. You may obtain a copy of
+ * the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package com.linkedin.drelephant.analysis;
+
+import org.apache.hadoop.conf.Configuration;
+
+
+/**
+ * Hadoop System Information
+ */
+public final class HadoopSystemContext {
+
+ private static final String MAPREDUCE_FRAMEWORK_NAME_PROP = "mapreduce.framework.name";
+ private static final String YARN = "yarn";
+
+ /**
+ * Detect if the current Hadoop environment is 2.x
+ *
+ * @return true if it is Hadoop 2 env, else false
+ */
+ public static boolean isHadoop2Env() {
+ Configuration hadoopConf = new Configuration();
+ String hadoopVersion = hadoopConf.get(MAPREDUCE_FRAMEWORK_NAME_PROP);
+ return hadoopVersion != null && hadoopVersion.equals(YARN);
+ }
+
+ /**
+ * Check if a Hadoop version matches the current Hadoop environment
+ *
+ * @param majorVersion the major version number of hadoop
+ * @return true if we have a major version match else false
+ */
+ public static boolean matchCurrentHadoopVersion(int majorVersion) {
+ return majorVersion == 2 && isHadoop2Env();
+ }
+}
diff --git a/app/com/linkedin/drelephant/analysis/Heuristic.java b/app/com/linkedin/drelephant/analysis/Heuristic.java
index 073e90171..31e7a0a67 100644
--- a/app/com/linkedin/drelephant/analysis/Heuristic.java
+++ b/app/com/linkedin/drelephant/analysis/Heuristic.java
@@ -1,9 +1,42 @@
+/*
+ * Copyright 2016 LinkedIn Corp.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License"); you may not
+ * use this file except in compliance with the License. You may obtain a copy of
+ * the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
package com.linkedin.drelephant.analysis;
-import com.linkedin.drelephant.hadoop.HadoopJobData;
+import com.linkedin.drelephant.configurations.heuristic.HeuristicConfigurationData;
+
-public interface Heuristic {
- public HeuristicResult apply(HadoopJobData data);
+/**
+ * This interface defines the Heuristic rule interface.
+ *
+ * @param An implementation that extends from HadoopApplicationData
+ */
+public interface Heuristic {
+ /**
+ * Given an application data instance, returns the analyzed heuristic result.
+ *
+ * @param data The data to analyze
+ * @return The heuristic result
+ */
+ public HeuristicResult apply(T data);
- public String getHeuristicName();
+ /**
+ * Get the heuristic Configuration
+ *
+ * @return the heuristic configuration data
+ */
+ public HeuristicConfigurationData getHeuristicConfData();
}
diff --git a/app/com/linkedin/drelephant/analysis/HeuristicResult.java b/app/com/linkedin/drelephant/analysis/HeuristicResult.java
index c716f665d..b4519e982 100644
--- a/app/com/linkedin/drelephant/analysis/HeuristicResult.java
+++ b/app/com/linkedin/drelephant/analysis/HeuristicResult.java
@@ -1,66 +1,141 @@
+/*
+ * Copyright 2016 LinkedIn Corp.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License"); you may not
+ * use this file except in compliance with the License. You may obtain a copy of
+ * the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
package com.linkedin.drelephant.analysis;
-import com.linkedin.drelephant.util.Utils;
import java.util.ArrayList;
+import java.util.Collections;
import java.util.List;
+import org.apache.commons.lang.StringUtils;
+
+/**
+ * Holds the Heuristic analysis result Information
+ */
public class HeuristicResult {
- private String analysis;
- private Severity severity;
- private List details;
- private int detailsColumns = 0;
-
- public HeuristicResult(String analysis, Severity severity) {
- this.analysis = analysis;
- this.severity = severity;
- this.details = new ArrayList();
- }
-
- public String getAnalysis() {
- return analysis;
- }
-
- public Severity getSeverity() {
- return severity;
- }
-
- /**
- * Gets a list of lines of comma-separated strings
- *
- * @return
- */
- public List getDetails() {
- return details;
- }
-
- /**
- * Create a string that contains lines of comma-separated strings
- *
- * @return
- */
- public String getDetailsCSV() {
- return Utils.combineCsvLines(details.toArray(new String[details.size()]));
- }
-
- /**
- * Gets the number of columns in the csv formatted details store
- *
- * @return
- */
- public int getDetailsColumns() {
- return detailsColumns;
- }
-
- /**
- * Add a new line to the csv formatted details store
- *
- * @param parts strings to join into a single line
- */
- public void addDetail(String... parts) {
- details.add(Utils.createCsvLine(parts));
- if (parts.length > detailsColumns) {
- detailsColumns = parts.length;
- }
- }
+ public static final HeuristicResult NO_DATA = new HeuristicResult("NoDataReceived", "No Data Received", Severity.LOW,
+ 0, Collections.singletonList(new HeuristicResultDetails("No Data Received", "", null)));
+
+ private String _heuristicClass;
+ private String _heuristicName;
+ private Severity _severity;
+ private int _score;
+ private List _heuristicResultDetails;
+
+ /**
+ * Heuristic Result Constructor
+ *
+ * @param heuristicClass The Heuristic class
+ * @param heuristicName The name of the Heursitic
+ * @param severity The severity of the result
+ * @param score The computed score
+ */
+ public HeuristicResult(String heuristicClass, String heuristicName, Severity severity, int score) {
+ this._heuristicClass = heuristicClass;
+ this._heuristicName = heuristicName;
+ this._severity = severity;
+ this._score = score;
+ this._heuristicResultDetails = new ArrayList();
+ }
+
+ /**
+ * Heuristic Result Constructor
+ *
+ * @param heuristicClass The Heuristic class
+ * @param heuristicName The name of the Heursitic
+ * @param severity The severity of the result
+ * @param score The computed score
+ * @param heuristicResultDetails more information on the heuristic details.
+ */
+ public HeuristicResult(String heuristicClass, String heuristicName, Severity severity, int score,
+ List heuristicResultDetails) {
+ this._heuristicClass = heuristicClass;
+ this._heuristicName = heuristicName;
+ this._severity = severity;
+ this._score = score;
+ this._heuristicResultDetails = heuristicResultDetails;
+ }
+
+ /**
+ * Returns the heuristic analyser class name
+ *
+ * @return the heursitic class name
+ */
+ public String getHeuristicClassName() {
+ return _heuristicClass;
+ }
+
+ /**
+ * Returns the heuristic analyser name
+ *
+ * @return the heuristic name
+ */
+ public String getHeuristicName() {
+ return _heuristicName;
+ }
+
+ /**
+ * Returns the severity of the Heuristic
+ *
+ * @return The severity
+ */
+ public Severity getSeverity() {
+ return _severity;
+ }
+
+ public int getScore() {
+ return _score;
+ }
+
+ /**
+ * Gets a list of HeuristicResultDetails
+ *
+ * @return
+ */
+ public List getHeuristicResultDetails() {
+ return _heuristicResultDetails;
+ }
+
+ /**
+ * Add the App Heuristic Result Detail entry
+ */
+ public void addResultDetail(String name, String value, String details) {
+ _heuristicResultDetails.add(new HeuristicResultDetails(name, value, details));
+ }
+
+ /**
+ * Add the App Heuristic Result Detail without details
+ */
+ public void addResultDetail(String name, String value) {
+ _heuristicResultDetails.add(new HeuristicResultDetails(name, value, null));
+ }
+
+ /**
+ * Set the severity of the heuristic
+ *
+ * @param severity The severity to be set
+ */
+ public void setSeverity(Severity severity) {
+ this._severity = severity;
+ }
+
+ @Override
+ public String toString() {
+ return "{analysis: " + _heuristicClass + ", severity: " + _severity + ", details: ["
+ + StringUtils.join(_heuristicResultDetails, " ") + "]}";
+ }
}
diff --git a/app/com/linkedin/drelephant/analysis/HeuristicResultDetails.java b/app/com/linkedin/drelephant/analysis/HeuristicResultDetails.java
new file mode 100644
index 000000000..caf2d8be6
--- /dev/null
+++ b/app/com/linkedin/drelephant/analysis/HeuristicResultDetails.java
@@ -0,0 +1,53 @@
+/*
+ * Copyright 2016 LinkedIn Corp.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License"); you may not
+ * use this file except in compliance with the License. You may obtain a copy of
+ * the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package com.linkedin.drelephant.analysis;
+
+import java.util.ArrayList;
+import java.util.List;
+
+
+/**
+ * Holds the analysis details for each Heuristic
+ */
+public class HeuristicResultDetails {
+
+ private String _name;
+ private String _value;
+ private String _details;
+
+ public HeuristicResultDetails(String name, String value) {
+ this(name, value, null);
+ }
+
+ public HeuristicResultDetails(String name, String value, String details) {
+ this._name = name;
+ this._value = value;
+ this._details = details;
+ }
+
+ public String getDetails() {
+ return _details;
+ }
+
+ public String getValue() {
+ return _value;
+ }
+
+ public String getName() {
+ return _name;
+ }
+}
diff --git a/app/com/linkedin/drelephant/analysis/JobType.java b/app/com/linkedin/drelephant/analysis/JobType.java
new file mode 100644
index 000000000..44cb8cb1d
--- /dev/null
+++ b/app/com/linkedin/drelephant/analysis/JobType.java
@@ -0,0 +1,74 @@
+/*
+ * Copyright 2016 LinkedIn Corp.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License"); you may not
+ * use this file except in compliance with the License. You may obtain a copy of
+ * the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package com.linkedin.drelephant.analysis;
+
+import java.util.Properties;
+import java.util.regex.Pattern;
+
+
+/**
+ * Job Type holder. Includes for each Job type, the configuration key that uniquely identifies that type and the
+ * regex pattern to match the property.
+ */
+public class JobType {
+ private final String _name;
+ private final String _confName;
+ private final Pattern _confPattern;
+
+ /**
+ * Constructor for a JobType
+ *
+ * @param name The name of the job type
+ * @param confName The configuration to look into
+ * @param confPattern The regex pattern to match the configuration property
+ */
+ public JobType(String name, String confName, String confPattern) {
+ _name = name;
+ _confName = confName;
+ _confPattern = Pattern.compile(confPattern);
+ }
+
+ /**
+ * Check if a JobType matches a property
+ *
+ * @param jobProp The properties to match
+ * @return true if matched else false
+ */
+ public boolean matchType(Properties jobProp) {
+ // Always return false if confName/confPattern is undefined,
+ // which means we cannot tell if the properties are matching the pattern
+ if (_confName == null || _confPattern == null) {
+ return false;
+ }
+
+ return jobProp.containsKey(_confName) && _confPattern.matcher((String) jobProp.get(_confName)).matches();
+ }
+
+ /**
+ * Get the name of the job type
+ *
+ * @return The name
+ */
+ public String getName() {
+ return _name;
+ }
+
+ @Override
+ public String toString() {
+ return getName();
+ }
+}
diff --git a/app/com/linkedin/drelephant/analysis/Metrics.java b/app/com/linkedin/drelephant/analysis/Metrics.java
new file mode 100644
index 000000000..2ce3e8e0e
--- /dev/null
+++ b/app/com/linkedin/drelephant/analysis/Metrics.java
@@ -0,0 +1,60 @@
+/*
+ * Copyright 2016 LinkedIn Corp.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License"); you may not
+ * use this file except in compliance with the License. You may obtain a copy of
+ * the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package com.linkedin.drelephant.analysis;
+
+public enum Metrics {
+
+ // Currently supported metrics
+ USED_RESOURCES("Used Resources", "resources", "The resources used by the job"),
+ WASTED_RESOURCES("Wasted Resources", "resources", "The resources wasted by the job"),
+ RUNTIME("Run Time", "time", "The run time of the job"),
+ WAIT_TIME("Wait Time", "time", "The wait time of the job");
+
+ private String text;
+ private String type;
+ private String description;
+
+ Metrics(String text, String type, String description) {
+ this.text = text;
+ this.type = type;
+ this.description = description;
+ }
+
+ /**
+ * Returns the value of the text for the metrics
+ * @return The text value
+ */
+ public String getText() {
+ return text;
+ }
+
+ /**
+ * Returns the type of the metrics. It can be one of resources or time
+ * @return The type of the metrics.
+ */
+ public String getType() {
+ return type;
+ }
+
+ /**
+ * Returns the description of the metrics
+ * @return The description of the metrics
+ */
+ public String getDescription() {
+ return description;
+ }
+}
diff --git a/app/com/linkedin/drelephant/analysis/Severity.java b/app/com/linkedin/drelephant/analysis/Severity.java
index ba0a1635e..8a35ca91a 100644
--- a/app/com/linkedin/drelephant/analysis/Severity.java
+++ b/app/com/linkedin/drelephant/analysis/Severity.java
@@ -1,105 +1,193 @@
+/*
+ * Copyright 2016 LinkedIn Corp.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License"); you may not
+ * use this file except in compliance with the License. You may obtain a copy of
+ * the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
package com.linkedin.drelephant.analysis;
import com.avaje.ebean.annotation.EnumValue;
-public enum Severity {
- @EnumValue("4")
- CRITICAL(4, "Critical", "danger"),
-
- @EnumValue("3")
- SEVERE(3, "Severe", "severe"),
-
- @EnumValue("2")
- MODERATE(2, "Moderate", "warning"),
-
- @EnumValue("1")
- LOW(1, "Low", "success"),
- @EnumValue("0")
- NONE(0, "None", "success");
-
- private int value;
- private String text;
- private String bootstrapColor;
-
- Severity(int value, String text, String bootstrapColor) {
- this.value = value;
- this.text = text;
- this.bootstrapColor = bootstrapColor;
+/**
+ * The severities that you can use
+ */
+public enum Severity {
+ @EnumValue("4")
+ CRITICAL(4, "Critical", "danger"),
+
+ @EnumValue("3")
+ SEVERE(3, "Severe", "severe"),
+
+ @EnumValue("2")
+ MODERATE(2, "Moderate", "warning"),
+
+ @EnumValue("1")
+ LOW(1, "Low", "success"),
+
+ @EnumValue("0")
+ NONE(0, "None", "success");
+
+ private int _value;
+ private String _text;
+ private String _bootstrapColor;
+
+ /**
+ * @param value The severity value
+ * @param text The severity name
+ * @param bootstrapColor The severity level for color coding
+ */
+ Severity(int value, String text, String bootstrapColor) {
+ this._value = value;
+ this._text = text;
+ this._bootstrapColor = bootstrapColor;
+ }
+
+ /**
+ * Returns the severity level
+ *
+ * @return The severity value (0 to 5)
+ */
+ public int getValue() {
+ return _value;
+ }
+
+ /**
+ * Returns the Severity level Name
+ *
+ * @return Severity level (None, Low, Moderate, Sever, Critical)
+ */
+ public String getText() {
+ return _text;
+ }
+
+ /**
+ * Returns the severity level for color coding
+ *
+ * @return The severity level (color)
+ */
+ public String getBootstrapColor() {
+ return _bootstrapColor;
+ }
+
+ /**
+ * Returns the Severity corresponding to the severity value, NONE severity otherwise
+ *
+ * @param value The severity values (0 to 5)
+ * @return The severity
+ */
+ public static Severity byValue(int value) {
+ for (Severity severity : values()) {
+ if (severity._value == value) {
+ return severity;
+ }
}
-
- public int getValue() {
- return value;
+ return NONE;
+ }
+
+ /**
+ * Returns the maximum of the severities
+ *
+ * @param a One severity
+ * @param b The other severity
+ * @return Max(a,b)
+ */
+ public static Severity max(Severity a, Severity b) {
+ if (a._value > b._value) {
+ return a;
}
-
- public String getText() {
- return text;
+ return b;
+ }
+
+ /**
+ * Returns the maximum of the severities in the array
+ *
+ * @param severities Arbitrary number of severities
+ * @return Max(severities)
+ */
+ public static Severity max(Severity... severities) {
+ Severity currentSeverity = NONE;
+ for (Severity severity : severities) {
+ currentSeverity = max(currentSeverity, severity);
}
-
- public String getBootstrapColor() {
- return bootstrapColor;
+ return currentSeverity;
+ }
+
+ /**
+ * Returns the minimum of the severities
+ *
+ * @param a One severity
+ * @param b The other severity
+ * @return Min(a,b)
+ */
+ public static Severity min(Severity a, Severity b) {
+ if (a._value < b._value) {
+ return a;
}
-
- public static Severity byValue(int value) {
- for (Severity severity : values()) {
- if (severity.value == value) {
- return severity;
- }
- }
- return NONE;
+ return b;
+ }
+
+ /**
+ * Returns the severity level of the value in the given thresholds
+ * low < moderate < severe < critical
+ *
+ * Critical when value is greater than the critical threshold
+ * None when the value is less than the low threshold.
+ *
+ * @param value The value being tested
+ * @return One of the 5 severity levels
+ */
+ public static Severity getSeverityAscending(Number value, Number low, Number moderate, Number severe,
+ Number critical) {
+ if (value.doubleValue() >= critical.doubleValue()) {
+ return CRITICAL;
}
-
- public static Severity max(Severity a, Severity b) {
- if (a.value > b.value) {
- return a;
- }
- return b;
+ if (value.doubleValue() >= severe.doubleValue()) {
+ return SEVERE;
}
-
- public static Severity max(Severity... severities) {
- Severity currentSeverity = NONE;
- for (Severity severity : severities) {
- currentSeverity = max(currentSeverity, severity);
- }
- return currentSeverity;
+ if (value.doubleValue() >= moderate.doubleValue()) {
+ return MODERATE;
}
-
- public static Severity min(Severity a, Severity b) {
- if (a.value < b.value) {
- return a;
- }
- return b;
+ if (value.doubleValue() >= low.doubleValue()) {
+ return LOW;
}
-
- public static Severity getSeverityAscending(long value, long low, long moderate, long severe, long critical) {
- if (value >= critical) {
- return CRITICAL;
- }
- if (value >= severe) {
- return SEVERE;
- }
- if (value >= moderate) {
- return MODERATE;
- }
- if (value >= low) {
- return LOW;
- }
- return NONE;
+ return NONE;
+ }
+
+ /**
+ * Returns the severity level of the value in the given thresholds
+ * low > moderate > severe > critical
+ *
+ * Critical when value is less than the critical threshold
+ * None when the value is greater than the low threshold.
+ *
+ * @param value The value being tested
+ * @return One of the 5 severity levels
+ */
+ public static Severity getSeverityDescending(Number value, Number low, Number moderate, Number severe,
+ Number critical) {
+ if (value.doubleValue() <= critical.doubleValue()) {
+ return CRITICAL;
}
-
- public static Severity getSeverityDescending(long value, long low, long moderate, long severe, long critical) {
- if (value <= critical) {
- return CRITICAL;
- }
- if (value <= severe) {
- return SEVERE;
- }
- if (value <= moderate) {
- return MODERATE;
- }
- if (value <= low) {
- return LOW;
- }
- return NONE;
+ if (value.doubleValue() <= severe.doubleValue()) {
+ return SEVERE;
+ }
+ if (value.doubleValue() <= moderate.doubleValue()) {
+ return MODERATE;
+ }
+ if (value.doubleValue() <= low.doubleValue()) {
+ return LOW;
}
+ return NONE;
+ }
}
diff --git a/app/com/linkedin/drelephant/analysis/SeverityThresholds.scala b/app/com/linkedin/drelephant/analysis/SeverityThresholds.scala
new file mode 100644
index 000000000..7359c7cac
--- /dev/null
+++ b/app/com/linkedin/drelephant/analysis/SeverityThresholds.scala
@@ -0,0 +1,53 @@
+/*
+ * Copyright 2016 LinkedIn Corp.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License"); you may not
+ * use this file except in compliance with the License. You may obtain a copy of
+ * the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package com.linkedin.drelephant.analysis
+
+import com.linkedin.drelephant.util.Utils
+
+
+/**
+ * A convenience case class for containing severity thresholds and calculating severity.
+ */
+case class SeverityThresholds(low: Number, moderate: Number, severe: Number, critical: Number, ascending: Boolean) {
+ if (ascending) {
+ require(low.doubleValue <= moderate.doubleValue)
+ require(moderate.doubleValue <= severe.doubleValue)
+ require(severe.doubleValue <= critical.doubleValue)
+ } else {
+ require(low.doubleValue >= moderate.doubleValue)
+ require(moderate.doubleValue >= severe.doubleValue)
+ require(severe.doubleValue >= critical.doubleValue)
+ }
+
+ def severityOf(value: Number): Severity = if (ascending) {
+ Severity.getSeverityAscending(value, low, moderate, severe, critical)
+ } else {
+ Severity.getSeverityDescending(value, low, moderate, severe, critical)
+ }
+}
+
+object SeverityThresholds {
+ val NUM_THRESHOLDS = 4
+
+ /** Returns a SeverityThresholds object from a Dr. Elephant configuration string parseable by Utils.getParam(String, int). */
+ def parse(
+ rawString: String,
+ ascending: Boolean
+ ): Option[SeverityThresholds] = Option(Utils.getParam(rawString, NUM_THRESHOLDS)).map { thresholds =>
+ SeverityThresholds(low = thresholds(0), moderate = thresholds(1), severe = thresholds(2), critical = thresholds(3), ascending)
+ }
+}
diff --git a/app/com/linkedin/drelephant/analysis/heuristics/GenericDataSkewHeuristic.java b/app/com/linkedin/drelephant/analysis/heuristics/GenericDataSkewHeuristic.java
deleted file mode 100644
index ad9f63c51..000000000
--- a/app/com/linkedin/drelephant/analysis/heuristics/GenericDataSkewHeuristic.java
+++ /dev/null
@@ -1,82 +0,0 @@
-package com.linkedin.drelephant.analysis.heuristics;
-
-import com.linkedin.drelephant.analysis.Constants;
-import com.linkedin.drelephant.analysis.Heuristic;
-import com.linkedin.drelephant.analysis.HeuristicResult;
-import com.linkedin.drelephant.analysis.Severity;
-import com.linkedin.drelephant.hadoop.HadoopCounterHolder;
-import com.linkedin.drelephant.hadoop.HadoopJobData;
-import com.linkedin.drelephant.hadoop.HadoopTaskData;
-import com.linkedin.drelephant.math.Statistics;
-import org.apache.commons.io.FileUtils;
-
-public abstract class GenericDataSkewHeuristic implements Heuristic {
- private HadoopCounterHolder.CounterName counterName;
- private String heuristicName;
-
- @Override
- public String getHeuristicName() {
- return heuristicName;
- }
-
- protected GenericDataSkewHeuristic(HadoopCounterHolder.CounterName counterName, String heuristicName) {
- this.counterName = counterName;
- this.heuristicName = heuristicName;
- }
-
- protected abstract HadoopTaskData[] getTasks(HadoopJobData data);
-
- @Override
- public HeuristicResult apply(HadoopJobData data) {
- HadoopTaskData[] tasks = getTasks(data);
-
- //Gather data
- long[] inputBytes = new long[tasks.length];
-
- for (int i = 0; i < tasks.length; i++) {
- inputBytes[i] = tasks[i].getCounters().get(counterName);
- }
-
- //Analyze data
- long[][] groups = Statistics.findTwoGroups(inputBytes);
-
- long avg1 = Statistics.average(groups[0]);
- long avg2 = Statistics.average(groups[1]);
-
- long min = Math.min(avg1, avg2);
- long diff = Math.abs(avg2 - avg1);
-
- Severity severity = getDeviationSeverity(min, diff);
-
- //This reduces severity if the largest file sizes are insignificant
- severity = Severity.min(severity, getFilesSeverity(avg2));
-
- //This reduces severity if number of tasks is insignificant
- severity = Severity.min(severity, Statistics.getNumTasksSeverity(groups[0].length));
-
- HeuristicResult result = new HeuristicResult(heuristicName, severity);
-
- result.addDetail("Number of tasks", Integer.toString(tasks.length));
- result.addDetail("Group A", groups[0].length + " tasks @ " + FileUtils.byteCountToDisplaySize(avg1) + " avg");
- result.addDetail("Group B", groups[1].length + " tasks @ " + FileUtils.byteCountToDisplaySize(avg2) + " avg");
-
- return result;
- }
-
- public static Severity getDeviationSeverity(long averageMin, long averageDiff) {
- if (averageMin <= 0) {
- averageMin = 1;
- }
- long value = averageDiff / averageMin;
- return Severity.getSeverityAscending(value,
- 2, 4, 8, 16);
- }
-
- public static Severity getFilesSeverity(long value) {
- return Severity.getSeverityAscending(value,
- Constants.HDFS_BLOCK_SIZE / 8,
- Constants.HDFS_BLOCK_SIZE / 4,
- Constants.HDFS_BLOCK_SIZE / 2,
- Constants.HDFS_BLOCK_SIZE);
- }
-}
diff --git a/app/com/linkedin/drelephant/analysis/heuristics/MapperDataSkewHeuristic.java b/app/com/linkedin/drelephant/analysis/heuristics/MapperDataSkewHeuristic.java
deleted file mode 100644
index 43d806f5d..000000000
--- a/app/com/linkedin/drelephant/analysis/heuristics/MapperDataSkewHeuristic.java
+++ /dev/null
@@ -1,18 +0,0 @@
-package com.linkedin.drelephant.analysis.heuristics;
-
-import com.linkedin.drelephant.hadoop.HadoopCounterHolder;
-import com.linkedin.drelephant.hadoop.HadoopJobData;
-import com.linkedin.drelephant.hadoop.HadoopTaskData;
-
-public class MapperDataSkewHeuristic extends GenericDataSkewHeuristic {
- public static final String heuristicName = "Mapper Data Skew";
-
- public MapperDataSkewHeuristic() {
- super(HadoopCounterHolder.CounterName.HDFS_BYTES_READ, heuristicName);
- }
-
- @Override
- protected HadoopTaskData[] getTasks(HadoopJobData data) {
- return data.getMapperData();
- }
-}
\ No newline at end of file
diff --git a/app/com/linkedin/drelephant/analysis/heuristics/MapperInputSizeHeuristic.java b/app/com/linkedin/drelephant/analysis/heuristics/MapperInputSizeHeuristic.java
deleted file mode 100644
index 3bc906747..000000000
--- a/app/com/linkedin/drelephant/analysis/heuristics/MapperInputSizeHeuristic.java
+++ /dev/null
@@ -1,84 +0,0 @@
-package com.linkedin.drelephant.analysis.heuristics;
-
-import com.linkedin.drelephant.analysis.Constants;
-import com.linkedin.drelephant.analysis.Heuristic;
-import com.linkedin.drelephant.analysis.HeuristicResult;
-import com.linkedin.drelephant.analysis.Severity;
-import com.linkedin.drelephant.hadoop.HadoopCounterHolder;
-import com.linkedin.drelephant.hadoop.HadoopJobData;
-import com.linkedin.drelephant.hadoop.HadoopTaskData;
-import com.linkedin.drelephant.math.Statistics;
-import org.apache.commons.io.FileUtils;
-
-public class MapperInputSizeHeuristic implements Heuristic {
- public static final String heuristicName = "Mapper Input Size";
-
- @Override
- public String getHeuristicName() {
- return heuristicName;
- }
-
- @Override
- public HeuristicResult apply(HadoopJobData data) {
- HadoopTaskData[] tasks = data.getMapperData();
-
- //Gather data
- long[] inputBytes = new long[tasks.length];
-
- for (int i = 0; i < tasks.length; i++) {
- inputBytes[i] = tasks[i].getCounters().get(HadoopCounterHolder.CounterName.HDFS_BYTES_READ);
- }
-
- //Analyze data
- long average = Statistics.average(inputBytes);
-
- Severity smallFilesSeverity = smallFilesSeverity(average, tasks.length);
- Severity largeFilesSeverity = largeFilesSeverity(average, tasks.length);
- Severity severity = Severity.max(smallFilesSeverity, largeFilesSeverity);
-
- HeuristicResult result = new HeuristicResult(heuristicName, severity);
-
- result.addDetail("Number of tasks", Integer.toString(tasks.length));
- result.addDetail("Average task input", FileUtils.byteCountToDisplaySize(average));
-
- return result;
- }
-
- private Severity smallFilesSeverity(long value, long numTasks) {
- Severity severity = getSmallFilesSeverity(value);
- Severity taskSeverity = getNumTasksSeverity(numTasks);
- return Severity.min(severity, taskSeverity);
- }
-
- private Severity largeFilesSeverity(long value, long numTasks) {
- Severity severity = getLargeFilesSeverity(value);
- Severity taskSeverity = getNumTasksSeverityReverse(numTasks);
- return Severity.min(severity, taskSeverity);
- }
-
- public static Severity getSmallFilesSeverity(long value) {
- return Severity.getSeverityDescending(value,
- Constants.HDFS_BLOCK_SIZE / 2,
- Constants.HDFS_BLOCK_SIZE / 4,
- Constants.HDFS_BLOCK_SIZE / 8,
- Constants.HDFS_BLOCK_SIZE / 32);
- }
-
- public static Severity getLargeFilesSeverity(long value) {
- return Severity.getSeverityAscending(value,
- Constants.HDFS_BLOCK_SIZE * 2,
- Constants.HDFS_BLOCK_SIZE * 3,
- Constants.HDFS_BLOCK_SIZE * 4,
- Constants.HDFS_BLOCK_SIZE * 5);
- }
-
- public static Severity getNumTasksSeverity(long numTasks) {
- return Severity.getSeverityAscending(numTasks,
- 10, 50, 200, 500);
- }
-
- public static Severity getNumTasksSeverityReverse(long numTasks) {
- return Severity.getSeverityDescending(numTasks,
- 1000, 500, 200, 100);
- }
-}
diff --git a/app/com/linkedin/drelephant/analysis/heuristics/MapperSpeedHeuristic.java b/app/com/linkedin/drelephant/analysis/heuristics/MapperSpeedHeuristic.java
deleted file mode 100644
index 8046ef5ce..000000000
--- a/app/com/linkedin/drelephant/analysis/heuristics/MapperSpeedHeuristic.java
+++ /dev/null
@@ -1,85 +0,0 @@
-package com.linkedin.drelephant.analysis.heuristics;
-
-import java.util.ArrayList;
-import java.util.List;
-
-import com.linkedin.drelephant.analysis.Constants;
-import com.linkedin.drelephant.analysis.Heuristic;
-import com.linkedin.drelephant.analysis.HeuristicResult;
-import com.linkedin.drelephant.analysis.Severity;
-import com.linkedin.drelephant.hadoop.HadoopCounterHolder;
-import com.linkedin.drelephant.hadoop.HadoopJobData;
-import com.linkedin.drelephant.hadoop.HadoopTaskData;
-import com.linkedin.drelephant.math.Statistics;
-
-import org.apache.commons.io.FileUtils;
-
-public class MapperSpeedHeuristic implements Heuristic {
- public static final String heuristicName = "Mapper Speed";
-
- @Override
- public String getHeuristicName() {
- return heuristicName;
- }
-
- @Override
- public HeuristicResult apply(HadoopJobData data) {
-
- HadoopTaskData[] tasks = data.getMapperData();
-
- List input_byte_sizes = new ArrayList();
- List speeds = new ArrayList();
- List runtimes = new ArrayList();
-
- for(HadoopTaskData task : tasks) {
- if(task.timed()) {
- long input_bytes = task.getCounters().get(HadoopCounterHolder.CounterName.HDFS_BYTES_READ);
- long runtime = task.getEndTime() - task.getStartTime();
- //Apply 1 minute buffer
- runtime -= 60 * 1000;
- if (runtime < 1000) {
- runtime = 1000;
- }
- input_byte_sizes.add(input_bytes);
- runtimes.add(runtime);
- //Speed is bytes per second
- speeds.add((1000 * input_bytes) / (runtime));
- }
- }
-
- //Analyze data
- long averageSpeed = Statistics.average(speeds);
- long averageSize = Statistics.average(input_byte_sizes);
- long averageRuntime = Statistics.average(runtimes);
-
- Severity severity = getDiskSpeedSeverity(averageSpeed);
-
- //This reduces severity if task runtime is insignificant
- severity = Severity.min(severity, getRuntimeSeverity(averageRuntime));
-
- HeuristicResult result = new HeuristicResult(heuristicName, severity);
-
- result.addDetail("Number of tasks", Integer.toString(tasks.length));
- result.addDetail("Average task input size", FileUtils.byteCountToDisplaySize(averageSize));
- result.addDetail("Average task speed", FileUtils.byteCountToDisplaySize(averageSpeed) + "/s");
- result.addDetail("Average task runtime", Statistics.readableTimespan(averageRuntime));
-
- return result;
- }
-
- public static Severity getDiskSpeedSeverity(long speed) {
- return Severity.getSeverityDescending(speed,
- Constants.DISK_READ_SPEED / 2,
- Constants.DISK_READ_SPEED / 4,
- Constants.DISK_READ_SPEED / 8,
- Constants.DISK_READ_SPEED / 32);
- }
-
- public static Severity getRuntimeSeverity(long runtime) {
- return Severity.getSeverityAscending(runtime,
- 5 * Statistics.MINUTE,
- 20 * Statistics.MINUTE,
- 40 * Statistics.MINUTE,
- 1 * Statistics.HOUR);
- }
-}
diff --git a/app/com/linkedin/drelephant/analysis/heuristics/ReducerDataSkewHeuristic.java b/app/com/linkedin/drelephant/analysis/heuristics/ReducerDataSkewHeuristic.java
deleted file mode 100644
index 72c6ec499..000000000
--- a/app/com/linkedin/drelephant/analysis/heuristics/ReducerDataSkewHeuristic.java
+++ /dev/null
@@ -1,18 +0,0 @@
-package com.linkedin.drelephant.analysis.heuristics;
-
-import com.linkedin.drelephant.hadoop.HadoopCounterHolder;
-import com.linkedin.drelephant.hadoop.HadoopJobData;
-import com.linkedin.drelephant.hadoop.HadoopTaskData;
-
-public class ReducerDataSkewHeuristic extends GenericDataSkewHeuristic {
- public static final String heuristicName = "Reducer Data Skew";
-
- public ReducerDataSkewHeuristic() {
- super(HadoopCounterHolder.CounterName.REDUCE_SHUFFLE_BYTES, heuristicName);
- }
-
- @Override
- protected HadoopTaskData[] getTasks(HadoopJobData data) {
- return data.getReducerData();
- }
-}
diff --git a/app/com/linkedin/drelephant/analysis/heuristics/ReducerTimeHeuristic.java b/app/com/linkedin/drelephant/analysis/heuristics/ReducerTimeHeuristic.java
deleted file mode 100644
index 4982000f9..000000000
--- a/app/com/linkedin/drelephant/analysis/heuristics/ReducerTimeHeuristic.java
+++ /dev/null
@@ -1,85 +0,0 @@
-package com.linkedin.drelephant.analysis.heuristics;
-
-import java.util.ArrayList;
-import java.util.List;
-
-import com.linkedin.drelephant.analysis.Heuristic;
-import com.linkedin.drelephant.analysis.HeuristicResult;
-import com.linkedin.drelephant.analysis.Severity;
-import com.linkedin.drelephant.hadoop.HadoopJobData;
-import com.linkedin.drelephant.hadoop.HadoopTaskData;
-import com.linkedin.drelephant.math.Statistics;
-
-public class ReducerTimeHeuristic implements Heuristic {
- public static final String heuristicName = "Reducer Time";
-
- @Override
- public String getHeuristicName() {
- return heuristicName;
- }
-
- @Override
- public HeuristicResult apply(HadoopJobData data) {
- HadoopTaskData[] tasks = data.getReducerData();
-
- List runTimes = new ArrayList();
-
- for(HadoopTaskData task : tasks) {
- if(task.timed()) {
- runTimes.add(task.getRunTime());
- }
- }
-
- //Analyze data
- long averageRuntime = Statistics.average(runTimes);
-
- Severity shortTimeSeverity = shortTimeSeverity(averageRuntime, tasks.length);
- Severity longTimeSeverity = longTimeSeverity(averageRuntime, tasks.length);
- Severity severity = Severity.max(shortTimeSeverity, longTimeSeverity);
-
- HeuristicResult result = new HeuristicResult(heuristicName, severity);
-
- result.addDetail("Number of tasks", Integer.toString(tasks.length));
- result.addDetail("Average task time", Statistics.readableTimespan(averageRuntime));
-
- return result;
- }
-
- private Severity shortTimeSeverity(long runtime, long numTasks) {
- Severity timeSeverity = getShortRuntimeSeverity(runtime);
- Severity taskSeverity = getNumTasksSeverity(numTasks);
- return Severity.min(timeSeverity, taskSeverity);
- }
-
- private Severity longTimeSeverity(long runtime, long numTasks) {
- Severity timeSeverity = getLongRuntimeSeverity(runtime);
- Severity taskSeverity = getNumTasksSeverityReverse(numTasks);
- return Severity.min(timeSeverity, taskSeverity);
- }
-
- public static Severity getShortRuntimeSeverity(long runtime) {
- return Severity.getSeverityDescending(runtime,
- 10 * Statistics.MINUTE,
- 5 * Statistics.MINUTE,
- 2 * Statistics.MINUTE,
- 1 * Statistics.MINUTE);
- }
-
- public static Severity getLongRuntimeSeverity(long runtime) {
- return Severity.getSeverityAscending(runtime,
- 15 * Statistics.MINUTE,
- 30 * Statistics.MINUTE,
- 1 * Statistics.HOUR,
- 2 * Statistics.HOUR);
- }
-
- public static Severity getNumTasksSeverity(long numTasks) {
- return Severity.getSeverityAscending(numTasks,
- 10, 50, 200, 500);
- }
-
- public static Severity getNumTasksSeverityReverse(long numTasks) {
- return Severity.getSeverityDescending(numTasks,
- 100, 50, 20, 10);
- }
-}
diff --git a/app/com/linkedin/drelephant/analysis/heuristics/ShuffleSortHeuristic.java b/app/com/linkedin/drelephant/analysis/heuristics/ShuffleSortHeuristic.java
deleted file mode 100644
index e00964ccd..000000000
--- a/app/com/linkedin/drelephant/analysis/heuristics/ShuffleSortHeuristic.java
+++ /dev/null
@@ -1,77 +0,0 @@
-package com.linkedin.drelephant.analysis.heuristics;
-
-import java.util.ArrayList;
-import java.util.List;
-
-import com.linkedin.drelephant.analysis.Constants;
-import com.linkedin.drelephant.analysis.Heuristic;
-import com.linkedin.drelephant.analysis.HeuristicResult;
-import com.linkedin.drelephant.analysis.Severity;
-import com.linkedin.drelephant.hadoop.HadoopJobData;
-import com.linkedin.drelephant.hadoop.HadoopTaskData;
-import com.linkedin.drelephant.math.Statistics;
-
-public class ShuffleSortHeuristic implements Heuristic {
- public static final String heuristicName = "Shuffle & Sort";
-
- @Override
- public String getHeuristicName() {
- return heuristicName;
- }
-
- @Override
- public HeuristicResult apply(HadoopJobData data) {
-
- HadoopTaskData[] tasks = data.getReducerData();
-
- List execTime = new ArrayList();
- List shuffleTime = new ArrayList();
- List sortTime = new ArrayList();
-
- for(HadoopTaskData task : tasks) {
- if(task.timed()) {
- execTime.add(task.getExecutionTime());
- shuffleTime.add(task.getShuffleTime());
- sortTime.add(task.getSortTime());
- }
- }
-
- //Analyze data
- long avgExecTime = Statistics.average(execTime);
- long avgShuffleTime = Statistics.average(shuffleTime);
- long avgSortTime = Statistics.average(sortTime);
-
-
- Severity shuffleSeverity = getShuffleSortSeverity(avgShuffleTime, avgExecTime);
- Severity sortSeverity = getShuffleSortSeverity(avgSortTime, avgExecTime);
- Severity severity = Severity.max(shuffleSeverity, sortSeverity);
-
- HeuristicResult result = new HeuristicResult(heuristicName, severity);
-
- result.addDetail("Number of tasks", Integer.toString(data.getReducerData().length));
- result.addDetail("Average code runtime", Statistics.readableTimespan(avgExecTime));
- String shuffleFactor = Statistics.describeFactor(avgShuffleTime, avgExecTime, "x");
- result.addDetail("Average shuffle time", Statistics.readableTimespan(avgShuffleTime) + " " + shuffleFactor);
- String sortFactor = Statistics.describeFactor(avgSortTime, avgExecTime, "x");
- result.addDetail("Average sort time", Statistics.readableTimespan(avgSortTime) + " " + sortFactor);
-
- return result;
- }
-
- public static Severity getShuffleSortSeverity(long runtime, long codetime) {
- Severity runtimeSeverity = Severity.getSeverityAscending(runtime,
- 1 * Statistics.MINUTE,
- 5 * Statistics.MINUTE,
- 10 * Statistics.MINUTE,
- 30 * Statistics.MINUTE);
-
- if (codetime <= 0) {
- return runtimeSeverity;
- }
- long value = runtime * 2 / codetime;
- Severity runtimeRatioSeverity = Severity.getSeverityAscending(value,
- 1, 2, 4, 8);
-
- return Severity.min(runtimeSeverity, runtimeRatioSeverity);
- }
-}
\ No newline at end of file
diff --git a/app/com/linkedin/drelephant/configurations/aggregator/AggregatorConfiguration.java b/app/com/linkedin/drelephant/configurations/aggregator/AggregatorConfiguration.java
new file mode 100644
index 000000000..62120462a
--- /dev/null
+++ b/app/com/linkedin/drelephant/configurations/aggregator/AggregatorConfiguration.java
@@ -0,0 +1,102 @@
+/*
+ * Copyright 2016 LinkedIn Corp.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License"); you may not
+ * use this file except in compliance with the License. You may obtain a copy of
+ * the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package com.linkedin.drelephant.configurations.aggregator;
+
+import com.linkedin.drelephant.configurations.heuristic.HeuristicConfigurationData;
+import com.linkedin.drelephant.util.Utils;
+import java.util.List;
+import org.apache.log4j.Logger;
+
+import com.linkedin.drelephant.analysis.ApplicationType;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+
+import java.util.Map;
+import org.apache.log4j.Logger;
+import org.w3c.dom.Element;
+import org.w3c.dom.Node;
+import org.w3c.dom.NodeList;
+
+public class AggregatorConfiguration {
+
+ private static final Logger logger = Logger.getLogger(AggregatorConfiguration.class);
+ private List _aggregatorsConfDataList;
+
+ public AggregatorConfiguration(Element configuration) {
+ parseAggregatorConfiguration(configuration);
+ }
+
+ /**
+ * Returns the list of Aggregators along with their Configuration Information
+ *
+ * @return A list of Configuration Data for the aggregators
+ */
+ public List getAggregatorsConfigurationData() {
+ return _aggregatorsConfDataList;
+ }
+
+ /**
+ * Parses the Aggregator configuration file and loads the Aggregator Information to a list of AggregatorConfigurationData
+ *
+ * @param configuration The dom Element to be parsed
+ */
+ private void parseAggregatorConfiguration(Element configuration) {
+ _aggregatorsConfDataList = new ArrayList();
+
+ NodeList nodes = configuration.getChildNodes();
+ int n = 0;
+ for (int i = 0; i < nodes.getLength(); i++) {
+ Node node = nodes.item(i);
+ if (node.getNodeType() == Node.ELEMENT_NODE) {
+ n++;
+ Element aggregatorNode = (Element) node;
+
+ String className;
+ Node classNameNode = aggregatorNode.getElementsByTagName("classname").item(0);
+ if (classNameNode == null) {
+ throw new RuntimeException("No tag 'classname' in aggregator " + n);
+ }
+ className = classNameNode.getTextContent();
+ if (className.equals("")) {
+ throw new RuntimeException("Empty tag 'classname' in aggregator " + n);
+ }
+
+ Node appTypeNode = aggregatorNode.getElementsByTagName("applicationtype").item(0);
+ if (appTypeNode == null) {
+ throw new RuntimeException(
+ "No tag or invalid tag 'applicationtype' in aggregator " + n + " classname " + className);
+ }
+ String appTypeStr = appTypeNode.getTextContent();
+ if (appTypeStr == null) {
+ logger.error("Application type is not specified in aggregator " + n + " classname " + className
+ + ". Skipping this configuration.");
+ continue;
+ }
+ ApplicationType appType = new ApplicationType(appTypeStr);
+ // Check if parameters are defined for the heuristic
+ Map paramsMap = Utils.getConfigurationParameters(aggregatorNode);
+
+ AggregatorConfigurationData aggregatorData = new AggregatorConfigurationData(className, appType, paramsMap);
+
+ _aggregatorsConfDataList.add(aggregatorData);
+
+ }
+ }
+ }
+
+}
diff --git a/app/com/linkedin/drelephant/configurations/aggregator/AggregatorConfigurationData.java b/app/com/linkedin/drelephant/configurations/aggregator/AggregatorConfigurationData.java
new file mode 100644
index 000000000..f0b54b454
--- /dev/null
+++ b/app/com/linkedin/drelephant/configurations/aggregator/AggregatorConfigurationData.java
@@ -0,0 +1,47 @@
+/*
+ * Copyright 2016 LinkedIn Corp.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License"); you may not
+ * use this file except in compliance with the License. You may obtain a copy of
+ * the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package com.linkedin.drelephant.configurations.aggregator;
+
+import com.linkedin.drelephant.analysis.ApplicationType;
+import java.util.HashMap;
+import java.util.Map;
+
+
+public class AggregatorConfigurationData {
+
+ private final String _className;
+ private final ApplicationType _appType;
+ private final Map _paramMap;
+
+ public AggregatorConfigurationData(String className, ApplicationType appType, Map paramMap) {
+ _className = className;
+ _appType = appType;
+ _paramMap = paramMap == null ? new HashMap() : paramMap;
+ }
+
+ public String getClassName() {
+ return _className;
+ }
+
+ public ApplicationType getAppType() {
+ return _appType;
+ }
+
+ public Map getParamMap() {
+ return _paramMap;
+ }
+}
diff --git a/app/com/linkedin/drelephant/configurations/fetcher/FetcherConfiguration.java b/app/com/linkedin/drelephant/configurations/fetcher/FetcherConfiguration.java
new file mode 100644
index 000000000..bf31d0a49
--- /dev/null
+++ b/app/com/linkedin/drelephant/configurations/fetcher/FetcherConfiguration.java
@@ -0,0 +1,103 @@
+/*
+ * Copyright 2016 LinkedIn Corp.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License"); you may not
+ * use this file except in compliance with the License. You may obtain a copy of
+ * the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package com.linkedin.drelephant.configurations.fetcher;
+
+import com.linkedin.drelephant.analysis.ApplicationType;
+import com.linkedin.drelephant.util.Utils;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+
+import java.util.Map;
+import org.apache.log4j.Logger;
+import org.w3c.dom.Element;
+import org.w3c.dom.Node;
+import org.w3c.dom.NodeList;
+
+
+/**
+ * This class manages the Fetcher Configurations
+ */
+public class FetcherConfiguration {
+ private static final Logger logger = Logger.getLogger(FetcherConfiguration.class);
+ private List _fetchersConfDataList;
+
+ public FetcherConfiguration(Element configuration) {
+ parseFetcherConfiguration(configuration);
+ }
+
+ /**
+ * Returns the list of Fetchers along with their Configuration Information
+ *
+ * @return A list of Configuration Data for the fetchers
+ */
+ public List getFetchersConfigurationData() {
+ return _fetchersConfDataList;
+ }
+
+ /**
+ * Parses the Fetcher configuration file and loads the Fetcher Information to a list of FetcherConfigurationData
+ *
+ * @param configuration The dom Element to be parsed
+ */
+ private void parseFetcherConfiguration(Element configuration) {
+ _fetchersConfDataList = new ArrayList();
+
+ NodeList nodes = configuration.getChildNodes();
+ int n = 0;
+ for (int i = 0; i < nodes.getLength(); i++) {
+ // Each heuristic node
+ Node node = nodes.item(i);
+ if (node.getNodeType() == Node.ELEMENT_NODE) {
+ n++;
+ Element fetcherElem = (Element) node;
+
+ String className;
+ Node classNameNode = fetcherElem.getElementsByTagName("classname").item(0);
+ if (classNameNode == null) {
+ throw new RuntimeException("No tag 'classname' in fetcher " + n);
+ }
+ className = classNameNode.getTextContent();
+ if (className.equals("")) {
+ throw new RuntimeException("Empty tag 'classname' in fetcher " + n);
+ }
+
+ Node appTypeNode = fetcherElem.getElementsByTagName("applicationtype").item(0);
+ if (appTypeNode == null) {
+ throw new RuntimeException(
+ "No tag or invalid tag 'applicationtype' in fetcher " + n + " classname " + className);
+ }
+ String appTypeStr = appTypeNode.getTextContent();
+ if (appTypeStr == null) {
+ logger.error("Application type is not specified in fetcher " + n + " classname " + className
+ + ". Skipping this configuration.");
+ continue;
+ }
+ ApplicationType appType = new ApplicationType(appTypeStr);
+
+ // Check if parameters are defined for the heuristic
+ Map paramsMap = Utils.getConfigurationParameters(fetcherElem);
+
+ FetcherConfigurationData fetcherData = new FetcherConfigurationData(className, appType, paramsMap);
+ _fetchersConfDataList.add(fetcherData);
+
+ }
+ }
+ }
+
+}
diff --git a/app/com/linkedin/drelephant/configurations/fetcher/FetcherConfigurationData.java b/app/com/linkedin/drelephant/configurations/fetcher/FetcherConfigurationData.java
new file mode 100644
index 000000000..6088d7ed2
--- /dev/null
+++ b/app/com/linkedin/drelephant/configurations/fetcher/FetcherConfigurationData.java
@@ -0,0 +1,48 @@
+/*
+ * Copyright 2016 LinkedIn Corp.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License"); you may not
+ * use this file except in compliance with the License. You may obtain a copy of
+ * the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package com.linkedin.drelephant.configurations.fetcher;
+
+import com.linkedin.drelephant.analysis.ApplicationType;
+import java.util.Map;
+
+
+/**
+ * The Fetcher Configuration Holder
+ */
+public class FetcherConfigurationData {
+ private final String _className;
+ private final ApplicationType _appType;
+ private final Map _paramMap;
+
+ public FetcherConfigurationData(String className, ApplicationType appType, Map paramMap) {
+ _className = className;
+ _appType = appType;
+ _paramMap = paramMap;
+ }
+
+ public String getClassName() {
+ return _className;
+ }
+
+ public ApplicationType getAppType() {
+ return _appType;
+ }
+
+ public Map getParamMap() {
+ return _paramMap;
+ }
+}
diff --git a/app/com/linkedin/drelephant/configurations/heuristic/HeuristicConfiguration.java b/app/com/linkedin/drelephant/configurations/heuristic/HeuristicConfiguration.java
new file mode 100644
index 000000000..8528ece5c
--- /dev/null
+++ b/app/com/linkedin/drelephant/configurations/heuristic/HeuristicConfiguration.java
@@ -0,0 +1,114 @@
+/*
+ * Copyright 2016 LinkedIn Corp.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License"); you may not
+ * use this file except in compliance with the License. You may obtain a copy of
+ * the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package com.linkedin.drelephant.configurations.heuristic;
+
+import com.linkedin.drelephant.analysis.ApplicationType;
+import com.linkedin.drelephant.util.Utils;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+
+import java.util.Map;
+import org.apache.log4j.Logger;
+import org.w3c.dom.Element;
+import org.w3c.dom.Node;
+import org.w3c.dom.NodeList;
+
+
+/**
+ * This class manages the Heuristic Configurations
+ */
+public class HeuristicConfiguration {
+ private static final Logger logger = Logger.getLogger(HeuristicConfiguration.class);
+ private List _heuristicsConfDataList;
+
+ public HeuristicConfiguration(Element configuration) {
+ parseHeuristicConfiguration(configuration);
+ }
+
+ public List getHeuristicsConfigurationData() {
+ return _heuristicsConfDataList;
+ }
+
+ private void parseHeuristicConfiguration(Element configuration) {
+ _heuristicsConfDataList = new ArrayList();
+
+ NodeList nodes = configuration.getChildNodes();
+ int n = 0;
+ for (int i = 0; i < nodes.getLength(); i++) {
+ // Each heuristic node
+ Node node = nodes.item(i);
+ if (node.getNodeType() == Node.ELEMENT_NODE) {
+ n++;
+ Element heuristicElem = (Element) node;
+
+ String className;
+ Node classNameNode = heuristicElem.getElementsByTagName("classname").item(0);
+ if (classNameNode == null) {
+ throw new RuntimeException("No tag 'classname' in heuristic " + n);
+ }
+ className = classNameNode.getTextContent();
+ if (className.equals("")) {
+ throw new RuntimeException("Empty tag 'classname' in heuristic " + n);
+ }
+
+ String heuristicName;
+ Node heuristicNameNode = heuristicElem.getElementsByTagName("heuristicname").item(0);
+ if (heuristicNameNode == null) {
+ throw new RuntimeException("No tag 'heuristicname' in heuristic " + n + " classname " + className);
+ }
+ heuristicName = heuristicNameNode.getTextContent();
+ if (heuristicName.equals("")) {
+ throw new RuntimeException("Empty tag 'heuristicname' in heuristic " + n + " classname " + className);
+ }
+
+ String viewName;
+ Node viewNameNode = heuristicElem.getElementsByTagName("viewname").item(0);
+ if (viewNameNode == null) {
+ throw new RuntimeException("No tag 'viewname' in heuristic " + n + " classname " + className);
+ }
+ viewName = viewNameNode.getTextContent();
+ if (viewName.equals("")) {
+ throw new RuntimeException("Empty tag 'viewname' in heuristic " + n + " classname " + className);
+ }
+
+ Node appTypeNode = heuristicElem.getElementsByTagName("applicationtype").item(0);
+ if (appTypeNode == null) {
+ throw new RuntimeException(
+ "No tag or invalid tag 'applicationtype' in heuristic " + n + " classname " + className);
+ }
+ String appTypeStr = appTypeNode.getTextContent();
+ if (appTypeStr == null) {
+ logger.error("Application type is not specified in heuristic " + n + " classname " + className
+ + ". Skipping this configuration.");
+ continue;
+ }
+ ApplicationType appType = new ApplicationType(appTypeStr);
+
+ // Check if parameters are defined for the heuristic
+ Map paramsMap = Utils.getConfigurationParameters(heuristicElem);
+
+ HeuristicConfigurationData heuristicData = new HeuristicConfigurationData(heuristicName, className, viewName,
+ appType, paramsMap);
+ _heuristicsConfDataList.add(heuristicData);
+
+ }
+ }
+ }
+
+}
diff --git a/app/com/linkedin/drelephant/configurations/heuristic/HeuristicConfigurationData.java b/app/com/linkedin/drelephant/configurations/heuristic/HeuristicConfigurationData.java
new file mode 100644
index 000000000..6bcc47012
--- /dev/null
+++ b/app/com/linkedin/drelephant/configurations/heuristic/HeuristicConfigurationData.java
@@ -0,0 +1,61 @@
+/*
+ * Copyright 2016 LinkedIn Corp.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License"); you may not
+ * use this file except in compliance with the License. You may obtain a copy of
+ * the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package com.linkedin.drelephant.configurations.heuristic;
+
+import com.linkedin.drelephant.analysis.ApplicationType;
+import java.util.Map;
+
+
+/**
+ * The Heuristic Configuration Holder
+ */
+public class HeuristicConfigurationData {
+ private final String _heuristicName;
+ private final String _className;
+ private final String _viewName;
+ private final ApplicationType _appType;
+ private final Map _paramMap;
+
+ public HeuristicConfigurationData(String heuristicName, String className, String viewName, ApplicationType appType,
+ Map paramMap) {
+ _heuristicName = heuristicName;
+ _className = className;
+ _viewName = viewName;
+ _appType = appType;
+ _paramMap = paramMap;
+ }
+
+ public String getHeuristicName() {
+ return _heuristicName;
+ }
+
+ public String getClassName() {
+ return _className;
+ }
+
+ public String getViewName() {
+ return _viewName;
+ }
+
+ public ApplicationType getAppType() {
+ return _appType;
+ }
+
+ public Map getParamMap() {
+ return _paramMap;
+ }
+}
diff --git a/app/com/linkedin/drelephant/configurations/jobtype/JobTypeConfiguration.java b/app/com/linkedin/drelephant/configurations/jobtype/JobTypeConfiguration.java
new file mode 100644
index 000000000..b24a5a7d7
--- /dev/null
+++ b/app/com/linkedin/drelephant/configurations/jobtype/JobTypeConfiguration.java
@@ -0,0 +1,171 @@
+/*
+ * Copyright 2016 LinkedIn Corp.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License"); you may not
+ * use this file except in compliance with the License. You may obtain a copy of
+ * the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package com.linkedin.drelephant.configurations.jobtype;
+
+import com.linkedin.drelephant.analysis.ApplicationType;
+import com.linkedin.drelephant.util.Utils;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.regex.PatternSyntaxException;
+
+import org.apache.log4j.Logger;
+import org.w3c.dom.Document;
+import org.w3c.dom.Element;
+import org.w3c.dom.Node;
+import org.w3c.dom.NodeList;
+
+import com.linkedin.drelephant.analysis.JobType;
+
+
+/**
+ * This class manages the job type configurations
+ */
+public class JobTypeConfiguration {
+ private static final Logger logger = Logger.getLogger(JobTypeConfiguration.class);
+ private static final int TYPE_LEN_LIMIT = 20;
+
+ private Map> _appTypeToJobTypeList = new HashMap>();
+
+ public JobTypeConfiguration(Element configuration) {
+ parseJobTypeConfiguration(configuration);
+ }
+
+ public Map> getAppTypeToJobTypeList() {
+ return _appTypeToJobTypeList;
+ }
+
+ private void parseJobTypeConfiguration(Element configuration) {
+
+ Map defaultMap = new HashMap();
+
+ NodeList nodes = configuration.getChildNodes();
+ int n = 0;
+ for (int i = 0; i < nodes.getLength(); i++) {
+ Node node = nodes.item(i);
+ if (node.getNodeType() == Node.ELEMENT_NODE) {
+ n++;
+ Element jobTypeNode = (Element) node;
+
+ String jobTypeName;
+ Node jobTypeNameNode = jobTypeNode.getElementsByTagName("name").item(0);
+ if (jobTypeNameNode == null) {
+ throw new RuntimeException("No tag 'jobtype' in jobtype " + n);
+ }
+ jobTypeName = jobTypeNameNode.getTextContent();
+ if (jobTypeName.equals("")) {
+ throw new RuntimeException("Empty tag 'jobtype' in jobtype " + n);
+ }
+ // Truncate jobtype length for db constraint
+ if (jobTypeName.length() > TYPE_LEN_LIMIT) {
+ logger.info("Truncate type " + jobTypeName.length());
+ jobTypeName = jobTypeName.substring(0, TYPE_LEN_LIMIT);
+ }
+
+ String jobConfName;
+ Node jobConfNameNode = jobTypeNode.getElementsByTagName("conf").item(0);
+ if (jobConfNameNode == null) {
+ throw new RuntimeException("No tag 'conf' in jobtype " + jobTypeName);
+ }
+ jobConfName = jobConfNameNode.getTextContent();
+ if (jobConfName.equals("")) {
+ throw new RuntimeException("Empty tag 'conf' in jobtype " + jobTypeName);
+ }
+
+ String jobConfValue;
+ Node jobConfValueNode = jobTypeNode.getElementsByTagName("value").item(0);
+ if (jobConfValueNode == null) {
+ // Default regex. match any char one or more times
+ jobConfValue = ".*";
+ } else {
+ jobConfValue = jobConfValueNode.getTextContent();
+ if (jobConfValue.equals("")) {
+ jobConfValue = ".*";
+ }
+ }
+
+ String appTypeName;
+ Node appTypeNameNode = jobTypeNode.getElementsByTagName("applicationtype").item(0);
+ if (appTypeNameNode == null) {
+ throw new RuntimeException("No tag 'applicationtype' in jobtype " + jobTypeName);
+ }
+ appTypeName = appTypeNameNode.getTextContent();
+ ApplicationType appType = new ApplicationType(appTypeName);
+
+ boolean isDefault = jobTypeNode.getElementsByTagName("isDefault").item(0) != null;
+
+ JobType newJobType = null;
+ try {
+ newJobType = new JobType(jobTypeName, jobConfName, jobConfValue);
+ } catch (PatternSyntaxException e) {
+ throw new RuntimeException(
+ "Error processing this pattern. Pattern:" + jobConfValue + " jobtype:" + jobTypeName);
+ }
+
+ String newJobTypeStr = String
+ .format("jobType:%s, for application type:%s, isDefault:%s, confName:%s, confValue:%s.", jobTypeName,
+ appTypeName, isDefault, jobConfName, jobConfValue);
+ logger.info("Loaded " + newJobTypeStr);
+
+ if (isDefault) {
+ if (defaultMap.containsKey(appType)) {
+ throw new RuntimeException(
+ "Each application type should have one and only one default job type. Duplicate default job type: "
+ + newJobTypeStr + " for application type: " + appType.getName());
+ } else {
+ defaultMap.put(appType, newJobType);
+ }
+ } else {
+ List jobTypes = getJobTypeList(appType);
+ jobTypes.add(newJobType);
+ }
+ }
+ }
+
+ // Append default maps to the end of each job type list
+ for (Map.Entry entry : defaultMap.entrySet()) {
+ ApplicationType appType = entry.getKey();
+ JobType jobType = entry.getValue();
+ List jobTypes = getJobTypeList(appType);
+ jobTypes.add(jobType);
+ }
+
+ // Sanity check
+ for(ApplicationType appType : _appTypeToJobTypeList.keySet()) {
+ if (!defaultMap.containsKey(appType)) {
+ throw new RuntimeException("Each application type should have one and only one default job type, there is"
+ + " none for application type: " + appType.getName() + ". Use to tag one.");
+ }
+ }
+
+ Integer jobTypesSize = 0;
+ for (List jobTypes : _appTypeToJobTypeList.values() ) {
+ jobTypesSize += jobTypes.size();
+ }
+ logger.info("Loaded total " + jobTypesSize + " job types for " + _appTypeToJobTypeList.size() + " app types");
+ }
+
+ private List getJobTypeList(ApplicationType appType) {
+ List jobTypes = _appTypeToJobTypeList.get(appType);
+ if (jobTypes == null) {
+ jobTypes = new ArrayList();
+ _appTypeToJobTypeList.put(appType, jobTypes);
+ }
+ return jobTypes;
+ }
+}
diff --git a/app/com/linkedin/drelephant/configurations/scheduler/SchedulerConfiguration.java b/app/com/linkedin/drelephant/configurations/scheduler/SchedulerConfiguration.java
new file mode 100644
index 000000000..c7571b2c3
--- /dev/null
+++ b/app/com/linkedin/drelephant/configurations/scheduler/SchedulerConfiguration.java
@@ -0,0 +1,87 @@
+/*
+ * Copyright 2016 LinkedIn Corp.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License"); you may not
+ * use this file except in compliance with the License. You may obtain a copy of
+ * the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+
+package com.linkedin.drelephant.configurations.scheduler;
+
+import com.linkedin.drelephant.util.Utils;
+
+import org.w3c.dom.Element;
+import org.w3c.dom.Node;
+import org.w3c.dom.NodeList;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * This class manages the scheduler configurations
+ */
+public class SchedulerConfiguration {
+ private List _schedulerConfDataList;
+
+ public SchedulerConfiguration(Element configuration) {
+ parseSchedulerConfiguration(configuration);
+ }
+
+ public List getSchedulerConfigurationData() {
+ return _schedulerConfDataList;
+ }
+
+ private void parseSchedulerConfiguration(Element configuration) {
+ _schedulerConfDataList = new ArrayList();
+
+ NodeList nodes = configuration.getChildNodes();
+ int n = 0;
+ for (int i = 0; i < nodes.getLength(); i++) {
+ // Each scheduler node
+ Node node = nodes.item(i);
+ if (node.getNodeType() == Node.ELEMENT_NODE) {
+ n++;
+ Element schedulerElem = (Element) node;
+
+ String className;
+ Node classNameNode = schedulerElem.getElementsByTagName("classname").item(0);
+ if (classNameNode == null) {
+ throw new RuntimeException("No tag 'classname' in scheduler " + n);
+ }
+ className = classNameNode.getTextContent();
+ if (className.equals("")) {
+ throw new RuntimeException("Empty tag 'classname' in scheduler " + n);
+ }
+
+ String schedulerName;
+ Node schedulerNameNode = schedulerElem.getElementsByTagName("name").item(0);
+ if (schedulerNameNode == null) {
+ throw new RuntimeException("No tag 'name' in scheduler " + n + " classname " + className);
+ }
+ schedulerName = schedulerNameNode.getTextContent();
+ if (schedulerName.equals("")) {
+ throw new RuntimeException("Empty tag 'name' in scheduler " + n + " classname " + className);
+ }
+
+ // Check if parameters are defined for the scheduler
+ Map paramsMap = Utils.getConfigurationParameters(schedulerElem);
+
+ SchedulerConfigurationData schedulerData = new SchedulerConfigurationData(schedulerName, className, paramsMap);
+ _schedulerConfDataList.add(schedulerData);
+
+ }
+ }
+ }
+
+}
diff --git a/app/com/linkedin/drelephant/configurations/scheduler/SchedulerConfigurationData.java b/app/com/linkedin/drelephant/configurations/scheduler/SchedulerConfigurationData.java
new file mode 100644
index 000000000..303c6cdea
--- /dev/null
+++ b/app/com/linkedin/drelephant/configurations/scheduler/SchedulerConfigurationData.java
@@ -0,0 +1,47 @@
+/*
+ * Copyright 2016 LinkedIn Corp.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License"); you may not
+ * use this file except in compliance with the License. You may obtain a copy of
+ * the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package com.linkedin.drelephant.configurations.scheduler;
+
+import java.util.Map;
+
+
+/**
+ * Scheduler Configuration Holder
+ */
+public class SchedulerConfigurationData {
+ private final String _schedulerName;
+ private final String _className;
+ private final Map _paramMap;
+
+ public SchedulerConfigurationData(String schedulerName, String className, Map paramMap) {
+ _schedulerName = schedulerName;
+ _className = className;
+ _paramMap = paramMap;
+ }
+
+ public String getSchedulerName() {
+ return _schedulerName;
+ }
+
+ public String getClassName() {
+ return _className;
+ }
+
+ public Map getParamMap() {
+ return _paramMap;
+ }
+}
diff --git a/app/com/linkedin/drelephant/exceptions/EventException.java b/app/com/linkedin/drelephant/exceptions/EventException.java
new file mode 100644
index 000000000..31a0129ed
--- /dev/null
+++ b/app/com/linkedin/drelephant/exceptions/EventException.java
@@ -0,0 +1,111 @@
+/*
+ * Copyright 2016 LinkedIn Corp.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License"); you may not
+ * use this file except in compliance with the License. You may obtain a copy of
+ * the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package com.linkedin.drelephant.exceptions;
+
+import org.apache.log4j.Logger;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+
+/**
+ * This class represents an exception in the exception chain(a list of exceptions)
+ */
+
+public class EventException {
+ private final Logger logger = Logger.getLogger(EventException.class);
+
+
+ Pattern stackTraceLinePattern = Pattern.compile("^[\\\\t \\t]*at (.+)\\.(.+(?=\\())\\((.*)\\)");
+ /**
+ Example string: '\tat org.testng.Assert.fail(Assert.java:89)'
+ matches: ['org.testng.Assert', 'fail', "Assert.java:89']
+ */
+
+
+ Pattern exceptionDetailsPattern = Pattern.compile("^([^() :]*): (.*)");
+ /**
+ Example string: 'java.lang.AssertionError: Failure 1 expected: but was:'
+ matches: ['java.lang.AssertionError','Failure 1 expected: but was:']
+ */
+
+
+ Pattern separateLinesPattern = Pattern.compile(".*\\n");
+ private String _type;
+ private int _index;
+ private String _message;
+ private List _stackTrace;
+
+ public EventException(int index, String rawEventException) {
+ this._index = index;
+ processRawString(rawEventException);
+ }
+
+ /**
+ * Returns the message in EventException
+ * @return message in event exception
+ */
+ public String getMessage() {
+ return _message;
+ }
+
+ /**
+ * Process a raw exception string and sets the field of EventException Object
+ * @param rawEventException exception in a string form
+ */
+ private void processRawString(String rawEventException) {
+ int frameIndex = 0;
+ List stackTrace = new ArrayList();
+ List lines = stringToListOfLines(rawEventException);
+
+ for (String line : lines) {
+ Matcher exceptionDetailsMatcher = exceptionDetailsPattern.matcher(line);
+ if (exceptionDetailsMatcher.find()) {
+ this._type = exceptionDetailsMatcher.group(1);
+ this._message = exceptionDetailsMatcher.group(2);
+ } else {
+ Matcher stackTraceLineMatcher = stackTraceLinePattern.matcher(line);
+ if (stackTraceLineMatcher.find()) {
+ String source = stackTraceLineMatcher.group(1);
+ String call = stackTraceLineMatcher.group(2);
+ String fileDetails = stackTraceLineMatcher.group(3);
+ StackTraceFrame stackTraceFrame = new StackTraceFrame(frameIndex, source, call, fileDetails);
+ stackTrace.add(stackTraceFrame);
+ frameIndex += 1;
+ }
+ }
+ }
+ this._stackTrace = stackTrace;
+ }
+
+ /**
+ * Takes a exception in string form and converts it into a list of string where each string corresponds to a line in
+ * exception
+ * @param rawEventException exception in a string form
+ * @return list of lines in the exception
+ */
+ private List stringToListOfLines(String rawEventException) {
+ Matcher separateLinesMatcher = separateLinesPattern.matcher(rawEventException);
+ List lines = new ArrayList();
+ while (separateLinesMatcher.find()) {
+ lines.add(separateLinesMatcher.group());
+ }
+ return lines;
+ }
+}
diff --git a/app/com/linkedin/drelephant/exceptions/ExceptionFinder.java b/app/com/linkedin/drelephant/exceptions/ExceptionFinder.java
new file mode 100644
index 000000000..19098eab4
--- /dev/null
+++ b/app/com/linkedin/drelephant/exceptions/ExceptionFinder.java
@@ -0,0 +1,227 @@
+/*
+ * Copyright 2016 LinkedIn Corp.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License"); you may not
+ * use this file except in compliance with the License. You may obtain a copy of
+ * the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package com.linkedin.drelephant.exceptions;
+
+import com.linkedin.drelephant.configurations.scheduler.SchedulerConfigurationData;
+import com.linkedin.drelephant.security.HadoopSecurity;
+import com.linkedin.drelephant.util.InfoExtractor;
+import java.io.File;
+import java.io.IOException;
+import java.security.PrivilegedAction;
+import javax.naming.AuthenticationException;
+import org.apache.log4j.Logger;
+
+import java.net.MalformedURLException;
+import java.net.URISyntaxException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+
+/**
+ * ExceptionFinder class finds the exception along with the level of the exception. It takes the scheduler and the url of the workflow as
+ * parameters.
+ */
+public class ExceptionFinder {
+ private final Logger logger = Logger.getLogger(ExceptionFinder.class);
+ private HadoopException _exception;
+ private WorkflowClient _workflowClient;
+ private MRClient _mrClient;
+
+ private static String USERNAME = "username";
+ private static String PRIVATE_KEY = "private_key";
+ private static String PASSWORD = "password";
+ private static int SAMPLE_SIZE = 3;
+
+ /**
+ * Constructor for ExceptionFinder class
+ * @param url The url of the workflow to analyze
+ * @param scheduler The scheduler where the workflow was run.
+ * @throws URISyntaxException
+ * @throws MalformedURLException
+ */
+ public ExceptionFinder(String url, String scheduler)
+ throws URISyntaxException, MalformedURLException, AuthenticationException, IOException {
+
+ // create a new MRClient
+ _mrClient = new MRClient();
+
+ // create a new workflow client
+ _workflowClient = InfoExtractor.getWorkflowClientInstance(scheduler, url);
+
+ // get the schedulerData
+ SchedulerConfigurationData schedulerData = InfoExtractor.getSchedulerData(scheduler);
+
+
+ if(schedulerData==null) {
+ throw new RuntimeException(String.format("Cannot find scheduler %s", scheduler));
+ }
+
+ if(!schedulerData.getParamMap().containsKey(USERNAME)) {
+ throw new RuntimeException(String.format("Cannot find username for login"));
+ }
+
+ String username = schedulerData.getParamMap().get(USERNAME);
+
+ if(schedulerData.getParamMap().containsKey(PRIVATE_KEY)) {
+ _workflowClient.login(username, new File(schedulerData.getParamMap().get(PRIVATE_KEY)));
+ } else if (schedulerData.getParamMap().containsKey(PASSWORD)) {
+ _workflowClient.login(username, schedulerData.getParamMap().get(PASSWORD));
+ } else {
+ throw new RuntimeException("Neither private key nor password was specified");
+ }
+ _exception = analyzeFlow(url);
+ }
+
+ /**
+ * Analyzes a Flow and returns a HadoopException object which captures all the exception in the flow.
+ * @param execUrl the execution URL of the flow
+ * @return HadoopException object which captures all the exceptions in the given Flow
+ */
+ private HadoopException analyzeFlow(final String execUrl) throws AuthenticationException, IOException {
+ HadoopSecurity _hadoopSecurity = HadoopSecurity.getInstance();
+
+ return _hadoopSecurity.doAs(new PrivilegedAction() {
+ @Override
+ public HadoopException run() {
+ HadoopException flowLevelException = new HadoopException();
+ List childExceptions = new ArrayList();
+ Map jobIdStatus = _workflowClient.getJobsFromFlow();
+
+ // Find exceptions in all the unsuccessful jobs of the workflow
+ for (String unsuccessfulJobId : jobIdStatus.keySet()) {
+ if (jobIdStatus.get(unsuccessfulJobId).toLowerCase().equals("failed")) {
+ HadoopException jobLevelException = analyzeJob(unsuccessfulJobId);
+ childExceptions.add(jobLevelException);
+ }
+ }
+
+ flowLevelException.setType(HadoopException.HadoopExceptionType.FLOW);
+ flowLevelException.setId(execUrl);
+ flowLevelException.setLoggingEvent(null); // No flow level exception
+ flowLevelException.setChildExceptions(childExceptions);
+ return flowLevelException;
+ }
+ });
+ }
+
+ /**
+ * Given a failed Job, this method analyzes the job and returns a HadoopException object which captures all the exception in the given job.
+ * @param jobId The job execution id/url, specific to the scheduler
+ * @return HadoopException object which captures all the exceptions in the given job
+ */
+ private HadoopException analyzeJob(String jobId) {
+ HadoopException jobLevelException = new HadoopException();
+ List childExceptions = new ArrayList();
+
+ _workflowClient.analyzeJob(jobId);
+
+ // get the set of all the yarn jobs from workflowClient
+ Set yarnJobIds = _workflowClient.getYarnApplicationsFromJob(jobId);
+
+ for (String mrJobId : yarnJobIds) {
+ //To do: Check if mr job logs are there or not in job history server
+ String rawMRJobLog = _mrClient.getMRJobLog(mrJobId);
+ if (rawMRJobLog != null && !rawMRJobLog.isEmpty()) { // null for log not found and empty for successful mr jobs
+ //To do: rawMRJob is empty for successful mr jobs but this is not a good way to figure out whether a job failed
+ // or succeeded, do this using the state field in rest api
+ HadoopException mrJobLevelException = analyzeMRJob(mrJobId, rawMRJobLog);
+ childExceptions.add(mrJobLevelException);
+ }
+ }
+
+ if (_workflowClient.getJobState(jobId) == JobState.MRFAIL) {
+ jobLevelException.setType(HadoopException.HadoopExceptionType.MR);
+ jobLevelException.setLoggingEvent(_workflowClient.getJobException(jobId));
+ //LoggingEvent is set only for the case if mr logs could not be found in job history server and childException is
+ // empty
+ jobLevelException.setChildExceptions(childExceptions);
+ } else if (_workflowClient.getJobState(jobId) == JobState.SCHEDULERFAIL) {
+ jobLevelException.setType(HadoopException.HadoopExceptionType.SCHEDULER);
+ jobLevelException.setLoggingEvent(_workflowClient.getJobException(jobId));
+ jobLevelException.setChildExceptions(null);
+ } else if (_workflowClient.getJobState(jobId) == JobState.SCRIPTFAIL) {
+ jobLevelException.setType(HadoopException.HadoopExceptionType.SCRIPT);
+ jobLevelException.setLoggingEvent(_workflowClient.getJobException(jobId));
+ jobLevelException.setChildExceptions(null);
+ } else if (_workflowClient.getJobState(jobId) == JobState.KILLED) {
+ jobLevelException.setType(HadoopException.HadoopExceptionType.KILL);
+ jobLevelException.setLoggingEvent(null);
+ jobLevelException.setChildExceptions(null);
+ }
+ jobLevelException.setId(jobId);
+ return jobLevelException;
+ }
+
+ /**
+ * Given a failed MR Job id and diagnostics of the job, this method analyzes it and returns a HadoopException object which captures all the exception in the given MR Job.
+ * @param mrJobId Mapreduce job id
+ * @param rawMRJoblog Diagnostics of the mapreduce job in a string
+ * @return HadoopException object which captures all the exceptions in the given Mapreduce job
+ */
+ private HadoopException analyzeMRJob(String mrJobId, String rawMRJoblog) {
+ // This method is called only for unsuccessful MR jobs
+ HadoopException mrJobLevelException = new HadoopException();
+ List childExceptions = new ArrayList();
+ MRJobLogAnalyzer analyzedLog = new MRJobLogAnalyzer(rawMRJoblog);
+ Set failedMRTaskIds = analyzedLog.getFailedSubEvents();
+
+ // sampling of tasks
+ int samplingSize = SAMPLE_SIZE;
+ for (String failedMRTaskId : failedMRTaskIds) {
+ if(samplingSize<=0) {
+ break;
+ }
+ String rawMRTaskLog = _mrClient.getMRTaskLog(mrJobId, failedMRTaskId);
+ HadoopException mrTaskLevelException = analyzeMRTask(failedMRTaskId, rawMRTaskLog);
+ childExceptions.add(mrTaskLevelException);
+
+ samplingSize--;
+ }
+
+ mrJobLevelException.setChildExceptions(childExceptions);
+ mrJobLevelException.setLoggingEvent(analyzedLog.getException());
+ mrJobLevelException.setType(HadoopException.HadoopExceptionType.MRJOB);
+ mrJobLevelException.setId(mrJobId);
+ return mrJobLevelException;
+ }
+
+ /**
+ * Given a failed MR Task id and diagnostics of the task, this method analyzes it and returns a HadoopException object which captures all the exception in the given MR task.
+ * @param mrTaskId The task id of the map reduce job
+ * @param rawMRTaskLog Raw map-reduce log
+ * @return HadoopException object which captures all the exceptions in the given Mapreduce task
+ */
+ private HadoopException analyzeMRTask(String mrTaskId, String rawMRTaskLog) {
+ HadoopException mrTaskLevelException = new HadoopException();
+ MRTaskLogAnalyzer analyzedLog = new MRTaskLogAnalyzer(rawMRTaskLog);
+ mrTaskLevelException.setLoggingEvent(analyzedLog.getException());
+ mrTaskLevelException.setType(HadoopException.HadoopExceptionType.MRTASK);
+ mrTaskLevelException.setId(mrTaskId);
+ mrTaskLevelException.setChildExceptions(null);
+ return mrTaskLevelException;
+ }
+
+ /**
+ * Returns the Hadoop Exception object
+ * @return Returns the Hadoop Exception object
+ */
+ public HadoopException getExceptions() {
+ return this._exception;
+ }
+}
diff --git a/app/com/linkedin/drelephant/exceptions/HadoopException.java b/app/com/linkedin/drelephant/exceptions/HadoopException.java
new file mode 100644
index 000000000..6dce11208
--- /dev/null
+++ b/app/com/linkedin/drelephant/exceptions/HadoopException.java
@@ -0,0 +1,77 @@
+/*
+ * Copyright 2016 LinkedIn Corp.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License"); you may not
+ * use this file except in compliance with the License. You may obtain a copy of
+ * the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package com.linkedin.drelephant.exceptions;
+
+import org.apache.log4j.Logger;
+
+import java.util.List;
+
+
+public class HadoopException {
+ private final Logger logger = Logger.getLogger(HadoopException.class);
+ private String _id = "UNKNOWN";
+ public enum HadoopExceptionType {FLOW, SCHEDULER, SCRIPT, MR, KILL, MRJOB, MRTASK}
+ /**
+ * FLOW: HadoopException object for Azkaban flow
+ * SCHEDULER : HadoopException object for Azkaban job with Azkaban level failure
+ * SCRIPT : HadoopException object for Azkaban job with Script level failure
+ * MR: HadoopException object for Azkaban job with MR level failure
+ * KILL: HadoopException object for killed Azkaban job
+ * MRJOB: HadoopException object for MR Job
+ * MRTASK: HadoopException object for MR Task
+ * */
+
+ private HadoopExceptionType _type;
+ private LoggingEvent _loggingEvent;
+ private List _childExceptions;
+
+ public String getId() {
+ return _id;
+ }
+
+ public void setId(String id) {
+ _id = id;
+ }
+
+ public HadoopExceptionType getType() {
+ return _type;
+ }
+
+ public void setType(HadoopExceptionType type) {
+ _type = type;
+ }
+
+ public LoggingEvent getLoggingEvent() {
+ return _loggingEvent;
+ }
+
+ public void setLoggingEvent(LoggingEvent e) {
+ _loggingEvent = e;
+ }
+
+ public List getChildExceptions() {
+ return _childExceptions;
+ }
+
+ public void setChildExceptions(List childExceptions) {
+ _childExceptions = childExceptions;
+ }
+
+
+}
+
+
diff --git a/app/com/linkedin/drelephant/exceptions/JobState.java b/app/com/linkedin/drelephant/exceptions/JobState.java
new file mode 100644
index 000000000..2a034daee
--- /dev/null
+++ b/app/com/linkedin/drelephant/exceptions/JobState.java
@@ -0,0 +1,24 @@
+/*
+ * Copyright 2016 LinkedIn Corp.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License"); you may not
+ * use this file except in compliance with the License. You may obtain a copy of
+ * the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package com.linkedin.drelephant.exceptions;
+
+/**
+ * This enum represents the state of the job
+ */
+public enum JobState {
+ SCRIPTFAIL, SCHEDULERFAIL, MRFAIL, SUCCEEDED, KILLED
+}
diff --git a/app/com/linkedin/drelephant/exceptions/LoggingEvent.java b/app/com/linkedin/drelephant/exceptions/LoggingEvent.java
new file mode 100644
index 000000000..b1bf7e634
--- /dev/null
+++ b/app/com/linkedin/drelephant/exceptions/LoggingEvent.java
@@ -0,0 +1,124 @@
+/*
+ * Copyright 2016 LinkedIn Corp.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License"); you may not
+ * use this file except in compliance with the License. You may obtain a copy of
+ * the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package com.linkedin.drelephant.exceptions;
+
+import org.apache.log4j.Logger;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+
+public class LoggingEvent {
+
+ private final Logger logger = Logger.getLogger(LoggingEvent.class);
+ private List _rawLog;
+ private String _log; // To do
+ private long _timestamp; //To do: Get time from logs and fill this field
+ private enum LoggingLevel {DEBUG, INFO, WARNING, ERROR, FATAL}
+ private LoggingLevel _level = LoggingLevel.ERROR; // For now I have this to be eeror
+ private String _message;
+ private List _exceptionChain;
+
+ public LoggingEvent(String exceptionChainString) {
+ this._rawLog = exceptionChainStringToListOfExceptions(exceptionChainString);
+ setExceptionChain();
+ setMessage();
+ }
+
+
+
+ /**
+ @return Returns the exception chain in the form of list of list of string.
+ A list of string corresponds to an exception in the exception chain
+ A string corresponds to a line in an exception
+ */
+
+ public List> getLog() {
+ List> log = new ArrayList>();
+ for (String exceptionString : _rawLog) {
+ List exception = exceptionStringToListOfLines(exceptionString);
+ log.add(exception);
+ }
+ return log;
+ }
+
+
+ private void setExceptionChain() {
+ List exceptionChain = new ArrayList();
+ int index = 0;
+
+ for (String rawEventException : _rawLog) {
+ EventException eventException = new EventException(index, rawEventException);
+ exceptionChain.add(eventException);
+ index += 1;
+ }
+ _exceptionChain = exceptionChain;
+ }
+
+ /**
+ * Converts a exception chain string to a list of string exceptions
+ * @param s Exception chain in a string
+ * @return List of exceptions in given the exception chain
+ */
+ private List exceptionChainStringToListOfExceptions(String s) {
+ List chain = new ArrayList();
+ Pattern stackTraceCausedByClause = Pattern.compile(".*^(?!Caused by).+\\n(?:.*\\tat.+\\n)+");
+ Pattern stackTraceOtherThanCausedByClause = Pattern.compile(".*Caused by.+\\n(?:.*\\n)?(?:.*\\s+at.+\\n)*");
+
+ Matcher matcher = stackTraceCausedByClause.matcher(s);
+ while (matcher.find()) {
+ chain.add(matcher.group());
+ }
+ matcher = stackTraceOtherThanCausedByClause.matcher(s);
+ while (matcher.find()) {
+ chain.add(matcher.group());
+ }
+
+ if (chain.isEmpty()) {
+ //error logs other than stack traces for ex- logs of azkaban level failure in azkaban job
+ chain.add(s);
+ }
+ return chain;
+ }
+
+ /**
+ * Converts a exception string to a list of string corresponding to lines in the exception
+ * @param s Exception in a single string
+ * @return List of individual lines in the string
+ */
+ private List exceptionStringToListOfLines(String s) {
+ List exception = new ArrayList();
+ Matcher matcher = Pattern.compile(".*\\n").matcher(s);
+ while (matcher.find()) {
+ exception.add(matcher.group());
+ }
+ return exception;
+ }
+
+ /** Sets message for the logging event
+ For now, It is set to be equal to the message field of first EventException in _exceptionChain
+ This can be changed depending on message of which EventException is most relevant for the user to see
+ */
+ private void setMessage() {
+ if (!_exceptionChain.isEmpty()) {
+ this._message = _exceptionChain.get(0).getMessage();
+ }
+ }
+
+}
diff --git a/app/com/linkedin/drelephant/exceptions/MRClient.java b/app/com/linkedin/drelephant/exceptions/MRClient.java
new file mode 100644
index 000000000..8ef75c250
--- /dev/null
+++ b/app/com/linkedin/drelephant/exceptions/MRClient.java
@@ -0,0 +1,123 @@
+/*
+ * Copyright 2016 LinkedIn Corp.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License"); you may not
+ * use this file except in compliance with the License. You may obtain a copy of
+ * the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package com.linkedin.drelephant.exceptions;
+
+import com.linkedin.drelephant.security.HadoopSecurity;
+import java.security.PrivilegedAction;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.security.authentication.client.AuthenticatedURL;
+import org.apache.hadoop.security.authentication.client.AuthenticationException;
+import org.apache.log4j.Logger;
+import org.codehaus.jackson.JsonNode;
+import org.codehaus.jackson.map.ObjectMapper;
+
+import java.io.IOException;
+import java.net.HttpURLConnection;
+import java.net.MalformedURLException;
+import java.net.URL;
+
+
+/**
+ * Client to interact with job history server and get the mapreduce logs
+ **/
+public class MRClient {
+ private static final Logger logger = Logger.getLogger(MRClient.class);
+ final String jhistoryAddr = new Configuration().get("mapreduce.jobhistory.webapp.address");
+ private AuthenticatedURL.Token _token;
+ private AuthenticatedURL _authenticatedURL;
+
+ public MRClient() {
+ _token = new AuthenticatedURL.Token();
+ _authenticatedURL = new AuthenticatedURL();
+ }
+
+ /**
+ * For a given rest url, fetchs and return the jsonnode
+ *
+ * @param url rest job history server url
+ * @return Json node to which the url points
+ */
+ private JsonNode fetchJson(final URL url)
+ throws IOException {
+ try {
+ ObjectMapper objectMapper = new ObjectMapper();
+ HttpURLConnection conn = _authenticatedURL.openConnection(url, _token);
+ return objectMapper.readTree(conn.getInputStream());
+ } catch (AuthenticationException e) {
+ logger.error(String.format("Cannot authenticate in Mr Client %s", e.getMessage()));
+ } catch (IOException e) {
+ logger.error(String.format("Error reading stream in Mr Client %s", e.getMessage()));
+ }
+ return null;
+ }
+
+ /**
+ * Returns the diagnostics for a given MR Job Id
+ * @param mrJobId MR Job Id
+ * @return Diagnostics in a string format
+ */
+
+ public String getMRJobLog(String mrJobId) {
+ String mrJobHistoryURL = "http://" + jhistoryAddr + "/ws/v1/history/mapreduce/jobs/" + mrJobId;
+ try {
+ JsonNode response = fetchJson(new URL(mrJobHistoryURL));
+ if (response.get("job").get("state").toString() != "SUCCEEDED") {
+ return response.get("job").get("diagnostics").getTextValue();
+ }
+ } catch (MalformedURLException e) {
+ logger.error(String.format("Malformed URL %s in MR Client: %s ", mrJobHistoryURL, e.getMessage()));
+ } catch (NullPointerException e) {
+ logger.error(String.format("Invalid response %s", e.getMessage()));
+ } catch (IOException e) {
+ logger.error(String.format("IOException in Mr Client: %s", e.getMessage()));
+ }
+ return null;
+ }
+
+ /**
+ * Returns the last task attempt diagnostic for a given failed taskId
+ *
+ * @param mrJobId MR Job Id
+ * @param mrTaskId MRTask Id
+ * @return Diagnostic in a string format
+ */
+ public String getMRTaskLog(String mrJobId, String mrTaskId) {
+ String mrTaskHistoryURL =
+ "http://" + jhistoryAddr + "/ws/v1/history/mapreduce/jobs/" + mrJobId + "/tasks/" + mrTaskId + "/attempts";
+ ;
+ try {
+ JsonNode response = fetchJson(new URL(mrTaskHistoryURL));
+ int attempts = response.get("taskAttempts").get("taskAttempt").size();
+ int maxattempt = 0;
+ int maxattemptid = 0;
+ for (int i = 0; i < attempts; i++) {
+ int attempt = Integer
+ .parseInt(response.get("taskAttempts").get("taskAttempt").get(i).get("id").getTextValue().split("_")[5]);
+ if (attempt > maxattempt) {
+ maxattemptid = i;
+ maxattempt = attempt;
+ }
+ }
+ return response.get("taskAttempts").get("taskAttempt").get(maxattemptid).get("diagnostics").getTextValue();
+ } catch (MalformedURLException e) {
+ logger.error(e.toString());
+ } catch (IOException e) {
+ logger.error(e.toString());
+ }
+ return null;
+ }
+}
diff --git a/app/com/linkedin/drelephant/exceptions/MRJobLogAnalyzer.java b/app/com/linkedin/drelephant/exceptions/MRJobLogAnalyzer.java
new file mode 100644
index 000000000..17ef247b3
--- /dev/null
+++ b/app/com/linkedin/drelephant/exceptions/MRJobLogAnalyzer.java
@@ -0,0 +1,87 @@
+/*
+ * Copyright 2016 LinkedIn Corp.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License"); you may not
+ * use this file except in compliance with the License. You may obtain a copy of
+ * the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package com.linkedin.drelephant.exceptions;
+
+import org.apache.log4j.Logger;
+
+import java.util.HashSet;
+import java.util.Set;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+
+/**
+* Given a MR Job log, sets the list of unsuccessful tasks and MR job level exception (if any)
+*/
+
+public class MRJobLogAnalyzer {
+ private static final Logger logger = Logger.getLogger(MRJobLogAnalyzer.class);
+
+ private Pattern _mrJobExceptionPattern =
+ Pattern.compile(".*\\n(?:.*\\tat.+\\n)+(?:.*Caused by.+\\n(?:.*\\n)?(?:.*\\s+at.+\\n)*)*");
+ private Pattern _unsuccessfulMRTaskIdPattern =
+ Pattern.compile("Task (?:failed) (task_[0-9]+_[0-9]+_[mr]_[0-9]+)");
+ private LoggingEvent _exception;
+ private Set _failedSubEvents;
+
+ public MRJobLogAnalyzer(String rawLog) {
+ setFailedSubEvents(rawLog);
+ setException(rawLog);
+ }
+
+ /**
+ * Given MR Job log, finds the list of unsuccessful tasks and sets it equal to _failedSubEvents
+ * @param rawLog MR Job log in a string
+ */
+ private void setFailedSubEvents(String rawLog) {
+ Set failedSubEvents = new HashSet();
+ Matcher unsuccessfulMRTaskIdMatcher = _unsuccessfulMRTaskIdPattern.matcher(rawLog);
+ while (unsuccessfulMRTaskIdMatcher.find()) {
+ failedSubEvents.add(unsuccessfulMRTaskIdMatcher.group(1));
+ }
+ this._failedSubEvents = failedSubEvents;
+ }
+
+ /**
+ * Given MR Job log, finds the MR Job level exception and sets it equal to _exception
+ * @param rawLog MR Job log in a string
+ */
+ private void setException(String rawLog) {
+ Matcher mrJobExceptionMatcher = _mrJobExceptionPattern.matcher(rawLog);
+ if (mrJobExceptionMatcher.find()) {
+ this._exception = new LoggingEvent(mrJobExceptionMatcher.group());
+ }
+ }
+
+ /**
+ * Returns the list of unsuccessful tasks in given MR Job log
+ * @return list of unsuccessful tasks in MR Job log
+ */
+ public Set getFailedSubEvents() {
+ return this._failedSubEvents;
+ }
+
+ /**
+ * Returns the MR Job level exception
+ * @return _exception of type LoggingEvent.
+ */
+ public LoggingEvent getException() {
+ return this._exception;
+ }
+
+
+}
\ No newline at end of file
diff --git a/app/com/linkedin/drelephant/exceptions/MRTaskLogAnalyzer.java b/app/com/linkedin/drelephant/exceptions/MRTaskLogAnalyzer.java
new file mode 100644
index 000000000..13f63ac19
--- /dev/null
+++ b/app/com/linkedin/drelephant/exceptions/MRTaskLogAnalyzer.java
@@ -0,0 +1,61 @@
+/*
+ * Copyright 2016 LinkedIn Corp.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License"); you may not
+ * use this file except in compliance with the License. You may obtain a copy of
+ * the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package com.linkedin.drelephant.exceptions;
+
+import org.apache.log4j.Logger;
+
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+
+/**
+* Given a MR Task log, sets the exception (if any) in the log
+*/
+public class MRTaskLogAnalyzer {
+ private static final Logger logger = Logger.getLogger(MRTaskLogAnalyzer.class);
+ private LoggingEvent _exception;
+ private long MAX_EXCEPTIONS = 5;
+ private Pattern mrTaskExceptionPattern =
+ Pattern.compile("Error: (.*\\n(?:.*\\tat.+\\n)+(?:.*Caused by.+\\n(?:.*\\n)?(?:.*\\s+at.+\\n)*)*)");
+
+ public MRTaskLogAnalyzer(String rawLog) {
+ setException(rawLog);
+ }
+
+ /**
+ * Gets the exception of the mr task
+ * @return The LoggingEvent corresponding to the exception
+ */
+ public LoggingEvent getException() {
+ return this._exception;
+ }
+
+ /**
+ * Sets the exception of the mr task
+ * @param rawLog Raw log of the task
+ */
+ private void setException(String rawLog) {
+ Matcher matcher = mrTaskExceptionPattern.matcher(rawLog);
+ long limitOnExceptionChains = MAX_EXCEPTIONS;
+ StringBuilder exceptionBuilder = new StringBuilder();
+ while (matcher.find() && limitOnExceptionChains>=0) {
+ exceptionBuilder.append(matcher.group());
+ limitOnExceptionChains--;
+ }
+ this._exception = new LoggingEvent(exceptionBuilder.toString());
+ }
+}
\ No newline at end of file
diff --git a/app/com/linkedin/drelephant/exceptions/StackTraceFrame.java b/app/com/linkedin/drelephant/exceptions/StackTraceFrame.java
new file mode 100644
index 000000000..00cef2f88
--- /dev/null
+++ b/app/com/linkedin/drelephant/exceptions/StackTraceFrame.java
@@ -0,0 +1,66 @@
+/*
+ * Copyright 2016 LinkedIn Corp.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License"); you may not
+ * use this file except in compliance with the License. You may obtain a copy of
+ * the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package com.linkedin.drelephant.exceptions;
+
+import org.apache.log4j.Logger;
+
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+
+public class StackTraceFrame {
+
+ private final Logger logger = Logger.getLogger(StackTraceFrame.class);
+ private String _source;
+ private String _fileName;
+ private int _lineNumber;
+ private String _call;
+ private boolean _nativeMethod;
+ private int _index;
+
+ public StackTraceFrame(int index, String source, String call, String fileDetails) {
+ this._source = source;
+ this._call = call;
+ this._index = index;
+ getFileDetails(fileDetails);
+ }
+
+ private void getFileDetails(String fileDetails) {
+ boolean nativeMethod = false;
+ String fileName = fileDetails;
+ String lineNumber = "0";
+ Pattern file = Pattern.compile("(.*):(.*)");
+
+ /**
+ Example string: 'Assert.java:89'
+ matches: ['Assert.java', '89']
+ */
+
+ if (fileDetails.equals("Native Method")) {
+ nativeMethod = true;
+ } else {
+ Matcher match = file.matcher(fileDetails);
+ if (match.find()) {
+ fileName = match.group(1);
+ lineNumber = match.group(2);
+ }
+ }
+ this._fileName = fileName;
+ this._lineNumber = Integer.parseInt(lineNumber); // To do: Can throw parseException
+ this._nativeMethod = nativeMethod;
+ }
+}
diff --git a/app/com/linkedin/drelephant/exceptions/WorkflowClient.java b/app/com/linkedin/drelephant/exceptions/WorkflowClient.java
new file mode 100644
index 000000000..ab2020fb9
--- /dev/null
+++ b/app/com/linkedin/drelephant/exceptions/WorkflowClient.java
@@ -0,0 +1,78 @@
+/*
+ * Copyright 2016 LinkedIn Corp.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License"); you may not
+ * use this file except in compliance with the License. You may obtain a copy of
+ * the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package com.linkedin.drelephant.exceptions;
+
+import java.io.File;
+import java.util.Map;
+import java.util.Set;
+
+
+/**
+ * The interface WorkflowClient should be implemented by all the workflow client. The client should not
+ * be confused with the a client of the scheduler since the context of this client is limited to a workflow
+ * and it doesn't operate at a scheduler level.
+ */
+public interface WorkflowClient {
+
+ /**
+ * Login to the scheduler using the username and the password
+ * @param username The username of the user
+ * @param password The password of the user
+ */
+ public void login(String username, String password);
+
+ /**
+ * Login to the scheduler using the username and the private key
+ * @param username The username of the user
+ * @param privateKey The private key of the user
+ */
+ public void login(String username, File privateKey);
+
+ /**
+ * Return all the jobs in the workflow. It returns a Map where the key \n
+ * is the execution id of the job and the value is the status of the job.
+ * @return Return all the jobs in the workflow
+ */
+ public Map getJobsFromFlow();
+
+ /**
+ * Given a job id, this method analyzes the job
+ * @param jobId The execution id of the job
+ */
+ public void analyzeJob(String jobId);
+
+ /**
+ * This method extracts out all the yarn applications from the job and returns the set of them.
+ * @param jobId The jobid of the job.
+ * @return The set of all the yarn applications spawned by the job
+ */
+ public Set getYarnApplicationsFromJob(String jobId);
+
+ /**
+ * Returns the job state of the job.
+ * @param jobId The id of the job
+ * @return Retruns the state of the job
+ */
+ public JobState getJobState(String jobId);
+
+ /**
+ * Get the exception, given a job id
+ * @param jobId The id of the job
+ * @return The exeception encountered
+ */
+ public LoggingEvent getJobException(String jobId);
+}
diff --git a/app/com/linkedin/drelephant/exceptions/azkaban/AzkabanJobLogAnalyzer.java b/app/com/linkedin/drelephant/exceptions/azkaban/AzkabanJobLogAnalyzer.java
new file mode 100644
index 000000000..71ab0b742
--- /dev/null
+++ b/app/com/linkedin/drelephant/exceptions/azkaban/AzkabanJobLogAnalyzer.java
@@ -0,0 +1,208 @@
+/*
+ * Copyright 2016 LinkedIn Corp.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License"); you may not
+ * use this file except in compliance with the License. You may obtain a copy of
+ * the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package com.linkedin.drelephant.exceptions.azkaban;
+
+import com.linkedin.drelephant.exceptions.JobState;
+import com.linkedin.drelephant.exceptions.LoggingEvent;
+import java.util.LinkedHashSet;
+import org.apache.log4j.Logger;
+
+import java.util.HashSet;
+import java.util.Set;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+
+/*
+* Given a Azkaban job log returns the Azkaban Job State, list of all MR job ids in the given log and exception (if any) at the Azkaban job level
+*/
+
+public class AzkabanJobLogAnalyzer {
+
+ private static final Logger logger = Logger.getLogger(AzkabanJobLogAnalyzer.class);
+ private Pattern _successfulAzkabanJobPattern =
+ Pattern.compile("Finishing job [^\\s]+ attempt: [0-9]+ at [0-9]+ with status SUCCEEDED");
+ private Pattern _failedAzkabanJobPattern =
+ Pattern.compile("Finishing job [^\\s]+ attempt: [0-9]+ at [0-9]+ with status FAILED");
+ private Pattern _killedAzkabanJobPattern =
+ Pattern.compile("Finishing job [^\\s]+ attempt: [0-9]+ at [0-9]+ with status KILLED");
+ private Pattern _scriptFailPattern = Pattern.compile("ERROR - Job run failed!");
+ // Alternate pattern: (".+\\n(?:.+\\tat.+\\n)+(?:.+Caused by.+\\n(?:.*\\n)?(?:.+\\s+at.+\\n)*)*");
+ private Pattern _scriptOrMRFailExceptionPattern = Pattern.compile("(Caused by.+\\n(?:.*\\n)?((?:.+\\s+at.+\\n)*))+");
+ private Pattern _azkabanFailExceptionPattern = Pattern.compile(
+ "\\d{2}[-/]\\d{2}[-/]\\d{4} \\d{2}:\\d{2}:\\d{2} (PST|PDT) [^\\s]+ (?:ERROR|WARN|FATAL|Exception) .*\\n");
+ private Pattern _mrJobIdPattern = Pattern.compile("job_[0-9]+_[0-9]+");
+ private Pattern _mrPigJobIdPattern = Pattern.compile("job job_[0-9]+_[0-9]+ has failed!");
+ private Pattern _mrHiveJobIdPattern = Pattern.compile("ERROR Ended Job = job_[0-9]+_[0-9]+ with errors");
+ private static long SAMPLING_SIZE = 5;
+
+ /**
+ * Failure at Azkaban job log is broadly categorized into three categorized into three categories
+ * SCHEDULERFAIL: Failure at azkaban level
+ * SCRIPTFAIL: Failure at script level
+ * MRFAIL: Failure at mapreduce level
+ * */
+ private JobState _state;
+ private LoggingEvent _exception;
+ private Set _subEvents;
+ private String _rawLog;
+
+ public AzkabanJobLogAnalyzer(String rawLog) {
+ this._rawLog = rawLog;
+ setSubEvents();
+ analyzeLog();
+ }
+
+ /**
+ * Analyzes the log to find the level of exception
+ */
+ private void analyzeLog() {
+ if (_successfulAzkabanJobPattern.matcher(_rawLog).find()) {
+ succeededAzkabanJob();
+ } else if (_failedAzkabanJobPattern.matcher(_rawLog).find()) {
+ if (!_subEvents.isEmpty()) {
+ mrLevelFailedAzkabanJob();
+ } else if (_scriptFailPattern.matcher(_rawLog).find()) {
+ scriptLevelFailedAzkabanJob();
+ } else {
+ azkabanLevelFailedAzkabanJob();
+ }
+ } else if (_killedAzkabanJobPattern.matcher(_rawLog).find()) {
+ killedAzkabanJob();
+ }
+ }
+
+ /**
+ * Sets the _state and _exception for Succeeded Azkaban job
+ */
+ private void succeededAzkabanJob() {
+ this._state = JobState.SUCCEEDED;
+ this._exception = null;
+ }
+
+ /**
+ * Sets _state and _exception for Azkaban job which failed at the MR Level
+ */
+ private void mrLevelFailedAzkabanJob() {
+ this._state = JobState.MRFAIL;
+ Matcher matcher = _scriptOrMRFailExceptionPattern.matcher(_rawLog);
+ StringBuilder exceptionBuilder = new StringBuilder();
+ long limit = SAMPLING_SIZE;
+ while (matcher.find() && limit > 0) {
+ limit--;
+ exceptionBuilder.append(matcher.group());
+ }
+ this._exception = new LoggingEvent(exceptionBuilder.toString());
+ }
+
+ /**
+ * Set _state and _exception for Azkaban job which failed at the Script Level
+ */
+ private void scriptLevelFailedAzkabanJob() {
+ this._state = JobState.SCRIPTFAIL;
+ Matcher matcher = _scriptOrMRFailExceptionPattern.matcher(_rawLog);
+ StringBuilder exceptionBuilder = new StringBuilder();
+ long limit = SAMPLING_SIZE;
+ while (matcher.find() && limit > 0) {
+ limit--;
+ exceptionBuilder.append(matcher.group());
+ }
+ this._exception = new LoggingEvent(exceptionBuilder.toString());
+ }
+
+ /**
+ * Set _state and _exception for Azkaban job which failed at the Azkaban Level
+ */
+ private void azkabanLevelFailedAzkabanJob() {
+ this._state = JobState.SCHEDULERFAIL;
+ Matcher matcher = _azkabanFailExceptionPattern.matcher(_rawLog);
+ if (matcher.find()) {
+ this._exception = new LoggingEvent(matcher.group());
+ }
+ }
+
+ /**
+ * Set _state and _exception for killed Azkaban job
+ */
+ private void killedAzkabanJob() {
+ this._state = JobState.KILLED;
+ this._exception = null;
+ }
+
+ /**
+ * @return returns Azkaban job state
+ */
+ public JobState getState() {
+ return this._state;
+ }
+
+ /**
+ * @return returns list of MR Job Ids in the given Azkaban job log
+ */
+ public Set getSubEvents() {
+ return this._subEvents;
+ }
+
+ /**
+ * Sets _subEvents equal to the list of mr job ids in the given Azkaban job log
+ */
+ private void setSubEvents() {
+ Set subEvents = new LinkedHashSet();
+
+ // check for pig jobs
+ Matcher pigJobMatcher = _mrPigJobIdPattern.matcher(_rawLog);
+ while (pigJobMatcher.find()) {
+ String pigJobFailedString = pigJobMatcher.group();
+ Matcher jobIdMatcher = _mrJobIdPattern.matcher(pigJobFailedString);
+ if (jobIdMatcher.find()) {
+ subEvents.add(jobIdMatcher.group());
+ this._subEvents = subEvents;
+ return;
+ }
+ }
+
+ pigJobMatcher.reset();
+
+ // check for hive jobs
+ Matcher hiveJobMatcher = _mrHiveJobIdPattern.matcher(_rawLog);
+ while (hiveJobMatcher.find()) {
+ String hiveJobFailedString = hiveJobMatcher.group();
+ Matcher jobIdMatcher = _mrJobIdPattern.matcher(hiveJobFailedString);
+ if (jobIdMatcher.find()) {
+ subEvents.add(jobIdMatcher.group());
+ this._subEvents = subEvents;
+ return;
+ }
+ }
+
+ // any other job than pig or hive
+ Matcher matcher = _mrJobIdPattern.matcher(_rawLog);
+ long counter = SAMPLING_SIZE; // sample the applications
+ while (matcher.find() && counter > 0) {
+ counter--;
+ subEvents.add(matcher.group());
+ }
+ this._subEvents = subEvents;
+ }
+
+ /**
+ * @return returns _exception
+ */
+ public LoggingEvent getException() {
+ return this._exception;
+ }
+}
\ No newline at end of file
diff --git a/app/com/linkedin/drelephant/exceptions/azkaban/AzkabanWorkflowClient.java b/app/com/linkedin/drelephant/exceptions/azkaban/AzkabanWorkflowClient.java
new file mode 100644
index 000000000..808fa641a
--- /dev/null
+++ b/app/com/linkedin/drelephant/exceptions/azkaban/AzkabanWorkflowClient.java
@@ -0,0 +1,456 @@
+/*
+ * Copyright 2016 LinkedIn Corp.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License"); you may not
+ * use this file except in compliance with the License. You may obtain a copy of
+ * the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package com.linkedin.drelephant.exceptions.azkaban;
+
+import com.linkedin.drelephant.exceptions.JobState;
+import com.linkedin.drelephant.exceptions.LoggingEvent;
+import com.linkedin.drelephant.exceptions.WorkflowClient;
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.io.UnsupportedEncodingException;
+import java.net.MalformedURLException;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.net.URL;
+import java.security.InvalidKeyException;
+import java.security.KeyFactory;
+import java.security.KeyManagementException;
+import java.security.KeyStoreException;
+import java.security.NoSuchAlgorithmException;
+import java.security.PrivateKey;
+import java.security.UnrecoverableKeyException;
+import java.security.cert.CertificateException;
+import java.security.cert.X509Certificate;
+import java.security.spec.InvalidKeySpecException;
+import java.security.spec.PKCS8EncodedKeySpec;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import javax.crypto.BadPaddingException;
+import javax.crypto.Cipher;
+import javax.crypto.IllegalBlockSizeException;
+import javax.crypto.NoSuchPaddingException;
+import org.apache.commons.io.IOUtils;
+import org.apache.http.HttpResponse;
+import org.apache.http.HttpStatus;
+import org.apache.http.NameValuePair;
+import org.apache.http.client.ClientProtocolException;
+import org.apache.http.client.HttpClient;
+import org.apache.http.client.entity.UrlEncodedFormEntity;
+import org.apache.http.client.methods.HttpPost;
+import org.apache.http.client.utils.URLEncodedUtils;
+import org.apache.http.conn.scheme.Scheme;
+import org.apache.http.conn.ssl.SSLSocketFactory;
+import org.apache.http.conn.ssl.TrustStrategy;
+import org.apache.http.entity.StringEntity;
+import org.apache.http.impl.client.CloseableHttpClient;
+import org.apache.http.impl.client.DefaultHttpClient;
+import org.apache.http.impl.client.HttpClientBuilder;
+import org.apache.http.message.BasicNameValuePair;
+import org.apache.http.util.EntityUtils;
+import org.apache.log4j.Logger;
+import org.codehaus.jettison.json.JSONArray;
+import org.codehaus.jettison.json.JSONException;
+import org.codehaus.jettison.json.JSONObject;
+
+
+/**
+ Client to interact with azkaban and get information about the workflow
+ */
+public class AzkabanWorkflowClient implements WorkflowClient {
+
+ private final Logger logger = Logger.getLogger(AzkabanWorkflowClient.class);
+
+ private String _workflowExecutionUrl;
+ private String _azkabanUrl;
+ private String _executionId;
+ private String _sessionId;
+ private String _username;
+ private String _password;
+
+ private String AZKABAN_LOG_OFFSET = "0";
+ private String AZKABAN_LOG_LENGTH_LIMIT = "9999999"; // limit the log limit to 10 mb
+
+ Map jobIdToLog;
+
+ /**
+ * Constructor for AzkabanWorkflowClient
+ * @param url The url of the workflow
+ * @throws URISyntaxException
+ * @throws MalformedURLException
+ */
+ public AzkabanWorkflowClient(String url)
+ throws URISyntaxException, MalformedURLException {
+ if (url == null || url.isEmpty()) {
+ throw new MalformedURLException("The Azkaban url is malformed");
+ }
+ this.setAzkabanServerUrl(url);
+ this.setExecutionId(url);
+ this._workflowExecutionUrl = url;
+ this.jobIdToLog = new HashMap();
+ }
+
+ /**
+ * Sets the azkaban server url given the azkaban workflow url
+ * @param azkabanWorkflowUrl The azkaban workflow url
+ * @throws MalformedURLException
+ * @throws URISyntaxException
+ */
+ private void setAzkabanServerUrl(String azkabanWorkflowUrl)
+ throws MalformedURLException, URISyntaxException {
+ this._azkabanUrl = "https://" + new URL(azkabanWorkflowUrl).getAuthority();
+ }
+
+ /**
+ * Sets the workflow execution id given the azkaban workflow url
+ * @param azkabanWorkflowUrl The url of the azkaban workflow
+ * @throws MalformedURLException
+ * @throws URISyntaxException
+ */
+ private void setExecutionId(String azkabanWorkflowUrl)
+ throws MalformedURLException, URISyntaxException {
+ List params = URLEncodedUtils.parse(new URI(azkabanWorkflowUrl), "UTF-8");
+ for (NameValuePair param : params) {
+ if (param.getName() == "execid") {
+ this._executionId = param.getValue();
+ }
+ }
+ }
+
+ /**
+ * Login using a private key
+ * @param username The username of the user
+ * @param _privateKey The path of the private key of the user
+ */
+ @Override
+ public void login(String username, File _privateKey) {
+ String headlessChallenge = null;
+ String decodedPwd = null;
+ try {
+ headlessChallenge = getHeadlessChallenge(username);
+ decodedPwd = decodeHeadlessChallenge(headlessChallenge, _privateKey);
+ } catch (Exception e) {
+ logger
+ .error("Unexpected error encountered while decoding headless challenge " + headlessChallenge + e.toString());
+ }
+ login(username, decodedPwd);
+ }
+
+ /**
+ * Authenticates Dr. Elephant in Azkaban and sets the sessionId
+ *
+ * @param userName The username of the user
+ * @param password The password of the user
+ */
+ @Override
+ public void login(String userName, String password) {
+ this._username = userName;
+ this._password = password;
+ List urlParameters = new ArrayList();
+ urlParameters.add(new BasicNameValuePair("action", "login"));
+ urlParameters.add(new BasicNameValuePair("username", userName));
+ urlParameters.add(new BasicNameValuePair("password", password));
+
+ try {
+ JSONObject jsonObject = fetchJson(urlParameters, _workflowExecutionUrl);
+ if (!jsonObject.has("session.id")) {
+ throw new RuntimeException("Login attempt failed. The session ID could not be obtained.");
+ }
+ this._sessionId = jsonObject.get("session.id").toString();
+ } catch (JSONException e) {
+ e.printStackTrace();
+ }
+ }
+
+ /**
+ * Makes REST API Call for given url parameters and returns the json object
+ *
+ * @param urlParameters
+ * @return Json Object in the response body
+ */
+ private JSONObject fetchJson(List urlParameters, String azkabanUrl) {
+ HttpPost httpPost = new HttpPost(azkabanUrl);
+ try {
+ httpPost.setEntity(new UrlEncodedFormEntity(urlParameters, "UTF-8"));
+ } catch (UnsupportedEncodingException e) {
+ e.printStackTrace();
+ }
+ httpPost.setHeader("Accept", "*/*");
+ httpPost.setHeader("Content-Type", "application/x-www-form-urlencoded");
+
+ HttpClient httpClient = new DefaultHttpClient();
+ JSONObject jsonObj = null;
+ try {
+ SSLSocketFactory socketFactory = new SSLSocketFactory(new TrustStrategy() {
+ @Override
+ public boolean isTrusted(X509Certificate[] x509Certificates, String s)
+ throws CertificateException {
+ return true;
+ }
+ });
+
+ Scheme scheme = new Scheme("https", 443, socketFactory);
+ httpClient.getConnectionManager().getSchemeRegistry().register(scheme);
+ HttpResponse response = httpClient.execute(httpPost);
+
+ if (response.getStatusLine().getStatusCode() != HttpStatus.SC_OK) {
+ throw new RuntimeException(
+ response.getStatusLine().toString() + "\nStatus code: " + response.getStatusLine().getStatusCode());
+ }
+
+ String result = parseContent(response.getEntity().getContent());
+ try {
+ jsonObj = new JSONObject(result);
+ if (jsonObj.has("error")) {
+ throw new RuntimeException(jsonObj.get("error").toString());
+ }
+ } catch (JSONException e) {
+ e.printStackTrace();
+ }
+ } catch (ClientProtocolException e) {
+ e.printStackTrace();
+ } catch (IOException e) {
+ e.printStackTrace();
+ } catch (NoSuchAlgorithmException e) {
+ e.printStackTrace();
+ } catch (UnrecoverableKeyException e) {
+ e.printStackTrace();
+ } catch (KeyManagementException e) {
+ e.printStackTrace();
+ } catch (KeyStoreException e) {
+ e.printStackTrace();
+ } finally {
+ httpClient.getConnectionManager().shutdown();
+ }
+ return jsonObj;
+ }
+
+ /**
+ * Parses the content given in the form of input stream to String
+ * @param response the inputstream
+ * @return The string parsed from the given inputstream
+ * @throws IOException Throws IOException if the inputstream cannot be parsed to the string
+ */
+ private String parseContent(InputStream response)
+ throws IOException {
+ BufferedReader reader = null;
+ StringBuilder result = new StringBuilder();
+ try {
+ reader = new BufferedReader(new InputStreamReader(response));
+
+ String line = null;
+ while ((line = reader.readLine()) != null) {
+ result.append(line);
+ }
+ return result.toString();
+ } catch (IOException e) {
+ e.printStackTrace();
+ } finally {
+ if (reader != null) {
+ reader.close();
+ }
+ }
+ return result.toString();
+ }
+
+ /**
+ * @param username The username of the user
+ * @return Encoded password of the user
+ * @throws IOException private String getHeadlessChallenge(String username) throws IOException {
+ */
+
+ private String getHeadlessChallenge(String username)
+ throws IOException {
+
+ CloseableHttpClient httpClient = HttpClientBuilder.create().build(); //Use this instead
+ String encodedPassword = null;
+
+ try {
+ String userUrl = _azkabanUrl + "/restli/liuser?action=headlessChallenge";
+ HttpPost request = new HttpPost(userUrl);
+ StringEntity params = new StringEntity("{\"username\":\"" + username + "\"}");
+ request.addHeader("content-type", "application/json");
+ request.setEntity(params);
+ HttpResponse response = httpClient.execute(request);
+ String responseString = EntityUtils.toString(response.getEntity());
+ JSONObject jobject = new JSONObject(responseString);
+ encodedPassword = jobject.getString("value");
+ } catch (Exception ex) {
+ throw new RuntimeException("Unexpected exception in decoding headless account " + ex.toString());
+ } finally {
+ httpClient.close();
+ return encodedPassword;
+ }
+ }
+
+ /**
+ * Decodes the encoded password using the _privateKey
+ * @param encodedPassword
+ * @param _privateKey
+ * @return The decoded password
+ * @throws IOException
+ * @throws NoSuchAlgorithmException
+ * @throws InvalidKeySpecException
+ * @throws NoSuchPaddingException
+ * @throws InvalidKeyException
+ * @throws IllegalBlockSizeException
+ * @throws BadPaddingException
+ */
+ private String decodeHeadlessChallenge(String encodedPassword, File _privateKey)
+ throws IOException, NoSuchAlgorithmException, InvalidKeySpecException, NoSuchPaddingException,
+ InvalidKeyException, IllegalBlockSizeException, BadPaddingException {
+
+ final String RSA = "RSA";
+ final String ASCII = "US-ASCII";
+
+ // Read private key from file
+ FileInputStream fstream = new FileInputStream(_privateKey);
+ byte[] sshPrivateKey = IOUtils.toByteArray(fstream);
+ PKCS8EncodedKeySpec keySpec = new PKCS8EncodedKeySpec(sshPrivateKey);
+ KeyFactory kf = KeyFactory.getInstance(RSA);
+ PrivateKey privateKey = kf.generatePrivate(keySpec);
+
+ // Init RSA decrypter with private key
+ Cipher decryptCipher = Cipher.getInstance(RSA);
+ decryptCipher.init(2, privateKey);
+
+ // Convert base 64 password string to raw bytes
+ byte[] rawBytes = org.apache.commons.codec.binary.Base64.decodeBase64(encodedPassword.getBytes(ASCII));
+
+ // Decrypt the encoded raw bytes using decrypter
+ byte[] decodedBytes = decryptCipher.doFinal(rawBytes);
+
+ // Return decoded bytes as string
+ return new String(decodedBytes, ASCII);
+ }
+
+ /**
+ * Returns the jobs from the flow
+ * @return The jobs from the flow
+ */
+ public Map getJobsFromFlow() {
+ List urlParameters = new ArrayList();
+ urlParameters.add(new BasicNameValuePair("session.id", _sessionId));
+ urlParameters.add(new BasicNameValuePair("ajax", "fetchexecflow"));
+ urlParameters.add(new BasicNameValuePair("execid", _executionId));
+
+ try {
+ JSONObject jsonObject = fetchJson(urlParameters, _workflowExecutionUrl);
+ JSONArray jobs = jsonObject.getJSONArray("nodes");
+ Map jobMap = new HashMap();
+ for (int i = 0; i < jobs.length(); i++) {
+ JSONObject job = jobs.getJSONObject(i);
+ jobMap.put(job.get("id").toString(), job.get("status").toString());
+ }
+ return jobMap;
+ } catch (JSONException e) {
+ e.printStackTrace();
+ }
+ return null;
+ }
+
+ /**
+ * Returns the azkaban flow log
+ * @param offset The offset from which logs should be found
+ * @param maximumlLogLengthLimit The maximum log length limit
+ * @return The azkaban flow logs
+ */
+ public String getAzkabanFlowLog(String offset, String maximumlLogLengthLimit) {
+ List urlParameters = new ArrayList();
+ urlParameters.add(new BasicNameValuePair("session.id", _sessionId));
+ urlParameters.add(new BasicNameValuePair("ajax", "fetchExecFlowLogs"));
+ urlParameters.add(new BasicNameValuePair("execid", _executionId));
+ urlParameters.add(new BasicNameValuePair("offset", offset));
+ urlParameters.add(new BasicNameValuePair("length", maximumlLogLengthLimit));
+
+ try {
+ JSONObject jsonObject = fetchJson(urlParameters, _workflowExecutionUrl);
+ if (jsonObject.getLong("length") == 0) {
+ throw new RuntimeException("No log found for given execution url!.");
+ }
+ return jsonObject.get("data").toString();
+ } catch (JSONException e) {
+ e.printStackTrace();
+ }
+ return null;
+ }
+
+ @Override
+ public void analyzeJob(String jobId) {
+ String rawAzkabanJobLog = getAzkabanJobLog(jobId, AZKABAN_LOG_OFFSET, AZKABAN_LOG_LENGTH_LIMIT);
+ AzkabanJobLogAnalyzer analyzedLog = new AzkabanJobLogAnalyzer(rawAzkabanJobLog);
+ jobIdToLog.put(jobId, analyzedLog);
+ }
+
+ @Override
+ public Set getYarnApplicationsFromJob(String jobId) {
+ if (!jobIdToLog.containsKey(jobId)) {
+ throw new RuntimeException("No job with id " + jobId + " found");
+ }
+ return jobIdToLog.get(jobId).getSubEvents();
+ }
+
+ @Override
+ public JobState getJobState(String jobId) {
+ if (!jobIdToLog.containsKey(jobId)) {
+ throw new RuntimeException("No job with id " + jobId + " found");
+ }
+ return jobIdToLog.get(jobId).getState();
+ }
+
+ @Override
+ public LoggingEvent getJobException(String jobId) {
+ if (!jobIdToLog.containsKey(jobId)) {
+ throw new RuntimeException("No job with id " + jobId + " found");
+ }
+ return jobIdToLog.get(jobId).getException();
+ }
+
+ /**
+ * Returns the Azkaban Job log for given Azkaban job id.
+ *
+ * @param jobId Azkaban job id
+ * @param offset Offset of log from the start
+ * @param length Maximum limit on length of log
+ * @return Azkaban job log in the form of string
+ */
+ public String getAzkabanJobLog(String jobId, String offset, String length) {
+ List urlParameters = new ArrayList();
+ urlParameters.add(new BasicNameValuePair("session.id", _sessionId));
+ urlParameters.add(new BasicNameValuePair("ajax", "fetchExecJobLogs"));
+ urlParameters.add(new BasicNameValuePair("execid", _executionId));
+ urlParameters.add(new BasicNameValuePair("jobId", jobId));
+ urlParameters.add(new BasicNameValuePair("offset", offset));
+ urlParameters.add(new BasicNameValuePair("length", length));
+ try {
+ JSONObject jsonObject = fetchJson(urlParameters, _workflowExecutionUrl);
+ if (jsonObject.getLong("length") == 0) { // To do: If length ==0 throw exception
+ logger.info("No log found for azkaban job" + jobId);
+ }
+ return jsonObject.get("data").toString();
+ } catch (JSONException e) {
+ e.printStackTrace();
+ }
+ return null;
+ }
+}
diff --git a/app/com/linkedin/drelephant/hadoop/HadoopCounterHolder.java b/app/com/linkedin/drelephant/hadoop/HadoopCounterHolder.java
deleted file mode 100644
index e0b698bf0..000000000
--- a/app/com/linkedin/drelephant/hadoop/HadoopCounterHolder.java
+++ /dev/null
@@ -1,90 +0,0 @@
-package com.linkedin.drelephant.hadoop;
-
-import java.util.Map;
-
-public class HadoopCounterHolder {
-
- private Map counters;
-
- public HadoopCounterHolder(Map counterMap) {
- counters = counterMap;
- }
-
- public long get(CounterName counterName) {
- Long value = counters.get(counterName);
- if (value == null) {
- return 0;
- }
- return value;
- }
-
- public void set(CounterName counterName, long value) {
- counters.put(counterName, value);
- }
-
- public static enum GroupName {
- FileInput("org.apache.hadoop.mapred.FileInputFormat$Counter"),
- FileSystemCounters("FileSystemCounters"),
- MapReduce("org.apache.hadoop.mapred.Task$Counter"),
- FileOutput("org.apache.hadoop.mapred.FileOutputFormat$Counter");
-
- String name;
-
- GroupName(String name) {
- this.name = name;
- }
-
- public String getName() {
- return name;
- }
- }
-
- public static enum CounterName {
- BYTES_READ(GroupName.FileInput, "BYTES_READ"),
- BYTES_WRITTEN(GroupName.FileOutput, "BYTES_WRITTEN"),
-
- FILE_BYTES_READ(GroupName.FileSystemCounters, "FILE_BYTES_READ"),
- FILE_BYTES_WRITTEN(GroupName.FileSystemCounters, "FILE_BYTES_WRITTEN"),
- HDFS_BYTES_READ(GroupName.FileSystemCounters, "HDFS_BYTES_READ"),
- HDFS_BYTES_WRITTEN(GroupName.FileSystemCounters, "HDFS_BYTES_WRITTEN"),
-
-
- MAP_INPUT_RECORDS(GroupName.MapReduce, "MAP_INPUT_RECORDS"),
- MAP_OUTPUT_RECORDS(GroupName.MapReduce, "MAP_OUTPUT_RECORDS"),
- MAP_OUTPUT_BYTES(GroupName.MapReduce, "MAP_OUTPUT_BYTES"),
- MAP_OUTPUT_MATERIALIZED_BYTES(GroupName.MapReduce, "MAP_OUTPUT_MATERIALIZED_BYTES"),
- SPLIT_RAW_BYTES(GroupName.MapReduce, "SPLIT_RAW_BYTES"),
-
- REDUCE_INPUT_GROUPS(GroupName.MapReduce, "REDUCE_INPUT_GROUPS"),
- REDUCE_SHUFFLE_BYTES(GroupName.MapReduce, "REDUCE_SHUFFLE_BYTES"),
- REDUCE_OUTPUT_RECORDS(GroupName.MapReduce, "REDUCE_OUTPUT_RECORDS"),
- REDUCE_INPUT_RECORDS(GroupName.MapReduce, "REDUCE_INPUT_RECORDS"),
-
- COMBINE_INPUT_RECORDS(GroupName.MapReduce, "COMBINE_INPUT_RECORDS"),
- COMBINE_OUTPUT_RECORDS(GroupName.MapReduce, "COMBINE_OUTPUT_RECORDS"),
- SPILLED_RECORDS(GroupName.MapReduce, "SPILLED_RECORDS"),
-
- CPU_MILLISECONDS(GroupName.MapReduce, "CPU_MILLISECONDS"),
- COMMITTED_HEAP_BYTES(GroupName.MapReduce, "COMMITTED_HEAP_BYTES"),
- PHYSICAL_MEMORY_BYTES(GroupName.MapReduce, "PHYSICAL_MEMORY_BYTES"),
- VIRTUAL_MEMORY_BYTES(GroupName.MapReduce, "VIRTUAL_MEMORY_BYTES");
-
- GroupName group;
- String name;
-
- CounterName(GroupName group, String name) {
- this.group = group;
- this.name = name;
- }
-
- public GroupName getGroup() {
- return group;
- }
-
- public String getName() {
- return name;
- }
- }
-}
-
-
diff --git a/app/com/linkedin/drelephant/hadoop/HadoopJobData.java b/app/com/linkedin/drelephant/hadoop/HadoopJobData.java
deleted file mode 100644
index b7a70c5d6..000000000
--- a/app/com/linkedin/drelephant/hadoop/HadoopJobData.java
+++ /dev/null
@@ -1,97 +0,0 @@
-package com.linkedin.drelephant.hadoop;
-
-import java.util.Properties;
-
-
-public class HadoopJobData {
- private String jobId="";
- private String username = "";
- private String url = "";
- private String jobName = "";
- private long startTime = 0;
- private HadoopCounterHolder counterHolder;
- private HadoopTaskData[] mapperData;
- private HadoopTaskData[] reducerData;
- private Properties jobConf;
-
- public HadoopJobData setJobId(String jobId) {
- this.jobId = jobId;
- return this;
- }
-
- public HadoopJobData setJobName(String jobName) {
- this.jobName = jobName;
- return this;
- }
-
- public HadoopJobData setUsername(String username) {
- this.username = username;
- return this;
- }
-
- public HadoopJobData setStartTime(long startTime) {
- this.startTime = startTime;
- return this;
- }
-
- public HadoopJobData setUrl(String url) {
- this.url = url;
- return this;
- }
-
- public HadoopJobData setCounters(HadoopCounterHolder counterHolder) {
- this.counterHolder = counterHolder;
- return this;
- }
-
- public HadoopJobData setMapperData(HadoopTaskData[] mappers) {
- this.mapperData = mappers;
- return this;
- }
-
- public HadoopJobData setReducerData(HadoopTaskData[] reducers) {
- this.reducerData = reducers;
- return this;
- }
-
- public HadoopJobData setJobConf(Properties jobConf) {
- this.jobConf = jobConf;
- return this;
- }
-
- public HadoopCounterHolder getCounters() {
- return counterHolder;
- }
-
- public HadoopTaskData[] getMapperData() {
- return mapperData;
- }
-
- public HadoopTaskData[] getReducerData() {
- return reducerData;
- }
-
- public Properties getJobConf() {
- return jobConf;
- }
-
- public String getUsername() {
- return username;
- }
-
- public long getStartTime() {
- return startTime;
- }
-
- public String getUrl() {
- return url;
- }
-
- public String getJobName() {
- return jobName;
- }
-
- public String getJobId() {
- return jobId;
- }
-}
diff --git a/app/com/linkedin/drelephant/hadoop/HadoopSecurity.java b/app/com/linkedin/drelephant/hadoop/HadoopSecurity.java
deleted file mode 100644
index aeb110b45..000000000
--- a/app/com/linkedin/drelephant/hadoop/HadoopSecurity.java
+++ /dev/null
@@ -1,58 +0,0 @@
-package com.linkedin.drelephant.hadoop;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.log4j.Logger;
-import play.Play;
-
-import java.io.IOException;
-import java.security.PrivilegedAction;
-
-
-public class HadoopSecurity {
- private static final Logger logger = Logger.getLogger(HadoopSecurity.class);
-
- private UserGroupInformation loginUser = null;
-
- private String keytabLocation;
- private String keytabUser;
- private boolean securityEnabled = false;
-
- public HadoopSecurity() throws IOException {
- Configuration conf = new Configuration();
- UserGroupInformation.setConfiguration(conf);
- securityEnabled = UserGroupInformation.isSecurityEnabled();
- if (securityEnabled) {
- keytabLocation = Play.application().configuration().getString("keytab.location");
- keytabUser = Play.application().configuration().getString("keytab.user");
- checkLogin();
- }
- }
-
- public UserGroupInformation getUGI() throws IOException {
- checkLogin();
- return loginUser;
- }
-
- public void checkLogin() throws IOException {
-
- if (loginUser == null) {
- logger.info("No login user. Creating login user");
- logger.info("Logging with " + keytabUser + " and " + keytabLocation);
- UserGroupInformation.loginUserFromKeytab(keytabUser, keytabLocation);
- loginUser = UserGroupInformation.getLoginUser();
- logger.info("Logged in with user " + loginUser);
- } else {
- loginUser.checkTGTAndReloginFromKeytab();
- }
-
- }
-
- public T doAs(PrivilegedAction action) throws IOException {
- UserGroupInformation ugi = getUGI();
- if (ugi != null) {
- return ugi.doAs(action);
- }
- return null;
- }
-}
diff --git a/app/com/linkedin/drelephant/hadoop/HadoopTaskData.java b/app/com/linkedin/drelephant/hadoop/HadoopTaskData.java
deleted file mode 100644
index 7c599ac36..000000000
--- a/app/com/linkedin/drelephant/hadoop/HadoopTaskData.java
+++ /dev/null
@@ -1,55 +0,0 @@
-package com.linkedin.drelephant.hadoop;
-
-public class HadoopTaskData{
- private HadoopCounterHolder counterHolder;
- private long startTime = 0;
- private long endTime = 0;
- private long shuffleTime = 0;
- private long sortTime = 0;
- private boolean timed = false;
-
- public HadoopTaskData(HadoopCounterHolder counterHolder, long[] time) {
- this.counterHolder = counterHolder;
- this.startTime = time[0];
- this.endTime = time[1];
- this.shuffleTime = time[2];
- this.sortTime = time[3];
- this.timed = true;
- }
-
- public HadoopTaskData(HadoopCounterHolder counterHolder) {
- this.counterHolder = counterHolder;
- }
-
- public HadoopCounterHolder getCounters() {
- return counterHolder;
- }
-
- public long getStartTime() {
- return startTime;
- }
-
- public long getEndTime() {
- return endTime;
- }
-
- public long getRunTime() {
- return endTime - startTime;
- }
-
- public long getExecutionTime() {
- return endTime - startTime - shuffleTime - sortTime;
- }
-
- public long getShuffleTime() {
- return shuffleTime;
- }
-
- public long getSortTime() {
- return sortTime;
- }
-
- public boolean timed() {
- return timed;
- }
-}
diff --git a/app/com/linkedin/drelephant/mapreduce/MapReduceMetricsAggregator.java b/app/com/linkedin/drelephant/mapreduce/MapReduceMetricsAggregator.java
new file mode 100644
index 000000000..8d2004c0e
--- /dev/null
+++ b/app/com/linkedin/drelephant/mapreduce/MapReduceMetricsAggregator.java
@@ -0,0 +1,101 @@
+/*
+ * Copyright 2016 LinkedIn Corp.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License"); you may not
+ * use this file except in compliance with the License. You may obtain a copy of
+ * the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package com.linkedin.drelephant.mapreduce;
+
+import com.linkedin.drelephant.analysis.HadoopApplicationData;
+import com.linkedin.drelephant.analysis.HadoopMetricsAggregator;
+import com.linkedin.drelephant.analysis.HadoopAggregatedData;
+import com.linkedin.drelephant.configurations.aggregator.AggregatorConfigurationData;
+import com.linkedin.drelephant.mapreduce.data.MapReduceApplicationData;
+import org.apache.commons.io.FileUtils;
+import org.apache.log4j.Logger;
+
+
+public class MapReduceMetricsAggregator implements HadoopMetricsAggregator {
+
+ private static final Logger logger = Logger.getLogger(MapReduceMetricsAggregator.class);
+ private static final String MAP_CONTAINER_CONFIG = "mapreduce.map.memory.mb";
+ private static final String REDUCER_CONTAINER_CONFIG = "mapreduce.reduce.memory.mb";
+ private static final String REDUCER_SLOW_START_CONFIG = "mapreduce.job.reduce.slowstart.completedmaps";
+ private static final long CONTAINER_MEMORY_DEFAULT_MBYTES = 2048L;
+
+ private HadoopAggregatedData _hadoopAggregatedData = null;
+ private TaskLevelAggregatedMetrics mapTasks;
+ private TaskLevelAggregatedMetrics reduceTasks;
+
+ private AggregatorConfigurationData _aggregatorConfigurationData;
+
+ public MapReduceMetricsAggregator(AggregatorConfigurationData _aggregatorConfigurationData) {
+ this._aggregatorConfigurationData = _aggregatorConfigurationData;
+ _hadoopAggregatedData = new HadoopAggregatedData();
+ }
+
+ @Override
+ public void aggregate(HadoopApplicationData hadoopData) {
+
+ MapReduceApplicationData data = (MapReduceApplicationData) hadoopData;
+
+ long mapTaskContainerSize = getMapContainerSize(data);
+ long reduceTaskContainerSize = getReducerContainerSize(data);
+
+ int reduceTaskSlowStartPercentage =
+ (int) (Double.parseDouble(data.getConf().getProperty(REDUCER_SLOW_START_CONFIG)) * 100);
+
+
+ //overwrite reduceTaskSlowStartPercentage to 100%. TODO: make use of the slow start percent
+ reduceTaskSlowStartPercentage = 100;
+
+ mapTasks = new TaskLevelAggregatedMetrics(data.getMapperData(), mapTaskContainerSize, data.getStartTime());
+
+ long reduceIdealStartTime = mapTasks.getNthPercentileFinishTime(reduceTaskSlowStartPercentage);
+
+ // Mappers list is empty
+ if(reduceIdealStartTime == -1) {
+ // ideal start time for reducer is infinite since it cannot start
+ reduceIdealStartTime = Long.MAX_VALUE;
+ }
+
+ reduceTasks = new TaskLevelAggregatedMetrics(data.getReducerData(), reduceTaskContainerSize, reduceIdealStartTime);
+
+ _hadoopAggregatedData.setResourceUsed(mapTasks.getResourceUsed() + reduceTasks.getResourceUsed());
+ _hadoopAggregatedData.setTotalDelay(mapTasks.getDelay() + reduceTasks.getDelay());
+ _hadoopAggregatedData.setResourceWasted(mapTasks.getResourceWasted() + reduceTasks.getResourceWasted());
+ }
+
+ @Override
+ public HadoopAggregatedData getResult() {
+ return _hadoopAggregatedData;
+ }
+
+ private long getMapContainerSize(HadoopApplicationData data) {
+ try {
+ long value = Long.parseLong(data.getConf().getProperty(MAP_CONTAINER_CONFIG));
+ return (value < 0) ? CONTAINER_MEMORY_DEFAULT_MBYTES : value;
+ } catch ( NumberFormatException ex) {
+ return CONTAINER_MEMORY_DEFAULT_MBYTES;
+ }
+ }
+
+ private long getReducerContainerSize(HadoopApplicationData data) {
+ try {
+ long value = Long.parseLong(data.getConf().getProperty(REDUCER_CONTAINER_CONFIG));
+ return (value < 0) ? CONTAINER_MEMORY_DEFAULT_MBYTES : value;
+ } catch ( NumberFormatException ex) {
+ return CONTAINER_MEMORY_DEFAULT_MBYTES;
+ }
+ }
+}
diff --git a/app/com/linkedin/drelephant/mapreduce/TaskLevelAggregatedMetrics.java b/app/com/linkedin/drelephant/mapreduce/TaskLevelAggregatedMetrics.java
new file mode 100644
index 000000000..33b90b7e0
--- /dev/null
+++ b/app/com/linkedin/drelephant/mapreduce/TaskLevelAggregatedMetrics.java
@@ -0,0 +1,149 @@
+/*
+ * Copyright 2016 LinkedIn Corp.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License"); you may not
+ * use this file except in compliance with the License. You may obtain a copy of
+ * the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package com.linkedin.drelephant.mapreduce;
+
+import com.linkedin.drelephant.mapreduce.data.MapReduceCounterData;
+import com.linkedin.drelephant.mapreduce.data.MapReduceTaskData;
+import com.linkedin.drelephant.math.Statistics;
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.commons.io.FileUtils;
+import org.apache.log4j.Logger;
+
+
+public class TaskLevelAggregatedMetrics {
+
+ private static final Logger logger = Logger.getLogger(TaskLevelAggregatedMetrics.class);
+
+ private long _delay = 0;
+ private long _resourceWasted = 0;
+ private long _resourceUsed = 0;
+
+ private List finishTimes = new ArrayList();
+ private List durations = new ArrayList();
+
+ private static final double MEMORY_BUFFER = 1.5;
+ private static final double CLUSTER_MEMORY_FACTOR = 2.1;
+
+ /**
+ * Returns the nth percentile finish job
+ * @param percentile The percentile of finish job to return
+ * @return The nth percentile finish job
+ */
+ public long getNthPercentileFinishTime(int percentile)
+ {
+ if(finishTimes == null || finishTimes.size() == 0 ) {
+ return -1;
+ }
+ return Statistics.percentile(finishTimes, percentile);
+ }
+
+ /**
+ * Constructor for TaskLevelAggregatedMetrics
+ * @param taskData Array containing the task data for mappers and/or reducers
+ * @param containerSize The container size of the tasks
+ * @param idealStartTime The ideal start time for the task. For mappers it is the submit time, for
+ * reducers, it is the time when the number of completed maps become more than
+ * the slow start time.
+ */
+ public TaskLevelAggregatedMetrics(MapReduceTaskData[] taskData, long containerSize, long idealStartTime) {
+ compute(taskData, containerSize, idealStartTime);
+ }
+
+ /**
+ * Returns the overall delay for the tasks.
+ * @return The delay of the tasks.
+ */
+ public long getDelay() {
+ return _delay;
+ }
+
+ /**
+ * Retruns the resources wasted by all the tasks in MB Seconds
+ * @return The wasted resources of all the tasks in MB Seconds
+ */
+ public long getResourceWasted() {
+ return _resourceWasted;
+ }
+
+ /**
+ * Returns the resource used by all the tasks in MB Seconds
+ * @return The total resources used by all tasks in MB Seconds
+ */
+ public long getResourceUsed() {
+ return _resourceUsed;
+ }
+
+ /**
+ * Computes the aggregated metrics -> peakMemory, delay, total task duration, wasted resources and memory usage.
+ * @param taskDatas
+ * @param containerSize
+ * @param idealStartTime
+ */
+ private void compute(MapReduceTaskData[] taskDatas, long containerSize, long idealStartTime) {
+
+ long peakMemoryNeed = 0;
+ long taskFinishTimeMax = 0;
+ long taskDurationMax = 0;
+
+ // if there are zero tasks, then nothing to compute.
+ if(taskDatas == null || taskDatas.length == 0) {
+ return;
+ }
+
+ for (MapReduceTaskData taskData: taskDatas) {
+ long taskMemory = taskData.getCounters().get(MapReduceCounterData.CounterName.PHYSICAL_MEMORY_BYTES)/ FileUtils.ONE_MB; // MB
+ long taskVM = taskData.getCounters().get(MapReduceCounterData.CounterName.VIRTUAL_MEMORY_BYTES)/ FileUtils.ONE_MB; // MB
+ long taskDuration = taskData.getFinishTimeMs() - taskData.getStartTimeMs(); // Milliseconds
+ long taskCost = (containerSize) * (taskDuration / Statistics.SECOND_IN_MS); // MB Seconds
+
+ durations.add(taskDuration);
+ finishTimes.add(taskData.getFinishTimeMs());
+
+ //peak Memory usage
+ long memoryRequiredForVM = (long) (taskVM/CLUSTER_MEMORY_FACTOR);
+ long biggerMemoryRequirement = memoryRequiredForVM > taskMemory ? memoryRequiredForVM : taskMemory;
+ peakMemoryNeed = biggerMemoryRequirement > peakMemoryNeed ? biggerMemoryRequirement : peakMemoryNeed;
+
+ if(taskFinishTimeMax < taskData.getFinishTimeMs()) {
+ taskFinishTimeMax = taskData.getFinishTimeMs();
+ }
+
+ if(taskDurationMax < taskDuration) {
+ taskDurationMax = taskDuration;
+ }
+ _resourceUsed += taskCost;
+ }
+
+ // Compute the delay in starting the task.
+ _delay = taskFinishTimeMax - (idealStartTime + taskDurationMax);
+
+ // invalid delay
+ if(_delay < 0) {
+ _delay = 0;
+ }
+
+ // wastedResources
+ long wastedMemory = containerSize - (long) (peakMemoryNeed * MEMORY_BUFFER); // give a 50% buffer
+ if(wastedMemory > 0) {
+ for (long duration : durations) {
+ _resourceWasted += (wastedMemory) * (duration / Statistics.SECOND_IN_MS); // MB Seconds
+ }
+ }
+ }
+
+}
diff --git a/app/com/linkedin/drelephant/mapreduce/data/MapReduceApplicationData.java b/app/com/linkedin/drelephant/mapreduce/data/MapReduceApplicationData.java
new file mode 100644
index 000000000..c30c46f64
--- /dev/null
+++ b/app/com/linkedin/drelephant/mapreduce/data/MapReduceApplicationData.java
@@ -0,0 +1,198 @@
+/*
+ * Copyright 2016 LinkedIn Corp.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License"); you may not
+ * use this file except in compliance with the License. You may obtain a copy of
+ * the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package com.linkedin.drelephant.mapreduce.data;
+
+import com.linkedin.drelephant.analysis.ApplicationType;
+import com.linkedin.drelephant.analysis.HadoopApplicationData;
+import java.util.Properties;
+
+
+/**
+ * This class contains the MapReduce Application Information
+ */
+public class MapReduceApplicationData implements HadoopApplicationData {
+ private static final ApplicationType APPLICATION_TYPE = new ApplicationType("MAPREDUCE");
+
+ private boolean _succeeded = true;
+ private String _diagnosticInfo = "";
+ private String _appId = "";
+ private String _jobId = "";
+ private String _username = "";
+ private String _url = "";
+ private String _jobName = "";
+ private long _submitTime = 0;
+ private long _startTime = 0;
+ private long _finishTime = 0;
+
+ private MapReduceCounterData _counterHolder;
+ private MapReduceTaskData[] _mapperData;
+ private MapReduceTaskData[] _reducerData;
+ private Properties _jobConf;
+ private boolean _isRetry = false;
+
+ public MapReduceApplicationData setSucceeded(boolean succeeded) {
+ this._succeeded = succeeded;
+ return this;
+ }
+
+ public MapReduceApplicationData setDiagnosticInfo(String diagnosticInfo) {
+ this._diagnosticInfo = diagnosticInfo;
+ return this;
+ }
+
+ public MapReduceApplicationData setRetry(boolean isRetry) {
+ this._isRetry = isRetry;
+ return this;
+ }
+
+ public MapReduceApplicationData setAppId(String appId) {
+ this._appId = appId;
+ return this;
+ }
+
+ public MapReduceApplicationData setJobId(String jobId) {
+ this._jobId = jobId;
+ return this;
+ }
+
+ public MapReduceApplicationData setJobName(String jobName) {
+ this._jobName = jobName;
+ return this;
+ }
+
+ public MapReduceApplicationData setUsername(String username) {
+ this._username = username;
+ return this;
+ }
+
+ public MapReduceApplicationData setSubmitTime(long submitTime) {
+ this._submitTime = submitTime;
+ return this;
+ }
+
+ public MapReduceApplicationData setStartTime(long startTime) {
+ this._startTime = startTime;
+ return this;
+ }
+
+ public MapReduceApplicationData setFinishTime(long finishTime) {
+ this._finishTime = finishTime;
+ return this;
+ }
+
+ public MapReduceApplicationData setUrl(String url) {
+ this._url = url;
+ return this;
+ }
+
+ public MapReduceApplicationData setCounters(MapReduceCounterData counterHolder) {
+ this._counterHolder = counterHolder;
+ return this;
+ }
+
+ public MapReduceApplicationData setMapperData(MapReduceTaskData[] mappers) {
+ this._mapperData = mappers;
+ return this;
+ }
+
+ public MapReduceApplicationData setReducerData(MapReduceTaskData[] reducers) {
+ this._reducerData = reducers;
+ return this;
+ }
+
+ public MapReduceApplicationData setJobConf(Properties jobConf) {
+ this._jobConf = jobConf;
+ return this;
+ }
+
+ public MapReduceCounterData getCounters() {
+ return _counterHolder;
+ }
+
+ public MapReduceTaskData[] getMapperData() {
+ return _mapperData;
+ }
+
+ public MapReduceTaskData[] getReducerData() {
+ return _reducerData;
+ }
+
+ @Override
+ public String getAppId() {
+ return _appId;
+ }
+
+ @Override
+ public Properties getConf() {
+ return _jobConf;
+ }
+
+ @Override
+ public ApplicationType getApplicationType() {
+ return APPLICATION_TYPE;
+ }
+
+ @Override
+ public boolean isEmpty() {
+ return _succeeded && getMapperData().length == 0 && getReducerData().length == 0;
+ }
+
+ public String getUsername() {
+ return _username;
+ }
+
+ public long getSubmitTime() {
+ return _submitTime;
+ }
+
+ public long getStartTime() {
+ return _startTime;
+ }
+
+ public long getFinishTime() {
+ return _finishTime;
+ }
+
+ public String getUrl() {
+ return _url;
+ }
+
+ public String getJobName() {
+ return _jobName;
+ }
+
+ public boolean isRetryJob() {
+ return _isRetry;
+ }
+
+ public String getJobId() {
+ return _jobId;
+ }
+
+ public boolean getSucceeded() {
+ return _succeeded;
+ }
+
+ public String getDiagnosticInfo() {
+ return _diagnosticInfo;
+ }
+
+ @Override
+ public String toString() {
+ return "id: " + getJobId() + ", name:" + getJobName();
+ }
+}
diff --git a/app/com/linkedin/drelephant/mapreduce/data/MapReduceCounterData.java b/app/com/linkedin/drelephant/mapreduce/data/MapReduceCounterData.java
new file mode 100644
index 000000000..36d9f11d0
--- /dev/null
+++ b/app/com/linkedin/drelephant/mapreduce/data/MapReduceCounterData.java
@@ -0,0 +1,181 @@
+/*
+ * Copyright 2016 LinkedIn Corp.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License"); you may not
+ * use this file except in compliance with the License. You may obtain a copy of
+ * the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package com.linkedin.drelephant.mapreduce.data;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Set;
+
+
+/**
+ * This class manages all the MapReduce Counters
+ */
+public class MapReduceCounterData {
+
+ // This is a map of group to all the counters in the group and their values.
+ private final Map> _pubCounters;
+
+ public String toString() {
+ return _pubCounters.toString();
+ }
+
+ public MapReduceCounterData() {
+ _pubCounters = new HashMap>(8);
+ }
+
+ /**
+ * @return the value of the counter, 0 if not present.
+ * This method is only used for job heuristics
+ * Due to h1 & h2 counter group incompatibility, we iterate every counter group (4 by default)
+ * to find a matching counter name, otherwise we have to hardcode the h1&h2 version of counter group
+ * and try twice with two names for each counter in this method.
+ * This approach is less efficient, but cleaner.
+ */
+ public long get(CounterName counterName) {
+ // For each counter group, try to match the counter name
+ for(Map counterGrp : _pubCounters.values()) {
+ if(counterGrp.containsKey(counterName._name)) {
+ return counterGrp.get(counterName._name);
+ }
+ }
+ return 0;
+ }
+
+ public void set(CounterName counterName, long value) {
+ set(counterName.getGroupName(), counterName.getName(), value);
+ }
+
+ /**
+ * Set the value of a counter that we may want to publish later
+ *
+ * @param groupName
+ * @param counterName
+ * @param value
+ */
+ public void set(String groupName, String counterName, long value) {
+ Map counterMap = _pubCounters.get(groupName);
+ if (counterMap == null) {
+ counterMap = new HashMap(4);
+ _pubCounters.put(groupName, counterMap);
+ }
+ counterMap.put(counterName, value);
+ }
+
+ public Set getGroupNames() {
+ Set groupNames = _pubCounters.keySet();
+ return Collections.unmodifiableSet(groupNames);
+ }
+
+ /**
+ * Get the values of all counters in a group
+ * @param groupName
+ * @return A map containing all the values of counters in a group.
+ */
+ public Map getAllCountersInGroup(String groupName) {
+ Map counterMap = _pubCounters.get(groupName);
+ if (counterMap == null) {
+ counterMap = new HashMap(1);
+ }
+ return counterMap;
+ }
+
+ public static enum GroupName {
+ FileInput,
+ FileSystemCounters,
+ MapReduce,
+ FileOutput;
+ }
+
+ public static enum CounterName {
+ BYTES_READ(GroupName.FileInput, "BYTES_READ", "Bytes Read"),
+ BYTES_WRITTEN(GroupName.FileOutput, "BYTES_WRITTEN", "Bytes Written"),
+
+ FILE_BYTES_READ(GroupName.FileSystemCounters, "FILE_BYTES_READ", "FILE_BYTES_READ"),
+ FILE_BYTES_WRITTEN(GroupName.FileSystemCounters, "FILE_BYTES_WRITTEN", "FILE_BYTES_WRITTEN"),
+ HDFS_BYTES_READ(GroupName.FileSystemCounters, "HDFS_BYTES_READ", "HDFS_BYTES_READ"),
+ HDFS_BYTES_WRITTEN(GroupName.FileSystemCounters, "HDFS_BYTES_WRITTEN", "HDFS_BYTES_WRITTEN"),
+
+ MAP_INPUT_RECORDS(GroupName.MapReduce, "MAP_INPUT_RECORDS", "Map input records"),
+ MAP_OUTPUT_RECORDS(GroupName.MapReduce, "MAP_OUTPUT_RECORDS", "Map output records"),
+ MAP_OUTPUT_BYTES(GroupName.MapReduce, "MAP_OUTPUT_BYTES", "Map output bytes"),
+ MAP_OUTPUT_MATERIALIZED_BYTES(GroupName.MapReduce, "MAP_OUTPUT_MATERIALIZED_BYTES", "Map output materialized bytes"),
+ SPLIT_RAW_BYTES(GroupName.MapReduce, "SPLIT_RAW_BYTES", "SPLIT_RAW_BYTES"),
+
+ REDUCE_INPUT_GROUPS(GroupName.MapReduce, "REDUCE_INPUT_GROUPS", "Reduce input groups"),
+ REDUCE_SHUFFLE_BYTES(GroupName.MapReduce, "REDUCE_SHUFFLE_BYTES", "Reduce shuffle bytes"),
+ REDUCE_OUTPUT_RECORDS(GroupName.MapReduce, "REDUCE_OUTPUT_RECORDS", "Reduce output records"),
+ REDUCE_INPUT_RECORDS(GroupName.MapReduce, "REDUCE_INPUT_RECORDS", "Reduce input records"),
+
+ COMBINE_INPUT_RECORDS(GroupName.MapReduce, "COMBINE_INPUT_RECORDS", "Combine input records"),
+ COMBINE_OUTPUT_RECORDS(GroupName.MapReduce, "COMBINE_OUTPUT_RECORDS", "Combine output records"),
+ SPILLED_RECORDS(GroupName.MapReduce, "SPILLED_RECORDS", "Spilled Records"),
+
+ CPU_MILLISECONDS(GroupName.MapReduce, "CPU_MILLISECONDS", "CPU time spent (ms)"),
+ GC_MILLISECONDS(GroupName.MapReduce, "GC_TIME_MILLIS", "GC time elapsed (ms)"),
+ COMMITTED_HEAP_BYTES(GroupName.MapReduce, "COMMITTED_HEAP_BYTES", "Total committed heap usage (bytes)"),
+ PHYSICAL_MEMORY_BYTES(GroupName.MapReduce, "PHYSICAL_MEMORY_BYTES", "Physical memory (bytes) snapshot"),
+ VIRTUAL_MEMORY_BYTES(GroupName.MapReduce, "VIRTUAL_MEMORY_BYTES", "Virtual memory (bytes) snapshot");
+
+ GroupName _group;
+ String _name;
+ String _displayName;
+
+ CounterName(GroupName group, String name, String displayName) {
+ this._group = group;
+ this._name = name;
+ this._displayName = displayName;
+ }
+
+ static Map _counterDisplayNameMap;
+ static Map _counterNameMap;
+ static {
+ _counterDisplayNameMap = new HashMap