From 625039ef20e6011ab360131d70582a6e4bf2ec1d Mon Sep 17 00:00:00 2001 From: Subru Krishnan Date: Wed, 25 Oct 2017 15:51:27 -0700 Subject: [PATCH] HADOOP-14840. Tool to estimate resource requirements of an application pipeline based on prior executions. (Rui Li via Subru). --- LICENSE.txt | 1 + .../assemblies/hadoop-resourceestimator.xml | 42 ++ .../resources/assemblies/hadoop-tools.xml | 11 + hadoop-project/pom.xml | 5 + hadoop-project/src/site/site.xml | 1 + .../hadoop-resourceestimator/README.md | 19 + hadoop-tools/hadoop-resourceestimator/pom.xml | 174 +++++++ .../src/config/checkstyle.xml | 50 ++ .../src/main/bin/estimator.cmd | 52 ++ .../src/main/bin/estimator.sh | 71 +++ .../src/main/bin/start-estimator.cmd | 37 ++ .../src/main/bin/start-estimator.sh | 42 ++ .../src/main/bin/stop-estimator.cmd | 37 ++ .../src/main/bin/stop-estimator.sh | 42 ++ .../main/conf/resourceestimator-config.xml | 85 ++++ .../main/data/resourceEstimatorService.txt | 2 + .../common/api/RecurrenceId.java | 95 ++++ .../common/api/ResourceSkyline.java | 211 ++++++++ .../common/api/package-info.java | 23 + .../ResourceEstimatorConfiguration.java | 125 +++++ .../common/config/ResourceEstimatorUtil.java | 81 +++ .../common/config/package-info.java | 23 + .../exception/ResourceEstimatorException.java | 35 ++ .../common/exception/package-info.java | 23 + .../RLESparseResourceAllocationSerDe.java | 77 +++ .../common/serialization/ResourceSerDe.java | 61 +++ .../common/serialization/package-info.java | 24 + .../service/ResourceEstimatorServer.java | 146 ++++++ .../service/ResourceEstimatorService.java | 238 +++++++++ .../service/ShutdownHook.java | 45 ++ .../service/package-info.java | 23 + .../skylinestore/api/HistorySkylineStore.java | 99 ++++ .../api/PredictionSkylineStore.java | 60 +++ .../skylinestore/api/SkylineStore.java | 30 ++ .../skylinestore/api/package-info.java | 23 + .../DuplicateRecurrenceIdException.java | 33 ++ .../EmptyResourceSkylineException.java | 33 ++ .../exceptions/NullPipelineIdException.java | 32 ++ ...lRLESparseResourceAllocationException.java | 33 ++ .../exceptions/NullRecurrenceIdException.java | 32 ++ .../NullResourceSkylineException.java | 32 ++ .../RecurrenceIdNotFoundException.java | 33 ++ .../exceptions/SkylineStoreException.java | 33 ++ .../skylinestore/exceptions/package-info.java | 24 + .../skylinestore/impl/InMemoryStore.java | 256 ++++++++++ .../skylinestore/impl/package-info.java | 23 + .../validator/SkylineStoreValidator.java | 118 +++++ .../skylinestore/validator/package-info.java | 23 + .../resourceestimator/solver/api/Solver.java | 76 +++ .../solver/api/package-info.java | 23 + .../exceptions/InvalidInputException.java | 34 ++ .../exceptions/InvalidSolverException.java | 34 ++ .../solver/exceptions/SolverException.java | 34 ++ .../solver/exceptions/package-info.java | 24 + .../solver/impl/BaseSolver.java | 94 ++++ .../solver/impl/LpSolver.java | 340 +++++++++++++ .../solver/impl/package-info.java | 23 + .../solver/preprocess/SolverPreprocessor.java | 219 +++++++++ .../solver/preprocess/package-info.java | 23 + .../translator/api/JobMetaData.java | 163 ++++++ .../translator/api/LogParser.java | 65 +++ .../translator/api/SingleLineParser.java | 52 ++ .../translator/api/package-info.java | 23 + .../DataFieldNotFoundException.java | 32 ++ .../translator/exceptions/package-info.java | 23 + .../translator/impl/BaseLogParser.java | 125 +++++ .../translator/impl/LogParserUtil.java | 97 ++++ .../impl/NativeSingleLineParser.java | 120 +++++ .../translator/impl/RmSingleLineParser.java | 203 ++++++++ .../translator/impl/package-info.java | 23 + .../translator/validator/ParserValidator.java | 41 ++ .../translator/validator/package-info.java | 23 + .../ResourceEstimatorServer/.gitignore | 14 + .../src/site/markdown/ResourceEstimator.md | 181 +++++++ .../src/site/resources/css/site.css | 29 ++ .../common/api/TestResourceSkyline.java | 128 +++++ .../TestHistorySkylineSerDe.java | 134 +++++ .../serialization/TestResourceSerDe.java | 64 +++ .../TestResourceSkylineSerDe.java | 112 +++++ .../common/serialization/package-info.java | 24 + .../service/GuiceServletConfig.java | 42 ++ .../service/TestResourceEstimatorService.java | 282 +++++++++++ .../skylinestore/impl/TestInMemoryStore.java | 32 ++ .../skylinestore/impl/TestSkylineStore.java | 464 ++++++++++++++++++ .../solver/impl/TestLpSolver.java | 112 +++++ .../solver/impl/TestSolver.java | 73 +++ .../translator/api/TestJobMetaData.java | 163 ++++++ .../translator/impl/TestNativeParser.java | 115 +++++ .../translator/impl/TestRmParser.java | 239 +++++++++ .../src/test/resources/log4j.properties | 25 + .../src/test/resources/lp/answer.txt | 7 + .../src/test/resources/lp/tinySample.txt | 2 + .../src/test/resources/lp/tpch_q12.txt | 13 + .../resources/resourceEstimatorService.txt | 2 + .../resources/resourceestimator-config.xml | 85 ++++ .../src/test/resources/trace/invalidLog1.txt | 4 + .../src/test/resources/trace/invalidLog2.txt | 3 + .../src/test/resources/trace/invalidLog3.txt | 5 + .../src/test/resources/trace/invalidLog4.txt | 5 + .../src/test/resources/trace/invalidLog5.txt | 7 + .../src/test/resources/trace/invalidLog6.txt | 5 + .../src/test/resources/trace/invalidLog7.txt | 5 + .../src/test/resources/trace/invalidLog8.txt | 6 + .../src/test/resources/trace/invalidLog9.txt | 6 + .../src/test/resources/trace/nativeLog.txt | 1 + .../src/test/resources/trace/rmLog.txt | 30 ++ hadoop-tools/hadoop-tools-dist/pom.xml | 6 + hadoop-tools/pom.xml | 1 + 108 files changed, 7158 insertions(+) create mode 100644 hadoop-assemblies/src/main/resources/assemblies/hadoop-resourceestimator.xml create mode 100644 hadoop-tools/hadoop-resourceestimator/README.md create mode 100644 hadoop-tools/hadoop-resourceestimator/pom.xml create mode 100644 hadoop-tools/hadoop-resourceestimator/src/config/checkstyle.xml create mode 100644 hadoop-tools/hadoop-resourceestimator/src/main/bin/estimator.cmd create mode 100644 hadoop-tools/hadoop-resourceestimator/src/main/bin/estimator.sh create mode 100644 hadoop-tools/hadoop-resourceestimator/src/main/bin/start-estimator.cmd create mode 100644 hadoop-tools/hadoop-resourceestimator/src/main/bin/start-estimator.sh create mode 100644 hadoop-tools/hadoop-resourceestimator/src/main/bin/stop-estimator.cmd create mode 100644 hadoop-tools/hadoop-resourceestimator/src/main/bin/stop-estimator.sh create mode 100644 hadoop-tools/hadoop-resourceestimator/src/main/conf/resourceestimator-config.xml create mode 100644 hadoop-tools/hadoop-resourceestimator/src/main/data/resourceEstimatorService.txt create mode 100644 hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/common/api/RecurrenceId.java create mode 100644 hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/common/api/ResourceSkyline.java create mode 100644 hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/common/api/package-info.java create mode 100644 hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/common/config/ResourceEstimatorConfiguration.java create mode 100644 hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/common/config/ResourceEstimatorUtil.java create mode 100644 hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/common/config/package-info.java create mode 100644 hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/common/exception/ResourceEstimatorException.java create mode 100644 hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/common/exception/package-info.java create mode 100644 hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/common/serialization/RLESparseResourceAllocationSerDe.java create mode 100644 hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/common/serialization/ResourceSerDe.java create mode 100644 hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/common/serialization/package-info.java create mode 100644 hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/service/ResourceEstimatorServer.java create mode 100644 hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/service/ResourceEstimatorService.java create mode 100644 hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/service/ShutdownHook.java create mode 100644 hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/service/package-info.java create mode 100644 hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/skylinestore/api/HistorySkylineStore.java create mode 100644 hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/skylinestore/api/PredictionSkylineStore.java create mode 100644 hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/skylinestore/api/SkylineStore.java create mode 100644 hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/skylinestore/api/package-info.java create mode 100644 hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/skylinestore/exceptions/DuplicateRecurrenceIdException.java create mode 100644 hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/skylinestore/exceptions/EmptyResourceSkylineException.java create mode 100644 hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/skylinestore/exceptions/NullPipelineIdException.java create mode 100644 hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/skylinestore/exceptions/NullRLESparseResourceAllocationException.java create mode 100644 hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/skylinestore/exceptions/NullRecurrenceIdException.java create mode 100644 hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/skylinestore/exceptions/NullResourceSkylineException.java create mode 100644 hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/skylinestore/exceptions/RecurrenceIdNotFoundException.java create mode 100644 hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/skylinestore/exceptions/SkylineStoreException.java create mode 100644 hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/skylinestore/exceptions/package-info.java create mode 100644 hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/skylinestore/impl/InMemoryStore.java create mode 100644 hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/skylinestore/impl/package-info.java create mode 100644 hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/skylinestore/validator/SkylineStoreValidator.java create mode 100644 hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/skylinestore/validator/package-info.java create mode 100644 hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/solver/api/Solver.java create mode 100644 hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/solver/api/package-info.java create mode 100644 hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/solver/exceptions/InvalidInputException.java create mode 100644 hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/solver/exceptions/InvalidSolverException.java create mode 100644 hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/solver/exceptions/SolverException.java create mode 100644 hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/solver/exceptions/package-info.java create mode 100644 hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/solver/impl/BaseSolver.java create mode 100644 hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/solver/impl/LpSolver.java create mode 100644 hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/solver/impl/package-info.java create mode 100644 hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/solver/preprocess/SolverPreprocessor.java create mode 100644 hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/solver/preprocess/package-info.java create mode 100644 hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/translator/api/JobMetaData.java create mode 100644 hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/translator/api/LogParser.java create mode 100644 hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/translator/api/SingleLineParser.java create mode 100644 hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/translator/api/package-info.java create mode 100644 hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/translator/exceptions/DataFieldNotFoundException.java create mode 100644 hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/translator/exceptions/package-info.java create mode 100644 hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/translator/impl/BaseLogParser.java create mode 100644 hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/translator/impl/LogParserUtil.java create mode 100644 hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/translator/impl/NativeSingleLineParser.java create mode 100644 hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/translator/impl/RmSingleLineParser.java create mode 100644 hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/translator/impl/package-info.java create mode 100644 hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/translator/validator/ParserValidator.java create mode 100644 hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/translator/validator/package-info.java create mode 100644 hadoop-tools/hadoop-resourceestimator/src/main/resources/webapps/ResourceEstimatorServer/.gitignore create mode 100644 hadoop-tools/hadoop-resourceestimator/src/site/markdown/ResourceEstimator.md create mode 100644 hadoop-tools/hadoop-resourceestimator/src/site/resources/css/site.css create mode 100644 hadoop-tools/hadoop-resourceestimator/src/test/java/org/apache/hadoop/resourceestimator/common/api/TestResourceSkyline.java create mode 100644 hadoop-tools/hadoop-resourceestimator/src/test/java/org/apache/hadoop/resourceestimator/common/serialization/TestHistorySkylineSerDe.java create mode 100644 hadoop-tools/hadoop-resourceestimator/src/test/java/org/apache/hadoop/resourceestimator/common/serialization/TestResourceSerDe.java create mode 100644 hadoop-tools/hadoop-resourceestimator/src/test/java/org/apache/hadoop/resourceestimator/common/serialization/TestResourceSkylineSerDe.java create mode 100644 hadoop-tools/hadoop-resourceestimator/src/test/java/org/apache/hadoop/resourceestimator/common/serialization/package-info.java create mode 100644 hadoop-tools/hadoop-resourceestimator/src/test/java/org/apache/hadoop/resourceestimator/service/GuiceServletConfig.java create mode 100644 hadoop-tools/hadoop-resourceestimator/src/test/java/org/apache/hadoop/resourceestimator/service/TestResourceEstimatorService.java create mode 100644 hadoop-tools/hadoop-resourceestimator/src/test/java/org/apache/hadoop/resourceestimator/skylinestore/impl/TestInMemoryStore.java create mode 100644 hadoop-tools/hadoop-resourceestimator/src/test/java/org/apache/hadoop/resourceestimator/skylinestore/impl/TestSkylineStore.java create mode 100644 hadoop-tools/hadoop-resourceestimator/src/test/java/org/apache/hadoop/resourceestimator/solver/impl/TestLpSolver.java create mode 100644 hadoop-tools/hadoop-resourceestimator/src/test/java/org/apache/hadoop/resourceestimator/solver/impl/TestSolver.java create mode 100644 hadoop-tools/hadoop-resourceestimator/src/test/java/org/apache/hadoop/resourceestimator/translator/api/TestJobMetaData.java create mode 100644 hadoop-tools/hadoop-resourceestimator/src/test/java/org/apache/hadoop/resourceestimator/translator/impl/TestNativeParser.java create mode 100644 hadoop-tools/hadoop-resourceestimator/src/test/java/org/apache/hadoop/resourceestimator/translator/impl/TestRmParser.java create mode 100644 hadoop-tools/hadoop-resourceestimator/src/test/resources/log4j.properties create mode 100644 hadoop-tools/hadoop-resourceestimator/src/test/resources/lp/answer.txt create mode 100644 hadoop-tools/hadoop-resourceestimator/src/test/resources/lp/tinySample.txt create mode 100644 hadoop-tools/hadoop-resourceestimator/src/test/resources/lp/tpch_q12.txt create mode 100644 hadoop-tools/hadoop-resourceestimator/src/test/resources/resourceEstimatorService.txt create mode 100644 hadoop-tools/hadoop-resourceestimator/src/test/resources/resourceestimator-config.xml create mode 100644 hadoop-tools/hadoop-resourceestimator/src/test/resources/trace/invalidLog1.txt create mode 100644 hadoop-tools/hadoop-resourceestimator/src/test/resources/trace/invalidLog2.txt create mode 100644 hadoop-tools/hadoop-resourceestimator/src/test/resources/trace/invalidLog3.txt create mode 100644 hadoop-tools/hadoop-resourceestimator/src/test/resources/trace/invalidLog4.txt create mode 100644 hadoop-tools/hadoop-resourceestimator/src/test/resources/trace/invalidLog5.txt create mode 100644 hadoop-tools/hadoop-resourceestimator/src/test/resources/trace/invalidLog6.txt create mode 100644 hadoop-tools/hadoop-resourceestimator/src/test/resources/trace/invalidLog7.txt create mode 100644 hadoop-tools/hadoop-resourceestimator/src/test/resources/trace/invalidLog8.txt create mode 100644 hadoop-tools/hadoop-resourceestimator/src/test/resources/trace/invalidLog9.txt create mode 100644 hadoop-tools/hadoop-resourceestimator/src/test/resources/trace/nativeLog.txt create mode 100644 hadoop-tools/hadoop-resourceestimator/src/test/resources/trace/rmLog.txt diff --git a/LICENSE.txt b/LICENSE.txt index 3f50521b5e..67472b6b5c 100644 --- a/LICENSE.txt +++ b/LICENSE.txt @@ -699,6 +699,7 @@ hadoop-tools/hadoop-sls/src/main/html/js/thirdparty/jquery.js hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/jquery Apache HBase - Server which contains JQuery minified javascript library version 1.8.3 Microsoft JDBC Driver for SQLServer - version 6.2.1.jre7 +oj! Algorithms - version 43.0 -------------------------------------------------------------------------------- Copyright 2005, 2012, 2013 jQuery Foundation and other contributors, https://jquery.org/ diff --git a/hadoop-assemblies/src/main/resources/assemblies/hadoop-resourceestimator.xml b/hadoop-assemblies/src/main/resources/assemblies/hadoop-resourceestimator.xml new file mode 100644 index 0000000000..d1d941d60a --- /dev/null +++ b/hadoop-assemblies/src/main/resources/assemblies/hadoop-resourceestimator.xml @@ -0,0 +1,42 @@ + + + hadoop-resourceestimator + + dir + + false + + + + ${basedir}/src/main/bin + resourceestimator/bin + 0755 + + + ${basedir}/src/main/conf + resourceestimator/conf + + + ${basedir}/src/main/data + resourceestimator/data + + + diff --git a/hadoop-assemblies/src/main/resources/assemblies/hadoop-tools.xml b/hadoop-assemblies/src/main/resources/assemblies/hadoop-tools.xml index 0a4367de3a..7d34154838 100644 --- a/hadoop-assemblies/src/main/resources/assemblies/hadoop-tools.xml +++ b/hadoop-assemblies/src/main/resources/assemblies/hadoop-tools.xml @@ -174,6 +174,17 @@ ../hadoop-sls/target/hadoop-sls-${project.version}/sls /share/hadoop/${hadoop.component}/sls + + ../hadoop-resourceestimator/target + /share/hadoop/${hadoop.component}/sources + + *-sources.jar + + + + ../hadoop-resourceestimator/target/hadoop-resourceestimator-${project.version}/resourceestimator + /share/hadoop/${hadoop.component}/resourceestimator + ../hadoop-aws/src/main/bin /bin diff --git a/hadoop-project/pom.xml b/hadoop-project/pom.xml index 3baa8f3f77..d7156d745e 100644 --- a/hadoop-project/pom.xml +++ b/hadoop-project/pom.xml @@ -666,6 +666,11 @@ asm 5.0.4 + + org.ojalgo + ojalgo + 43.0 + com.sun.jersey jersey-core diff --git a/hadoop-project/src/site/site.xml b/hadoop-project/src/site/site.xml index ae92605f08..6fc3cf993f 100644 --- a/hadoop-project/src/site/site.xml +++ b/hadoop-project/src/site/site.xml @@ -178,6 +178,7 @@ + diff --git a/hadoop-tools/hadoop-resourceestimator/README.md b/hadoop-tools/hadoop-resourceestimator/README.md new file mode 100644 index 0000000000..1cb8cb54ea --- /dev/null +++ b/hadoop-tools/hadoop-resourceestimator/README.md @@ -0,0 +1,19 @@ +Resource Estimator Service +========================== + +Resource Estimator Service can parse the history logs of production jobs, extract their resource consumption skylines in the past runs and predict their resource requirements for the new run. + +## Current Status + + * Support [Hadoop YARN ResourceManager](http://hadoop.apache.org/docs/current/hadoop-yarn/hadoop-yarn-site/YARN.html) logs. + * In-memory store for parsed history resource skyline and estimation. + * A [Linear Programming](https://github.com/optimatika/ojAlgo) based estimator. + * Provides REST interface to parse logs, query history store and estimations. + +## Upcoming features + + * UI to query history and edit and save estimations. + * Persisent store implementation for store (either DB based or distributed key-value like HBase). + * Integrate directly with the [Hadoop YARN Reservation System](http://hadoop.apache.org/docs/current/hadoop-yarn/hadoop-yarn-site/ReservationSystem.html) to make a recurring reservation based on the estimated resources. + +Refer to the [design document](https://issues.apache.org/jira/secure/attachment/12886714/ResourceEstimator-design-v1.pdf) for more details. diff --git a/hadoop-tools/hadoop-resourceestimator/pom.xml b/hadoop-tools/hadoop-resourceestimator/pom.xml new file mode 100644 index 0000000000..d44c9bc720 --- /dev/null +++ b/hadoop-tools/hadoop-resourceestimator/pom.xml @@ -0,0 +1,174 @@ + + + + 4.0.0 + + org.apache.hadoop + hadoop-project + 3.1.0-SNAPSHOT + ../../hadoop-project + + hadoop-resourceestimator + Apache Resource Estimator Service + jar + + + com.google.code.gson + gson + + + org.ojalgo + ojalgo + + + org.apache.hadoop + hadoop-common + + + org.apache.hadoop + hadoop-yarn-common + + + org.apache.hadoop + hadoop-yarn-api + + + org.apache.hadoop + hadoop-yarn-server-resourcemanager + + + org.slf4j + slf4j-api + + + javax.inject + javax.inject + 1 + + + com.sun.jersey.jersey-test-framework + jersey-test-framework-grizzly2 + test + + + javax.servlet + javax.servlet-api + + + com.sun.jersey + jersey-server + + + com.sun.jersey + jersey-json + + + junit + junit + test + + + + + + org.apache.rat + apache-rat-plugin + + + README.md + **/*.txt + + + + + org.apache.maven.plugins + maven-checkstyle-plugin + + src/config/checkstyle.xml + + + + + + + + dist + + false + + + + + org.apache.maven.plugins + maven-assembly-plugin + + false + false + + ${project.artifactId}-${project.version} + + + hadoop-resourceestimator + + + + + + org.apache.hadoop + hadoop-assemblies + ${project.version} + + + + + dist + prepare-package + + single + + + + + + org.apache.maven.plugins + maven-dependency-plugin + + + deplist + compile + + list + + + + ${project.basedir}/target/hadoop-tools-deps/${project.artifactId}.tools-builtin.txt + + + + + + + + + + diff --git a/hadoop-tools/hadoop-resourceestimator/src/config/checkstyle.xml b/hadoop-tools/hadoop-resourceestimator/src/config/checkstyle.xml new file mode 100644 index 0000000000..7f22659a89 --- /dev/null +++ b/hadoop-tools/hadoop-resourceestimator/src/config/checkstyle.xml @@ -0,0 +1,50 @@ + + + + + + + + + + + + + + + diff --git a/hadoop-tools/hadoop-resourceestimator/src/main/bin/estimator.cmd b/hadoop-tools/hadoop-resourceestimator/src/main/bin/estimator.cmd new file mode 100644 index 0000000000..317025cda9 --- /dev/null +++ b/hadoop-tools/hadoop-resourceestimator/src/main/bin/estimator.cmd @@ -0,0 +1,52 @@ +@echo off +@rem Licensed to the Apache Software Foundation (ASF) under one or more +@rem contributor license agreements. See the NOTICE file distributed with +@rem this work for additional information regarding copyright ownership. +@rem The ASF licenses this file to You under the Apache License, Version 2.0 +@rem (the "License"); you may not use this file except in compliance with +@rem the License. You may obtain a copy of the License at +@rem +@rem http://www.apache.org/licenses/LICENSE-2.0 +@rem +@rem Unless required by applicable law or agreed to in writing, software +@rem distributed under the License is distributed on an "AS IS" BASIS, +@rem WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +@rem See the License for the specific language governing permissions and +@rem limitations under the License. +@rem + +setlocal enabledelayedexpansion + +if not defined HADOOP_BIN_PATH ( + set HADOOP_BIN_PATH=%~dp0 +) + +if "%HADOOP_BIN_PATH:~-1%" == "\" ( + set HADOOP_BIN_PATH=%HADOOP_BIN_PATH:~0,-1% +) + +set DEFAULT_LIBEXEC_DIR=%HADOOP_BIN_PATH%\..\libexec +if not defined HADOOP_LIBEXEC_DIR ( + set HADOOP_LIBEXEC_DIR=%DEFAULT_LIBEXEC_DIR% +) + +:main + @rem CLASSPATH initially contains HADOOP_CONF_DIR + if not defined HADOOP_CONF_DIR ( + echo No HADOOP_CONF_DIR set. + echo Please specify it. + goto :eof + ) + + set CLASSPATH=%HADOOP_CONF_DIR%;%CLASSPATH% +goto :eof + +:classpath + set CLASS=org.apache.hadoop.util.Classpath + goto :eof + +:resourceestimator + set CLASS=org.apache.hadoop.resourceestimator.service.ResourceEstimatorServer + goto :eof + +endlocal diff --git a/hadoop-tools/hadoop-resourceestimator/src/main/bin/estimator.sh b/hadoop-tools/hadoop-resourceestimator/src/main/bin/estimator.sh new file mode 100644 index 0000000000..3e1ec278a9 --- /dev/null +++ b/hadoop-tools/hadoop-resourceestimator/src/main/bin/estimator.sh @@ -0,0 +1,71 @@ +#!/usr/bin/env bash +# +# 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. See accompanying LICENSE file. +# + +## @audience public +## @stability stable +function hadoop_usage() +{ + echo "Usage: estimator.sh" + #hadoop-daemon.sh. need both start and stop, status (query the status). run as background process. +} + +## @audience public +## @stability stable +function calculate_classpath +{ + hadoop_add_client_opts + hadoop_add_to_classpath_tools hadoop-resourceestimator +} + +## @audience public +## @stability stable +function resourceestimatorcmd_case +{ + # shellcheck disable=SC2034 + HADOOP_SUBCMD_SUPPORTDAEMONIZATION="true" + # shellcheck disable=SC2034 + HADOOP_CLASSNAME='org.apache.hadoop.resourceestimator.service.ResourceEstimatorServer' +} + +# let's locate libexec... +if [[ -n "${HADOOP_HOME}" ]]; then + HADOOP_DEFAULT_LIBEXEC_DIR="${HADOOP_HOME}/libexec" +else + this="${BASH_SOURCE-$0}" + bin=$(cd -P -- "$(dirname -- "${this}")" >/dev/null && pwd -P) + HADOOP_DEFAULT_LIBEXEC_DIR="${bin}/../../../../../libexec" +fi + +HADOOP_LIBEXEC_DIR="${HADOOP_LIBEXEC_DIR:-$HADOOP_DEFAULT_LIBEXEC_DIR}" +# shellcheck disable=SC2034 +HADOOP_NEW_CONFIG=true +if [[ -f "${HADOOP_LIBEXEC_DIR}/hadoop-config.sh" ]]; then + # shellcheck source=./hadoop-common-project/hadoop-common/src/main/bin/hadoop-config.sh + . "${HADOOP_LIBEXEC_DIR}/hadoop-config.sh" +else + echo "ERROR: Cannot execute ${HADOOP_LIBEXEC_DIR}/hadoop-config.sh." 2>&1 + exit 1 +fi + +# get arguments +HADOOP_SUBCMD=$1 +shift + +HADOOP_SUBCMD_ARGS=("$@") + +resourceestimatorcmd_case "${HADOOP_SUBCMD}" "${HADOOP_SUBCMD_ARGS[@]}" + +calculate_classpath +hadoop_generic_java_subcmd_handler diff --git a/hadoop-tools/hadoop-resourceestimator/src/main/bin/start-estimator.cmd b/hadoop-tools/hadoop-resourceestimator/src/main/bin/start-estimator.cmd new file mode 100644 index 0000000000..463e91c70f --- /dev/null +++ b/hadoop-tools/hadoop-resourceestimator/src/main/bin/start-estimator.cmd @@ -0,0 +1,37 @@ +@echo off +@rem Licensed to the Apache Software Foundation (ASF) under one or more +@rem contributor license agreements. See the NOTICE file distributed with +@rem this work for additional information regarding copyright ownership. +@rem The ASF licenses this file to You under the Apache License, Version 2.0 +@rem (the "License"); you may not use this file except in compliance with +@rem the License. You may obtain a copy of the License at +@rem +@rem http://www.apache.org/licenses/LICENSE-2.0 +@rem +@rem Unless required by applicable law or agreed to in writing, software +@rem distributed under the License is distributed on an "AS IS" BASIS, +@rem WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +@rem See the License for the specific language governing permissions and +@rem limitations under the License. +@rem +setlocal enabledelayedexpansion + +echo starting resource estimator daemons + +if not defined HADOOP_BIN_PATH ( + set HADOOP_BIN_PATH=%~dp0 +) + +if "%HADOOP_BIN_PATH:~-1%" == "\" ( + set HADOOP_BIN_PATH=%HADOOP_BIN_PATH:~0,-1% +) + +set DEFAULT_LIBEXEC_DIR=%HADOOP_BIN_PATH%\..\libexec +if not defined HADOOP_LIBEXEC_DIR ( + set HADOOP_LIBEXEC_DIR=%DEFAULT_LIBEXEC_DIR% +) + +@rem start resource estimator +start "Resource Estimator Service" estimator resourceestimator + +endlocal diff --git a/hadoop-tools/hadoop-resourceestimator/src/main/bin/start-estimator.sh b/hadoop-tools/hadoop-resourceestimator/src/main/bin/start-estimator.sh new file mode 100644 index 0000000000..bf4328adae --- /dev/null +++ b/hadoop-tools/hadoop-resourceestimator/src/main/bin/start-estimator.sh @@ -0,0 +1,42 @@ +#!/usr/bin/env bash +# +# 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. See accompanying LICENSE file. +# + +# let's locate libexec... +if [[ -n "${HADOOP_HOME}" ]]; then + HADOOP_DEFAULT_LIBEXEC_DIR="${HADOOP_HOME}/libexec" +else + this="${BASH_SOURCE-$0}" + bin=$(cd -P -- "$(dirname -- "${this}")" >/dev/null && pwd -P) + HADOOP_DEFAULT_LIBEXEC_DIR="${bin}/../../../../../libexec" +fi + +HADOOP_LIBEXEC_DIR="${HADOOP_LIBEXEC_DIR:-$HADOOP_DEFAULT_LIBEXEC_DIR}" +# shellcheck disable=SC2034 +HADOOP_NEW_CONFIG=true +if [[ -f "${HADOOP_LIBEXEC_DIR}/hadoop-config.sh" ]]; then + # shellcheck source=./hadoop-common-project/hadoop-common/src/main/bin/hadoop-config.sh + . "${HADOOP_LIBEXEC_DIR}/hadoop-config.sh" +else + echo "ERROR: Cannot execute ${HADOOP_LIBEXEC_DIR}/hadoop-config.sh." 2>&1 + exit 1 +fi + +# start resource estimator +echo "Starting resource estimator" +hadoop_uservar_su estimator resourceestimator "bin/estimator.sh" \ + --config "${HADOOP_CONF_DIR}" \ + --daemon start \ + resourceestimator +(( HADOOP_JUMBO_RETCOUNTER=HADOOP_JUMBO_RETCOUNTER + $? )) diff --git a/hadoop-tools/hadoop-resourceestimator/src/main/bin/stop-estimator.cmd b/hadoop-tools/hadoop-resourceestimator/src/main/bin/stop-estimator.cmd new file mode 100644 index 0000000000..e08c7e0238 --- /dev/null +++ b/hadoop-tools/hadoop-resourceestimator/src/main/bin/stop-estimator.cmd @@ -0,0 +1,37 @@ +@echo off +@rem Licensed to the Apache Software Foundation (ASF) under one or more +@rem contributor license agreements. See the NOTICE file distributed with +@rem this work for additional information regarding copyright ownership. +@rem The ASF licenses this file to You under the Apache License, Version 2.0 +@rem (the "License"); you may not use this file except in compliance with +@rem the License. You may obtain a copy of the License at +@rem +@rem http://www.apache.org/licenses/LICENSE-2.0 +@rem +@rem Unless required by applicable law or agreed to in writing, software +@rem distributed under the License is distributed on an "AS IS" BASIS, +@rem WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +@rem See the License for the specific language governing permissions and +@rem limitations under the License. +@rem +setlocal enabledelayedexpansion + +echo stopping resource estimator daemons + +if not defined HADOOP_BIN_PATH ( + set HADOOP_BIN_PATH=%~dp0 +) + +if "%HADOOP_BIN_PATH:~-1%" == "\" ( + set HADOOP_BIN_PATH=%HADOOP_BIN_PATH:~0,-1% +) + +set DEFAULT_LIBEXEC_DIR=%HADOOP_BIN_PATH%\..\libexec +if not defined HADOOP_LIBEXEC_DIR ( + set HADOOP_LIBEXEC_DIR=%DEFAULT_LIBEXEC_DIR% +) + +@rem start resource estimator +stop "Resource Estimator Service" estimator resourceestimator + +endlocal diff --git a/hadoop-tools/hadoop-resourceestimator/src/main/bin/stop-estimator.sh b/hadoop-tools/hadoop-resourceestimator/src/main/bin/stop-estimator.sh new file mode 100644 index 0000000000..e78c798333 --- /dev/null +++ b/hadoop-tools/hadoop-resourceestimator/src/main/bin/stop-estimator.sh @@ -0,0 +1,42 @@ +#!/usr/bin/env bash +# +# 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. See accompanying LICENSE file. +# + +# let's locate libexec... +if [[ -n "${HADOOP_HOME}" ]]; then + HADOOP_DEFAULT_LIBEXEC_DIR="${HADOOP_HOME}/libexec" +else + this="${BASH_SOURCE-$0}" + bin=$(cd -P -- "$(dirname -- "${this}")" >/dev/null && pwd -P) + HADOOP_DEFAULT_LIBEXEC_DIR="${bin}/../../../../../libexec" +fi + +HADOOP_LIBEXEC_DIR="${HADOOP_LIBEXEC_DIR:-$HADOOP_DEFAULT_LIBEXEC_DIR}" +# shellcheck disable=SC2034 +HADOOP_NEW_CONFIG=true +if [[ -f "${HADOOP_LIBEXEC_DIR}/hadoop-config.sh" ]]; then + # shellcheck source=./hadoop-common-project/hadoop-common/src/main/bin/hadoop-config.sh + . "${HADOOP_LIBEXEC_DIR}/hadoop-config.sh" +else + echo "ERROR: Cannot execute ${HADOOP_LIBEXEC_DIR}/hadoop-config.sh." 2>&1 + exit 1 +fi + +# stop resource estimator +echo "Stopping resource estimator" +hadoop_uservar_su estimator resourceestimator "bin/estimator.sh" \ + --config "${HADOOP_CONF_DIR}" \ + --daemon stop \ + resourceestimator +(( HADOOP_JUMBO_RETCOUNTER=HADOOP_JUMBO_RETCOUNTER + $? )) diff --git a/hadoop-tools/hadoop-resourceestimator/src/main/conf/resourceestimator-config.xml b/hadoop-tools/hadoop-resourceestimator/src/main/conf/resourceestimator-config.xml new file mode 100644 index 0000000000..f78085a0ab --- /dev/null +++ b/hadoop-tools/hadoop-resourceestimator/src/main/conf/resourceestimator-config.xml @@ -0,0 +1,85 @@ + + + + + + resourceestimator.solver.lp.alpha + 0.1 + + The resource estimator has an integrated Linear Programming solver to make the prediction, and this parameter tunes the tradeoff between resource over-allocation and under-allocation in the Linear Programming model. This parameter varies from 0 to 1, and a larger alpha value means the model minimizes over-allocation better. Default value is 0.1. + + + + resourceestimator.solver.lp.beta + 0.1 + + This parameter controls the generalization of the Linear Programming model. This parameter varies from 0 to 1. Deafult value is 0.1. + + + + resourceestimator.solver.lp.minJobRuns + 2 + + The minimum number of job runs required in order to make the prediction. Default value is 2. + + + + resourceestimator.timeInterval + 5 + + The time length which is used to discretize job execution into intervals. Note that the estimator makes resource allocation prediction for each interval. A smaller time interval has more fine-grained granularity for prediction, but it also takes longer time and more space for prediction. Default value is 5 (seconds). + + + + resourceestimator.translator.line-parser + org.apache.hadoop.resourceestimator.translator.impl.NativeSingleLineParser + + The class name of the translator single-line parser, which parses a single line in the log. Default value is org.apache.hadoop.resourceestimator.translator.impl.NativeSingleLineParser, which can parse one line in the sample log. Note that if users want to parse Hadoop Resource Manager logs, they need to set the value to be org.apache.hadoop.resourceestimator.translator.impl.RmSingleLineParser. If they want to implement single-line parser to parse their customized log file, they need to change this value accordingly. + + + + + diff --git a/hadoop-tools/hadoop-resourceestimator/src/main/data/resourceEstimatorService.txt b/hadoop-tools/hadoop-resourceestimator/src/main/data/resourceEstimatorService.txt new file mode 100644 index 0000000000..15fb84e98c --- /dev/null +++ b/hadoop-tools/hadoop-resourceestimator/src/main/data/resourceEstimatorService.txt @@ -0,0 +1,2 @@ +tpch_q12 50 1462148517000 1462148527345 1462148952345 tpch_q12_0 8192:0| 1:0|1:5|1074:10|2538:15|2468:20| +tpch_q12 50 1462148949000 1462148960455 1462149205455 tpch_q12_1 8192:0| 1:0|1:5|794:10|2517:15|2484:20| \ No newline at end of file diff --git a/hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/common/api/RecurrenceId.java b/hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/common/api/RecurrenceId.java new file mode 100644 index 0000000000..01ef97725d --- /dev/null +++ b/hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/common/api/RecurrenceId.java @@ -0,0 +1,95 @@ +/* + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package org.apache.hadoop.resourceestimator.common.api; + +/** + * RecurrenceId is the id for the recurring pipeline jobs. + *

We assume that the pipeline job can be uniquely identified with + * {pipelineId, runId}. + */ +public class RecurrenceId { + /** + * pipelineId is the unique id for the pipeline jobs. + */ + private String pipelineId; + /** + * runId is the unique instance id for the pipeline job in one run, and it + * will change across runs. + */ + private String runId; + // TODO: we may addHistory more ids of the pipeline jobs to identify them. + + /** + * Constructor. + * + * @param pipelineIdConfig the unique id for the pipeline jobs. + * @param runIdConfig the unique instance id for the pipeline job in one run. + */ + public RecurrenceId(final String pipelineIdConfig, final String runIdConfig) { + this.pipelineId = pipelineIdConfig; + this.runId = runIdConfig; + } + + /** + * Return the pipelineId for the pipeline jobs. + * + * @return the pipelineId. + */ + public final String getPipelineId() { + return pipelineId; + } + + public void setPipelineId(String pipelineId) { + this.pipelineId = pipelineId; + } + + /** + * Return the runId for the pipeline job in one run. + * + * @return the runId. + */ + public final String getRunId() { + return runId; + } + + public void setRunId(String runId) { + this.runId = runId; + } + + @Override public final String toString() { + return String.format("{pipelineId: %s, runId: %s}", pipelineId, runId); + } + + @Override public final int hashCode() { + return getPipelineId().hashCode() ^ getRunId().hashCode(); + } + + @Override public final boolean equals(final Object obj) { + if (this == obj) { + return true; + } + if ((obj == null) || (getClass() != obj.getClass())) { + return false; + } + final RecurrenceId other = (RecurrenceId) obj; + return pipelineId.equals(other.pipelineId) && runId.equals(other.runId); + } +} diff --git a/hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/common/api/ResourceSkyline.java b/hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/common/api/ResourceSkyline.java new file mode 100644 index 0000000000..a4bc69f358 --- /dev/null +++ b/hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/common/api/ResourceSkyline.java @@ -0,0 +1,211 @@ +/* + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package org.apache.hadoop.resourceestimator.common.api; + +import org.apache.hadoop.yarn.api.records.Resource; +import org.apache.hadoop.yarn.server.resourcemanager.reservation.RLESparseResourceAllocation; + +/** + * ResourceSkyline records the job identification information as well as job's + * requested {@code + * container}s information during its lifespan. + */ +public class ResourceSkyline { + /** + * The auto-generated {@code ApplicationId} in job's one run. + *

+ *

+ * For a pipeline job, we assume the {@code jobId} changes each time we run + * the pipeline job. + */ + private String jobId; + /** + * The input data size of the job. + */ + private double jobInputDataSize; + /** + * Job submission time. Different logs could have different time format, so we + * store the original string directly extracted from logs. + */ + private long jobSubmissionTime; + /** + * Job finish time. Different logs could have different time format, so we + * store the original string directly extracted from logs. + */ + private long jobFinishTime; + /** + * The resource spec of containers allocated to the job. + */ + private Resource containerSpec; + /** + * The list of {@link Resource} allocated to the job in its lifespan. + */ + private RLESparseResourceAllocation skylineList; + // TODO + // We plan to record pipeline job's actual resource consumptions in one run + // here. + // TODO + // We might need to addHistory more features to the ResourceSkyline, such as + // users, VC, etc. + + /** + * Constructor. + * + * @param jobIdConfig the id of the job. + * @param jobInputDataSizeConfig the input data size of the job. + * @param jobSubmissionTimeConfig the submission time of the job. + * @param jobFinishTimeConfig the finish time of the job. + * @param containerSpecConfig the resource spec of containers allocated + * to the job. + * @param skylineListConfig the list of {@link Resource} allocated in + * one run. + */ + public ResourceSkyline(final String jobIdConfig, + final double jobInputDataSizeConfig, final long jobSubmissionTimeConfig, + final long jobFinishTimeConfig, final Resource containerSpecConfig, + final RLESparseResourceAllocation skylineListConfig) { + this.jobId = jobIdConfig; + this.jobInputDataSize = jobInputDataSizeConfig; + this.jobSubmissionTime = jobSubmissionTimeConfig; + this.jobFinishTime = jobFinishTimeConfig; + this.containerSpec = containerSpecConfig; + this.skylineList = skylineListConfig; + } + + /** + * Empty constructor. + */ + public ResourceSkyline() { + } + + /** + * Get the id of the job. + * + * @return the id of this job. + */ + public final String getJobId() { + return jobId; + } + + /** + * Set jobId. + * + * @param jobIdConfig jobId. + */ + public final void setJobId(final String jobIdConfig) { + this.jobId = jobIdConfig; + } + + /** + * Get the job's input data size. + * + * @return job's input data size. + */ + public final double getJobInputDataSize() { + return jobInputDataSize; + } + + /** + * Set jobInputDataSize. + * + * @param jobInputDataSizeConfig jobInputDataSize. + */ + public final void setJobInputDataSize(final double jobInputDataSizeConfig) { + this.jobInputDataSize = jobInputDataSizeConfig; + } + + /** + * Get the job's submission time. + * + * @return job's submission time. + */ + public final long getJobSubmissionTime() { + return jobSubmissionTime; + } + + /** + * Set jobSubmissionTime. + * + * @param jobSubmissionTimeConfig jobSubmissionTime. + */ + public final void setJobSubmissionTime(final long jobSubmissionTimeConfig) { + this.jobSubmissionTime = jobSubmissionTimeConfig; + } + + /** + * Get the job's finish time. + * + * @return job's finish time. + */ + public final long getJobFinishTime() { + return jobFinishTime; + } + + /** + * Set jobFinishTime. + * + * @param jobFinishTimeConfig jobFinishTime. + */ + public final void setJobFinishTime(final long jobFinishTimeConfig) { + this.jobFinishTime = jobFinishTimeConfig; + } + + /** + * Get the resource spec of the job's allocated {@code container}s. + *

Key assumption: during job's lifespan, its allocated {@code container}s + * have the same {@link Resource} spec. + * + * @return the {@link Resource} spec of the job's allocated + * {@code container}s. + */ + public final Resource getContainerSpec() { + return containerSpec; + } + + /** + * Set containerSpec. + * + * @param containerSpecConfig containerSpec. + */ + public final void setContainerSpec(final Resource containerSpecConfig) { + this.containerSpec = containerSpecConfig; + } + + /** + * Get the list of {@link Resource}s allocated to the job. + * + * @return the {@link RLESparseResourceAllocation} which contains the list of + * {@link Resource}s allocated to the job. + */ + public final RLESparseResourceAllocation getSkylineList() { + return skylineList; + } + + /** + * Set skylineList. + * + * @param skylineListConfig skylineList. + */ + public final void setSkylineList( + final RLESparseResourceAllocation skylineListConfig) { + this.skylineList = skylineListConfig; + } +} diff --git a/hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/common/api/package-info.java b/hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/common/api/package-info.java new file mode 100644 index 0000000000..a7c752752c --- /dev/null +++ b/hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/common/api/package-info.java @@ -0,0 +1,23 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/** + * The common APIs for the resource estimator. + */ + +package org.apache.hadoop.resourceestimator.common.api; \ No newline at end of file diff --git a/hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/common/config/ResourceEstimatorConfiguration.java b/hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/common/config/ResourceEstimatorConfiguration.java new file mode 100644 index 0000000000..bb041b6745 --- /dev/null +++ b/hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/common/config/ResourceEstimatorConfiguration.java @@ -0,0 +1,125 @@ +/* + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package org.apache.hadoop.resourceestimator.common.config; + +import org.apache.hadoop.resourceestimator.skylinestore.impl.InMemoryStore; +import org.apache.hadoop.resourceestimator.solver.impl.LpSolver; +import org.apache.hadoop.resourceestimator.translator.impl.BaseLogParser; +import org.apache.hadoop.resourceestimator.translator.impl.NativeSingleLineParser; + +/** + * Defines configuration keys for ResourceEstimatorServer. + */ +public final class ResourceEstimatorConfiguration { + + /** + * The location of the configuration file for ResourceEstimatorService. + */ + public static final String CONFIG_FILE = "resourceestimator-config.xml"; + + /** + * The URI for ResourceEstimatorService. + */ + public static final String SERVICE_URI = "http://0.0.0.0/"; + + /** + * The port which ResourceEstimatorService listens to. + */ + public static final String SERVICE_PORT = "resourceestimator.service-port"; + + /** + * Default port number of ResourceEstimatorService. + */ + public static final int DEFAULT_SERVICE_PORT = 9998; + + /** + * The class name of the skylinestore provider. + */ + public static final String SKYLINESTORE_PROVIDER = + "resourceestimator.skylinestore.provider"; + + /** + * Default value for skylinestore provider, which is an in-memory implementation of skylinestore. + */ + public static final String DEFAULT_SKYLINESTORE_PROVIDER = + InMemoryStore.class.getName(); + + /** + * The class name of the translator provider. + */ + public static final String TRANSLATOR_PROVIDER = + "resourceestimator.translator.provider"; + + /** + * Default value for translator provider, which extracts resourceskylines from log streams. + */ + public static final String DEFAULT_TRANSLATOR_PROVIDER = + BaseLogParser.class.getName(); + + /** + * The class name of the translator single-line parser, which parses a single line in the log. + */ + public static final String TRANSLATOR_LINE_PARSER = + "resourceestimator.translator.line-parser"; + + /** + * Default value for translator single-line parser, which can parse one line in the sample log. + */ + public static final String DEFAULT_TRANSLATOR_LINE_PARSER = + NativeSingleLineParser.class.getName(); + + /** + * The class name of the solver provider. + */ + public static final String SOLVER_PROVIDER = + "resourceestimator.solver.provider"; + + /** + * Default value for solver provider, which incorporates a Linear Programming model to make the prediction. + */ + public static final String DEFAULT_SOLVER_PROVIDER = LpSolver.class.getName(); + + /** + * The time length which is used to discretize job execution into intervals. + */ + public static final String TIME_INTERVAL_KEY = + "resourceestimator.timeInterval"; + + /** + * The parameter which tunes the tradeoff between resource over-allocation and under-allocation in the Linear Programming model. + */ + public static final String SOLVER_ALPHA_KEY = + "resourceestimator.solver.lp.alpha"; + + /** + * This parameter which controls the generalization of the Linear Programming model. + */ + public static final String SOLVER_BETA_KEY = + "resourceestimator.solver.lp.beta"; + + /** + * The minimum number of job runs required in order to make the prediction. + */ + public static final String SOLVER_MIN_JOB_RUN_KEY = + "resourceestimator.solver.lp.minJobRuns"; + + private ResourceEstimatorConfiguration() {} +} diff --git a/hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/common/config/ResourceEstimatorUtil.java b/hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/common/config/ResourceEstimatorUtil.java new file mode 100644 index 0000000000..c00dd00689 --- /dev/null +++ b/hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/common/config/ResourceEstimatorUtil.java @@ -0,0 +1,81 @@ +/* + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package org.apache.hadoop.resourceestimator.common.config; + +import java.lang.reflect.Constructor; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.resourceestimator.common.exception.ResourceEstimatorException; + +/** + * General resourceestimator utils. + */ +public final class ResourceEstimatorUtil { + + private static final Class[] EMPTY_ARRAY = new Class[0]; + + private ResourceEstimatorUtil() {} + + /** + * Helper method to create instances of Object using the class name specified + * in the configuration object. + * + * @param conf the yarn configuration + * @param configuredClassName the configuration provider key + * @param defaultValue the default implementation class + * @param type the required interface/base class + * @param The type of the instance to create + * @return the instances created + * @throws ResourceEstimatorException if the provider initialization fails. + */ + @SuppressWarnings("unchecked") public static T createProviderInstance( + Configuration conf, String configuredClassName, String defaultValue, + Class type) throws ResourceEstimatorException { + String className = conf.get(configuredClassName); + if (className == null) { + className = defaultValue; + } + try { + Class concreteClass = Class.forName(className); + if (type.isAssignableFrom(concreteClass)) { + Constructor meth = + (Constructor) concreteClass.getDeclaredConstructor(EMPTY_ARRAY); + meth.setAccessible(true); + return meth.newInstance(); + } else { + StringBuilder errMsg = new StringBuilder(); + errMsg.append("Class: ").append(className).append(" not instance of ") + .append(type.getCanonicalName()); + throw new ResourceEstimatorException(errMsg.toString()); + } + } catch (ClassNotFoundException e) { + StringBuilder errMsg = new StringBuilder(); + errMsg.append("Could not instantiate : ").append(className) + .append(" due to exception: ").append(e.getCause()); + throw new ResourceEstimatorException(errMsg.toString()); + } catch (ReflectiveOperationException e) { + StringBuilder errMsg = new StringBuilder(); + errMsg.append("Could not instantiate : ").append(className) + .append(" due to exception: ").append(e.getCause()); + throw new ResourceEstimatorException(errMsg.toString()); + } + } +} diff --git a/hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/common/config/package-info.java b/hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/common/config/package-info.java new file mode 100644 index 0000000000..ecf140a483 --- /dev/null +++ b/hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/common/config/package-info.java @@ -0,0 +1,23 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/** + * The common configuration for the resource estimator. + */ + +package org.apache.hadoop.resourceestimator.common.config; \ No newline at end of file diff --git a/hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/common/exception/ResourceEstimatorException.java b/hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/common/exception/ResourceEstimatorException.java new file mode 100644 index 0000000000..ba5666d8b8 --- /dev/null +++ b/hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/common/exception/ResourceEstimatorException.java @@ -0,0 +1,35 @@ +/* + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package org.apache.hadoop.resourceestimator.common.exception; + +/** + * Exception thrown by ResourceEstimatorServer utility classes. + */ +public class ResourceEstimatorException extends Exception { + + public ResourceEstimatorException(String message) { + super(message); + } + + public ResourceEstimatorException(String message, Exception ex) { + super(message, ex); + } +} diff --git a/hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/common/exception/package-info.java b/hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/common/exception/package-info.java new file mode 100644 index 0000000000..6ebecb6d0d --- /dev/null +++ b/hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/common/exception/package-info.java @@ -0,0 +1,23 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/** + * The common exception for the resource estimator. + */ + +package org.apache.hadoop.resourceestimator.common.exception; \ No newline at end of file diff --git a/hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/common/serialization/RLESparseResourceAllocationSerDe.java b/hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/common/serialization/RLESparseResourceAllocationSerDe.java new file mode 100644 index 0000000000..379dc3a71d --- /dev/null +++ b/hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/common/serialization/RLESparseResourceAllocationSerDe.java @@ -0,0 +1,77 @@ +/* + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package org.apache.hadoop.resourceestimator.common.serialization; + +import java.lang.reflect.Type; +import java.util.NavigableMap; + +import org.apache.hadoop.yarn.api.records.Resource; +import org.apache.hadoop.yarn.server.resourcemanager.reservation.RLESparseResourceAllocation; +import org.apache.hadoop.yarn.util.resource.DefaultResourceCalculator; +import org.apache.hadoop.yarn.util.resource.ResourceCalculator; + +import com.google.gson.Gson; +import com.google.gson.GsonBuilder; +import com.google.gson.JsonDeserializationContext; +import com.google.gson.JsonDeserializer; +import com.google.gson.JsonElement; +import com.google.gson.JsonObject; +import com.google.gson.JsonParseException; +import com.google.gson.JsonSerializationContext; +import com.google.gson.JsonSerializer; +import com.google.gson.reflect.TypeToken; + +/** + * Serialize/deserialize RLESparseResourceAllocation object to/from JSON. + */ +public class RLESparseResourceAllocationSerDe + implements JsonSerializer, + JsonDeserializer { + private static final String KEY = "resourceAllocation"; + private final Gson gson = + new GsonBuilder().registerTypeAdapter(Resource.class, new ResourceSerDe()) + .create(); + private final Type type = new TypeToken>() { + }.getType(); + private final ResourceCalculator resourceCalculator = + new DefaultResourceCalculator(); + + @Override public final JsonElement serialize( + final RLESparseResourceAllocation resourceAllocation, + final Type typeOfSrc, final JsonSerializationContext context) { + NavigableMap myMap = resourceAllocation.getCumulative(); + JsonObject jo = new JsonObject(); + JsonElement element = gson.toJsonTree(myMap, type); + jo.add(KEY, element); + + return jo; + } + + @Override public final RLESparseResourceAllocation deserialize( + final JsonElement json, final Type typeOfT, + final JsonDeserializationContext context) throws JsonParseException { + NavigableMap resAllocation = + gson.fromJson(json.getAsJsonObject().get(KEY), type); + RLESparseResourceAllocation rleSparseResourceAllocation = + new RLESparseResourceAllocation(resAllocation, resourceCalculator); + return rleSparseResourceAllocation; + } +} diff --git a/hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/common/serialization/ResourceSerDe.java b/hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/common/serialization/ResourceSerDe.java new file mode 100644 index 0000000000..22b079c971 --- /dev/null +++ b/hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/common/serialization/ResourceSerDe.java @@ -0,0 +1,61 @@ +/* + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package org.apache.hadoop.resourceestimator.common.serialization; + +import java.lang.reflect.Type; + +import org.apache.hadoop.yarn.api.records.Resource; + +import com.google.gson.JsonDeserializationContext; +import com.google.gson.JsonDeserializer; +import com.google.gson.JsonElement; +import com.google.gson.JsonObject; +import com.google.gson.JsonParseException; +import com.google.gson.JsonSerializationContext; +import com.google.gson.JsonSerializer; + +/** + * Serialize/deserialize Resource object to/from JSON. + */ +public class ResourceSerDe + implements JsonSerializer, JsonDeserializer { + private static final String KEY1 = "memory"; + private static final String KEY2 = "vcores"; + + @Override public final JsonElement serialize(final Resource resource, + final Type type, final JsonSerializationContext context) { + JsonObject jo = new JsonObject(); + jo.addProperty(KEY1, resource.getMemorySize()); + jo.addProperty(KEY2, resource.getVirtualCores()); + return jo; + } + + @Override public final Resource deserialize(final JsonElement json, + final Type type, final JsonDeserializationContext context) + throws JsonParseException { + JsonObject jo = json.getAsJsonObject(); + long mem = jo.getAsJsonPrimitive(KEY1).getAsLong(); + int vcore = jo.getAsJsonPrimitive(KEY2).getAsInt(); + Resource resource = Resource.newInstance(mem, vcore); + + return resource; + } +} diff --git a/hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/common/serialization/package-info.java b/hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/common/serialization/package-info.java new file mode 100644 index 0000000000..06957b39a7 --- /dev/null +++ b/hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/common/serialization/package-info.java @@ -0,0 +1,24 @@ +/* + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +/** + * SkylineStore serialization module. + */ +package org.apache.hadoop.resourceestimator.common.serialization; diff --git a/hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/service/ResourceEstimatorServer.java b/hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/service/ResourceEstimatorServer.java new file mode 100644 index 0000000000..27a5864afd --- /dev/null +++ b/hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/service/ResourceEstimatorServer.java @@ -0,0 +1,146 @@ +/* + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package org.apache.hadoop.resourceestimator.service; + +import java.io.IOException; +import java.net.URI; + +import javax.ws.rs.core.UriBuilder; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.http.HttpServer2; +import org.apache.hadoop.resourceestimator.common.config.ResourceEstimatorConfiguration; +import org.apache.hadoop.service.CompositeService; +import org.apache.hadoop.util.ShutdownHookManager; +import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.apache.hadoop.yarn.webapp.GenericExceptionHandler; +import org.apache.hadoop.yarn.webapp.YarnJacksonJaxbJsonProvider; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * A simple embedded Hadoop HTTP server. + */ +public final class ResourceEstimatorServer extends CompositeService { + private static final Logger LOGGER = + LoggerFactory.getLogger(ResourceEstimatorServer.class.getName()); + private HttpServer2 webServer; + private static URI baseURI; + + public ResourceEstimatorServer() { + super(ResourceEstimatorServer.class.getName()); + } + + private static URI getBaseURI(Configuration config) { + baseURI = UriBuilder.fromUri(ResourceEstimatorConfiguration.SERVICE_URI) + .port(getPort(config)).build(); + return baseURI; + } + + private static int getPort(Configuration config) { + return config.getInt(ResourceEstimatorConfiguration.SERVICE_PORT, + ResourceEstimatorConfiguration.DEFAULT_SERVICE_PORT); + } + + @Override protected void serviceInit(Configuration conf) throws Exception { + super.serviceInit(conf); + } + + @Override protected void serviceStart() throws Exception { + super.serviceStart(); + startResourceEstimatorApp(); + } + + private void join() { + // keep the main thread that started the server up until it receives a stop + // signal + if (webServer != null) { + try { + webServer.join(); + } catch (InterruptedException ignore) { + } + } + } + + @Override protected void serviceStop() throws Exception { + if (webServer != null) { + webServer.stop(); + } + super.serviceStop(); + } + + private void startResourceEstimatorApp() throws IOException { + Configuration config = new YarnConfiguration(); + config.addResource(ResourceEstimatorConfiguration.CONFIG_FILE); + HttpServer2.Builder builder = + new HttpServer2.Builder().setName("ResourceEstimatorServer") + .setConf(config) + //.setFindPort(true) + .addEndpoint(getBaseURI(config)); + webServer = builder.build(); + webServer.addJerseyResourcePackage( + ResourceEstimatorService.class.getPackage().getName() + ";" + + GenericExceptionHandler.class.getPackage().getName() + ";" + + YarnJacksonJaxbJsonProvider.class.getPackage().getName(), "/*"); + webServer.start(); + } + + /** + * Start embedded Hadoop HTTP server. + * + * @return an instance of the started HTTP server. + * @throws IOException in case there is an error while starting server. + */ + static ResourceEstimatorServer startResourceEstimatorServer() + throws IOException, InterruptedException { + Configuration config = new YarnConfiguration(); + config.addResource(ResourceEstimatorConfiguration.CONFIG_FILE); + ResourceEstimatorServer resourceEstimatorServer = null; + try { + resourceEstimatorServer = new ResourceEstimatorServer(); + ShutdownHookManager.get().addShutdownHook( + new CompositeServiceShutdownHook(resourceEstimatorServer), 30); + resourceEstimatorServer.init(config); + resourceEstimatorServer.start(); + } catch (Throwable t) { + LOGGER.error("Error starting ResourceEstimatorServer", t); + } + + return resourceEstimatorServer; + } + + public static void main(String[] args) + throws InterruptedException, IOException { + ResourceEstimatorServer server = startResourceEstimatorServer(); + server.join(); + } + + /** + * Stop embedded Hadoop HTTP server. + * + * @throws Exception in case the HTTP server fails to shut down. + */ + public void shutdown() throws Exception { + LOGGER.info("Stopping resourceestimator service at: {}.", + baseURI.toString()); + webServer.stop(); + } +} diff --git a/hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/service/ResourceEstimatorService.java b/hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/service/ResourceEstimatorService.java new file mode 100644 index 0000000000..933332e0d6 --- /dev/null +++ b/hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/service/ResourceEstimatorService.java @@ -0,0 +1,238 @@ +/* + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package org.apache.hadoop.resourceestimator.service; + +import java.io.IOException; +import java.io.InputStream; +import java.lang.reflect.Type; +import java.util.List; +import java.util.Map; + +import javax.ws.rs.DELETE; +import javax.ws.rs.GET; +import javax.ws.rs.POST; +import javax.ws.rs.Path; +import javax.ws.rs.PathParam; +import javax.ws.rs.Produces; +import javax.ws.rs.core.MediaType; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.resourceestimator.common.api.RecurrenceId; +import org.apache.hadoop.resourceestimator.common.api.ResourceSkyline; +import org.apache.hadoop.resourceestimator.common.config.ResourceEstimatorConfiguration; +import org.apache.hadoop.resourceestimator.common.config.ResourceEstimatorUtil; +import org.apache.hadoop.resourceestimator.common.exception.ResourceEstimatorException; +import org.apache.hadoop.resourceestimator.common.serialization.RLESparseResourceAllocationSerDe; +import org.apache.hadoop.resourceestimator.common.serialization.ResourceSerDe; +import org.apache.hadoop.resourceestimator.skylinestore.api.SkylineStore; +import org.apache.hadoop.resourceestimator.skylinestore.exceptions.SkylineStoreException; +import org.apache.hadoop.resourceestimator.solver.api.Solver; +import org.apache.hadoop.resourceestimator.solver.exceptions.SolverException; +import org.apache.hadoop.resourceestimator.translator.api.LogParser; +import org.apache.hadoop.resourceestimator.translator.impl.LogParserUtil; +import org.apache.hadoop.yarn.api.records.Resource; +import org.apache.hadoop.yarn.server.resourcemanager.reservation.RLESparseResourceAllocation; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.google.gson.Gson; +import com.google.gson.GsonBuilder; +import com.google.gson.reflect.TypeToken; +import com.google.inject.Singleton; + +/** + * Resource Estimator Service which provides a set of REST APIs for users to + * use the estimation service. + */ +@Singleton @Path("/resourceestimator") public class ResourceEstimatorService { + private static final Logger LOGGER = + LoggerFactory.getLogger(ResourceEstimatorService.class); + private static SkylineStore skylineStore; + private static Solver solver; + private static LogParser logParser; + private static LogParserUtil logParserUtil = new LogParserUtil(); + private static Configuration config; + private static Gson gson; + private static Type rleType; + private static Type skylineStoreType; + + public ResourceEstimatorService() throws ResourceEstimatorException { + if (skylineStore == null) { + try { + config = new Configuration(); + config.addResource(ResourceEstimatorConfiguration.CONFIG_FILE); + skylineStore = ResourceEstimatorUtil.createProviderInstance(config, + ResourceEstimatorConfiguration.SKYLINESTORE_PROVIDER, + ResourceEstimatorConfiguration.DEFAULT_SKYLINESTORE_PROVIDER, + SkylineStore.class); + logParser = ResourceEstimatorUtil.createProviderInstance(config, + ResourceEstimatorConfiguration.TRANSLATOR_PROVIDER, + ResourceEstimatorConfiguration.DEFAULT_TRANSLATOR_PROVIDER, + LogParser.class); + logParser.init(config, skylineStore); + logParserUtil.setLogParser(logParser); + solver = ResourceEstimatorUtil.createProviderInstance(config, + ResourceEstimatorConfiguration.SOLVER_PROVIDER, + ResourceEstimatorConfiguration.DEFAULT_SOLVER_PROVIDER, + Solver.class); + solver.init(config, skylineStore); + } catch (Exception ex) { + LOGGER + .error("Server initialization failed due to: {}", ex.getMessage()); + throw new ResourceEstimatorException(ex.getMessage(), ex); + } + gson = new GsonBuilder() + .registerTypeAdapter(Resource.class, new ResourceSerDe()) + .registerTypeAdapter(RLESparseResourceAllocation.class, + new RLESparseResourceAllocationSerDe()) + .enableComplexMapKeySerialization().create(); + rleType = new TypeToken() { + }.getType(); + skylineStoreType = + new TypeToken>>() { + }.getType(); + } + } + + /** + * Parse the log file. See also {@link LogParser#parseStream(InputStream)}. + * + * @param logFile file/directory of the log to be parsed. + * @throws IOException if fails to parse the log. + * @throws SkylineStoreException if fails to addHistory to + * {@link SkylineStore}. + * @throws ResourceEstimatorException if the {@link LogParser} + * is not initialized. + */ + @POST @Path("/translator/{logFile : .+}") public void parseFile( + @PathParam("logFile") String logFile) + throws IOException, SkylineStoreException, ResourceEstimatorException { + logParserUtil.parseLog(logFile); + LOGGER.debug("Parse logFile: {}.", logFile); + } + + /** + * Get predicted {code Resource} allocation for the pipeline. If the + * prediction for the pipeline already exists in the {@link SkylineStore}, it + * will directly get the prediction from {@link SkylineStore}, otherwise it + * will call the {@link Solver} to make prediction, and store the predicted + * {code Resource} allocation to the {@link SkylineStore}. Note that invoking + * {@link Solver} could be a time-consuming operation. + * + * @param pipelineId the id of the pipeline. + * @return Json format of {@link RLESparseResourceAllocation}. + * @throws SolverException if {@link Solver} fails; + * @throws SkylineStoreException if fails to get history + * {@link ResourceSkyline} or predicted {code Resource} allocation + * from {@link SkylineStore}. + */ + @GET @Path("/estimator/{pipelineId}") @Produces(MediaType.APPLICATION_JSON) + public String getPrediction( + @PathParam(value = "pipelineId") String pipelineId) + throws SolverException, SkylineStoreException { + // first, try to grab the predicted resource allocation from the skyline + // store + RLESparseResourceAllocation result = skylineStore.getEstimation(pipelineId); + // if received resource allocation is null, then run the solver + if (result == null) { + RecurrenceId recurrenceId = new RecurrenceId(pipelineId, "*"); + Map> jobHistory = + skylineStore.getHistory(recurrenceId); + result = solver.solve(jobHistory); + } + final String prediction = gson.toJson(result, rleType); + LOGGER.debug("Predict resource requests for pipelineId: {}." + pipelineId); + + return prediction; + } + + /** + * Get history {@link ResourceSkyline} from {@link SkylineStore}. This + * function supports the following special wildcard operations regarding + * {@link RecurrenceId}: If the {@code pipelineId} is "*", it will return all + * entries in the store; else, if the {@code runId} is "*", it will return all + * {@link ResourceSkyline}s belonging to the {@code pipelineId}; else, it will + * return all {@link ResourceSkyline}s belonging to the {{@code pipelineId}, + * {@code runId}}. If the {@link RecurrenceId} does not exist, it will not do + * anything. + * + * @param pipelineId pipelineId of the history run. + * @param runId runId of the history run. + * @return Json format of history {@link ResourceSkyline}s. + * @throws SkylineStoreException if fails to getHistory + * {@link ResourceSkyline} from {@link SkylineStore}. + */ + @GET @Path("/skylinestore/history/{pipelineId}/{runId}") + @Produces(MediaType.APPLICATION_JSON) + public String getHistoryResourceSkyline( + @PathParam("pipelineId") String pipelineId, + @PathParam("runId") String runId) throws SkylineStoreException { + RecurrenceId recurrenceId = new RecurrenceId(pipelineId, runId); + Map> jobHistory = + skylineStore.getHistory(recurrenceId); + final String skyline = gson.toJson(jobHistory, skylineStoreType); + LOGGER + .debug("Query the skyline store for recurrenceId: {}." + recurrenceId); + + recurrenceId = new RecurrenceId("*", "*"); + jobHistory = skylineStore.getHistory(recurrenceId); + + return skyline; + } + + /** + * Get estimated {code Resource} allocation for the pipeline. + * + * @param pipelineId id of the pipeline. + * @return Json format of {@link RLESparseResourceAllocation}. + * @throws SkylineStoreException if fails to get estimated {code Resource} + * allocation from {@link SkylineStore}. + */ + @GET @Path("/skylinestore/estimation/{pipelineId}") + @Produces(MediaType.APPLICATION_JSON) + public String getEstimatedResourceAllocation( + @PathParam("pipelineId") String pipelineId) throws SkylineStoreException { + RLESparseResourceAllocation result = skylineStore.getEstimation(pipelineId); + final String skyline = gson.toJson(result, rleType); + LOGGER.debug("Query the skyline store for pipelineId: {}." + pipelineId); + + return skyline; + } + + /** + * Delete history {@link ResourceSkyline}s from {@link SkylineStore}. + *

Note that for safety considerations, we only allow users to delete + * history {@link ResourceSkyline}s of one job run. + * + * @param pipelineId pipelineId of the history run. + * @param runId runId runId of the history run. + * @throws SkylineStoreException if fails to deleteHistory + * {@link ResourceSkyline}s. + */ + @DELETE @Path("/skylinestore/history/{pipelineId}/{runId}") + public void deleteHistoryResourceSkyline( + @PathParam("pipelineId") String pipelineId, + @PathParam("runId") String runId) throws SkylineStoreException { + RecurrenceId recurrenceId = new RecurrenceId(pipelineId, runId); + skylineStore.deleteHistory(recurrenceId); + LOGGER.info("Delete ResourceSkyline for recurrenceId: {}.", recurrenceId); + } +} diff --git a/hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/service/ShutdownHook.java b/hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/service/ShutdownHook.java new file mode 100644 index 0000000000..23e1413fca --- /dev/null +++ b/hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/service/ShutdownHook.java @@ -0,0 +1,45 @@ +/* + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package org.apache.hadoop.resourceestimator.service; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Simple shutdown hook for {@link ResourceEstimatorServer}. + */ +public class ShutdownHook extends Thread { + private static final Logger LOGGER = + LoggerFactory.getLogger(ShutdownHook.class); + private final ResourceEstimatorServer server; + + ShutdownHook(ResourceEstimatorServer server) { + this.server = server; + } + + public void run() { + try { + server.shutdown(); + } catch (Exception e) { + LOGGER.error("HttpServer fails to shut down!"); + } + } +} diff --git a/hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/service/package-info.java b/hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/service/package-info.java new file mode 100644 index 0000000000..3571736c50 --- /dev/null +++ b/hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/service/package-info.java @@ -0,0 +1,23 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/** + * Resource estimator service. + */ + +package org.apache.hadoop.resourceestimator.service; \ No newline at end of file diff --git a/hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/skylinestore/api/HistorySkylineStore.java b/hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/skylinestore/api/HistorySkylineStore.java new file mode 100644 index 0000000000..8fe461937a --- /dev/null +++ b/hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/skylinestore/api/HistorySkylineStore.java @@ -0,0 +1,99 @@ +/* + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package org.apache.hadoop.resourceestimator.skylinestore.api; + +import java.util.List; +import java.util.Map; + +import org.apache.hadoop.resourceestimator.common.api.RecurrenceId; +import org.apache.hadoop.resourceestimator.common.api.ResourceSkyline; +import org.apache.hadoop.resourceestimator.skylinestore.exceptions.SkylineStoreException; + +/** + * HistorySkylineStore stores pipeline job's {@link ResourceSkyline}s in all + * runs. {@code Estimator} will query the {@link ResourceSkyline}s for pipeline + * jobs. {@code Parser} will parse various types of job logs, construct + * {@link ResourceSkyline}s out of the logs and store them in the SkylineStore. + */ +public interface HistorySkylineStore { + /** + * Add job's resource skyline to the store indexed by the job's + * {@link RecurrenceId}. {@link RecurrenceId} is used to identify recurring + * pipeline jobs, and we assume that {@code + * ResourceEstimatorServer} users will provide the correct + * {@link RecurrenceId}.

If {@link ResourceSkyline}s to be added contain + * null elements, the function will skip them. + * + * @param recurrenceId the unique id of user's recurring pipeline jobs. + * @param resourceSkylines the list of {@link ResourceSkyline}s in one run. + * @throws SkylineStoreException if: (1) input parameters are invalid; (2) + * {@link ResourceSkyline}s to be added contain some duplicate + * {@link RecurrenceId}s which already exist in the + * {@link HistorySkylineStore}. + */ + void addHistory(RecurrenceId recurrenceId, + List resourceSkylines) throws SkylineStoreException; + + /** + * Delete all {@link ResourceSkyline}s belonging to given + * {@link RecurrenceId}. + *

Note that for safety considerations, we only allow users to + * deleteHistory {@link ResourceSkyline}s of one job run. + * + * @param recurrenceId the unique id of user's recurring pipeline jobs. + * @throws SkylineStoreException if: (1) input parameters are invalid; (2) + * recurrenceId does not exist in the {@link HistorySkylineStore}. + */ + void deleteHistory(RecurrenceId recurrenceId) throws SkylineStoreException; + + /** + * Update {@link RecurrenceId} with given {@link ResourceSkyline}s. This + * function will deleteHistory all the {@link ResourceSkyline}s belonging to + * the {@link RecurrenceId}, and re-insert the given {@link ResourceSkyline}s + * to the SkylineStore. + *

If {@link ResourceSkyline}s contain null elements, + * the function will skip them. + * + * @param recurrenceId the unique id of the pipeline job. + * @param resourceSkylines the list of {@link ResourceSkyline}s in one run. + * @throws SkylineStoreException if: (1) input parameters are invalid; (2) + * recurrenceId does not exist in the SkylineStore. + */ + void updateHistory(RecurrenceId recurrenceId, + List resourceSkylines) throws SkylineStoreException; + + /** + * Return all {@link ResourceSkyline}s belonging to {@link RecurrenceId}. + *

This function supports the following special wildcard operations + * regarding {@link RecurrenceId}: If the {@code pipelineId} is "*", it will + * return all entries in the store; else, if the {@code runId} is "*", it + * will return all {@link ResourceSkyline}s belonging to the + * {@code pipelineId}; else, it will return all {@link ResourceSkyline}s + * belonging to the {{@code pipelineId}, {@code runId}}. If the + * {@link RecurrenceId} does not exist, it will return null. + * + * @param recurrenceId the unique id of the pipeline job. + * @return all {@link ResourceSkyline}s belonging to the recurrenceId. + * @throws SkylineStoreException if recurrenceId is null. + */ + Map> getHistory(RecurrenceId recurrenceId) + throws SkylineStoreException; +} diff --git a/hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/skylinestore/api/PredictionSkylineStore.java b/hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/skylinestore/api/PredictionSkylineStore.java new file mode 100644 index 0000000000..c3fedce47b --- /dev/null +++ b/hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/skylinestore/api/PredictionSkylineStore.java @@ -0,0 +1,60 @@ +/* + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package org.apache.hadoop.resourceestimator.skylinestore.api; + +import org.apache.hadoop.resourceestimator.skylinestore.exceptions.SkylineStoreException; +import org.apache.hadoop.yarn.server.resourcemanager.reservation.RLESparseResourceAllocation; + +/** + * PredictionSkylineStore stores the predicted + * {@code RLESparseResourceAllocation} of a job as computed by the + * {@code Estimator} based on the {@code ResourceSkyline}s of past executions in + * the {@code HistorySkylineStore}. + */ +public interface PredictionSkylineStore { + + /** + * Add job's predicted {@code Resource} allocation to the store + * indexed by the {@code + * pipelineId}. + *

Note that right now we only keep the latest copy of predicted + * {@code Resource} allocation for the recurring pipeline. + * + * @param pipelineId the id of the recurring pipeline. + * @param resourceOverTime the predicted {@code Resource} allocation for the + * pipeline. + * @throws SkylineStoreException if input parameters are invalid. + */ + void addEstimation(String pipelineId, + RLESparseResourceAllocation resourceOverTime) + throws SkylineStoreException; + + /** + * Return the predicted {@code Resource} allocation for the pipeline. + *

If the pipelineId does not exist, it will return null. + * + * @param pipelineId the unique id of the pipeline. + * @return the predicted {@code Resource} allocation for the pipeline. + * @throws SkylineStoreException if pipelineId is null. + */ + RLESparseResourceAllocation getEstimation(String pipelineId) + throws SkylineStoreException; +} diff --git a/hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/skylinestore/api/SkylineStore.java b/hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/skylinestore/api/SkylineStore.java new file mode 100644 index 0000000000..f352ed4aa3 --- /dev/null +++ b/hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/skylinestore/api/SkylineStore.java @@ -0,0 +1,30 @@ +/* + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package org.apache.hadoop.resourceestimator.skylinestore.api; + +/** + * SkylineStore is composable interface for storing the history + * {@code ResourceSkyline}s of past job runs and the predicted + * {@code RLESparseResourceAllocation} for future execution. + */ +public interface SkylineStore + extends HistorySkylineStore, PredictionSkylineStore { +} diff --git a/hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/skylinestore/api/package-info.java b/hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/skylinestore/api/package-info.java new file mode 100644 index 0000000000..e833486075 --- /dev/null +++ b/hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/skylinestore/api/package-info.java @@ -0,0 +1,23 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/** + * APIs for the {@code SkylineStore}. + */ + +package org.apache.hadoop.resourceestimator.skylinestore.api; \ No newline at end of file diff --git a/hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/skylinestore/exceptions/DuplicateRecurrenceIdException.java b/hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/skylinestore/exceptions/DuplicateRecurrenceIdException.java new file mode 100644 index 0000000000..7c924809d0 --- /dev/null +++ b/hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/skylinestore/exceptions/DuplicateRecurrenceIdException.java @@ -0,0 +1,33 @@ +/* + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package org.apache.hadoop.resourceestimator.skylinestore.exceptions; + +/** + * Exception thrown the {@code RecurrenceId} already exists in the + * {@code SkylineStore}. + */ +public class DuplicateRecurrenceIdException extends SkylineStoreException { + private static final long serialVersionUID = -684069387367879218L; + + public DuplicateRecurrenceIdException(final String message) { + super(message); + } +} diff --git a/hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/skylinestore/exceptions/EmptyResourceSkylineException.java b/hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/skylinestore/exceptions/EmptyResourceSkylineException.java new file mode 100644 index 0000000000..55a8fa7eb3 --- /dev/null +++ b/hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/skylinestore/exceptions/EmptyResourceSkylineException.java @@ -0,0 +1,33 @@ +/* + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package org.apache.hadoop.resourceestimator.skylinestore.exceptions; + +/** + * Exception thrown if the @link{ResourceSkyline}s to be added to the + * {@code SkylineStore} is empty. + */ +public class EmptyResourceSkylineException extends SkylineStoreException { + private static final long serialVersionUID = -684069387367879218L; + + public EmptyResourceSkylineException(final String message) { + super(message); + } +} diff --git a/hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/skylinestore/exceptions/NullPipelineIdException.java b/hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/skylinestore/exceptions/NullPipelineIdException.java new file mode 100644 index 0000000000..d48be7d61f --- /dev/null +++ b/hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/skylinestore/exceptions/NullPipelineIdException.java @@ -0,0 +1,32 @@ +/* + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package org.apache.hadoop.resourceestimator.skylinestore.exceptions; + +/** + * Exception thrown when pipelineId to be added is null. + */ +public class NullPipelineIdException extends SkylineStoreException { + private static final long serialVersionUID = -684069387367879218L; + + public NullPipelineIdException(final String message) { + super(message); + } +} diff --git a/hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/skylinestore/exceptions/NullRLESparseResourceAllocationException.java b/hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/skylinestore/exceptions/NullRLESparseResourceAllocationException.java new file mode 100644 index 0000000000..9aee0b69a1 --- /dev/null +++ b/hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/skylinestore/exceptions/NullRLESparseResourceAllocationException.java @@ -0,0 +1,33 @@ +/* + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package org.apache.hadoop.resourceestimator.skylinestore.exceptions; + +/** + * Exception thrown if the {@code ResourceSkyline} to be added is null. + */ +public class NullRLESparseResourceAllocationException + extends SkylineStoreException { + private static final long serialVersionUID = -684069387367879218L; + + public NullRLESparseResourceAllocationException(final String message) { + super(message); + } +} diff --git a/hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/skylinestore/exceptions/NullRecurrenceIdException.java b/hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/skylinestore/exceptions/NullRecurrenceIdException.java new file mode 100644 index 0000000000..518c0650b5 --- /dev/null +++ b/hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/skylinestore/exceptions/NullRecurrenceIdException.java @@ -0,0 +1,32 @@ +/* + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package org.apache.hadoop.resourceestimator.skylinestore.exceptions; + +/** + * Exception thrown the {@code RecurrenceId} to be added is null. + */ +public class NullRecurrenceIdException extends SkylineStoreException { + private static final long serialVersionUID = -684069387367879218L; + + public NullRecurrenceIdException(final String message) { + super(message); + } +} diff --git a/hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/skylinestore/exceptions/NullResourceSkylineException.java b/hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/skylinestore/exceptions/NullResourceSkylineException.java new file mode 100644 index 0000000000..b70c7646a3 --- /dev/null +++ b/hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/skylinestore/exceptions/NullResourceSkylineException.java @@ -0,0 +1,32 @@ +/* + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package org.apache.hadoop.resourceestimator.skylinestore.exceptions; + +/** + * Exception thrown if the {@code ResourceSkyline} to be added is null. + */ +public class NullResourceSkylineException extends SkylineStoreException { + private static final long serialVersionUID = -684069387367879218L; + + public NullResourceSkylineException(final String message) { + super(message); + } +} diff --git a/hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/skylinestore/exceptions/RecurrenceIdNotFoundException.java b/hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/skylinestore/exceptions/RecurrenceIdNotFoundException.java new file mode 100644 index 0000000000..b5e734dc49 --- /dev/null +++ b/hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/skylinestore/exceptions/RecurrenceIdNotFoundException.java @@ -0,0 +1,33 @@ +/* + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package org.apache.hadoop.resourceestimator.skylinestore.exceptions; + +/** + * Exception thrown if {@code RecurrenceId} is not found in the + * {@code SkylineStore}. + */ +public class RecurrenceIdNotFoundException extends SkylineStoreException { + private static final long serialVersionUID = -684069387367879218L; + + public RecurrenceIdNotFoundException(final String message) { + super(message); + } +} diff --git a/hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/skylinestore/exceptions/SkylineStoreException.java b/hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/skylinestore/exceptions/SkylineStoreException.java new file mode 100644 index 0000000000..751b5dd311 --- /dev/null +++ b/hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/skylinestore/exceptions/SkylineStoreException.java @@ -0,0 +1,33 @@ +/* + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package org.apache.hadoop.resourceestimator.skylinestore.exceptions; + +/** + * Exception thrown the @link{SkylineStore} or the {@code Estimator} tries to + * addHistory or query pipeline job's resource skylines. + */ +public abstract class SkylineStoreException extends Exception { + private static final long serialVersionUID = -684069387367879218L; + + public SkylineStoreException(final String message) { + super(message); + } +} diff --git a/hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/skylinestore/exceptions/package-info.java b/hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/skylinestore/exceptions/package-info.java new file mode 100644 index 0000000000..716e090c68 --- /dev/null +++ b/hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/skylinestore/exceptions/package-info.java @@ -0,0 +1,24 @@ +/* + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +/** + * SkylineStore exception module. + */ +package org.apache.hadoop.resourceestimator.skylinestore.exceptions; diff --git a/hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/skylinestore/impl/InMemoryStore.java b/hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/skylinestore/impl/InMemoryStore.java new file mode 100644 index 0000000000..e00f3a061e --- /dev/null +++ b/hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/skylinestore/impl/InMemoryStore.java @@ -0,0 +1,256 @@ +/* + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package org.apache.hadoop.resourceestimator.skylinestore.impl; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantReadWriteLock; + +import org.apache.hadoop.resourceestimator.common.api.RecurrenceId; +import org.apache.hadoop.resourceestimator.common.api.ResourceSkyline; +import org.apache.hadoop.resourceestimator.skylinestore.api.SkylineStore; +import org.apache.hadoop.resourceestimator.skylinestore.exceptions.DuplicateRecurrenceIdException; +import org.apache.hadoop.resourceestimator.skylinestore.exceptions.EmptyResourceSkylineException; +import org.apache.hadoop.resourceestimator.skylinestore.exceptions.RecurrenceIdNotFoundException; +import org.apache.hadoop.resourceestimator.skylinestore.exceptions.SkylineStoreException; +import org.apache.hadoop.resourceestimator.skylinestore.validator.SkylineStoreValidator; +import org.apache.hadoop.yarn.server.resourcemanager.reservation.RLESparseResourceAllocation; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * An in-memory implementation of {@link SkylineStore}. + */ +public class InMemoryStore implements SkylineStore { + private static final Logger LOGGER = + LoggerFactory.getLogger(InMemoryStore.class); + private final ReentrantReadWriteLock readWriteLock = + new ReentrantReadWriteLock(); + private final Lock readLock = readWriteLock.readLock(); + private final Lock writeLock = readWriteLock.writeLock(); + private final SkylineStoreValidator inputValidator = + new SkylineStoreValidator(); + /** + * A pipeline job's history {@link ResourceSkyline}s. TODO: we may flatten it + * out for quick access. + */ + private final Map> skylineStore = + new HashMap<>(); // pipelineId, resource skyline + // Recurring pipeline's predicted {@link ResourceSkyline}s. + private final Map estimationStore = + new HashMap<>(); // pipelineId, ResourceSkyline + + private List eliminateNull( + final List resourceSkylines) { + final List result = new ArrayList<>(); + for (final ResourceSkyline resourceSkyline : resourceSkylines) { + if (resourceSkyline != null) { + result.add(resourceSkyline); + } + } + return result; + } + + @Override public final void addHistory(final RecurrenceId recurrenceId, + final List resourceSkylines) + throws SkylineStoreException { + inputValidator.validate(recurrenceId, resourceSkylines); + writeLock.lock(); + try { + // remove the null elements in the resourceSkylines + final List filteredInput = + eliminateNull(resourceSkylines); + if (filteredInput.size() > 0) { + if (skylineStore.containsKey(recurrenceId)) { + // if filteredInput has duplicate jobIds with existing skylines in the + // store, + // throw out an exception + final List jobHistory = + skylineStore.get(recurrenceId); + final List oldJobIds = new ArrayList<>(); + for (final ResourceSkyline resourceSkyline : jobHistory) { + oldJobIds.add(resourceSkyline.getJobId()); + } + if (!oldJobIds.isEmpty()) { + for (ResourceSkyline elem : filteredInput) { + if (oldJobIds.contains(elem.getJobId())) { + StringBuilder errMsg = new StringBuilder(); + errMsg.append( + "Trying to addHistory duplicate resource skylines for " + + recurrenceId + + ". Use updateHistory function instead."); + LOGGER.error(errMsg.toString()); + throw new DuplicateRecurrenceIdException(errMsg.toString()); + } + } + } + skylineStore.get(recurrenceId).addAll(filteredInput); + LOGGER.info("Successfully addHistory new resource skylines for {}.", + recurrenceId); + } else { + skylineStore.put(recurrenceId, filteredInput); + LOGGER.info("Successfully addHistory new resource skylines for {}.", + recurrenceId); + } + } + } finally { + writeLock.unlock(); + } + } + + @Override public void addEstimation(String pipelineId, + RLESparseResourceAllocation resourceSkyline) + throws SkylineStoreException { + inputValidator.validate(pipelineId, resourceSkyline); + writeLock.lock(); + try { + estimationStore.put(pipelineId, resourceSkyline); + LOGGER.info("Successfully add estimated resource allocation for {}.", + pipelineId); + } finally { + writeLock.unlock(); + } + } + + @Override public final void deleteHistory(final RecurrenceId recurrenceId) + throws SkylineStoreException { + inputValidator.validate(recurrenceId); + writeLock.lock(); + try { + if (skylineStore.containsKey(recurrenceId)) { + skylineStore.remove(recurrenceId); + LOGGER.warn("Delete resource skylines for {}.", recurrenceId); + } else { + StringBuilder errMsg = new StringBuilder(); + errMsg.append( + "Trying to deleteHistory non-existing recurring pipeline " + + recurrenceId + "\'s resource skylines"); + LOGGER.error(errMsg.toString()); + throw new RecurrenceIdNotFoundException(errMsg.toString()); + } + } finally { + writeLock.unlock(); + } + } + + @Override public final void updateHistory(final RecurrenceId recurrenceId, + final List resourceSkylines) + throws SkylineStoreException { + inputValidator.validate(recurrenceId, resourceSkylines); + writeLock.lock(); + try { + if (skylineStore.containsKey(recurrenceId)) { + // remove the null elements in the resourceSkylines + List filteredInput = eliminateNull(resourceSkylines); + if (filteredInput.size() > 0) { + skylineStore.put(recurrenceId, filteredInput); + LOGGER.info("Successfully updateHistory resource skylines for {}.", + recurrenceId); + } else { + StringBuilder errMsg = new StringBuilder(); + errMsg.append("Trying to updateHistory " + recurrenceId + + " with empty resource skyline"); + LOGGER.error(errMsg.toString()); + throw new EmptyResourceSkylineException(errMsg.toString()); + } + } else { + StringBuilder errMsg = new StringBuilder(); + errMsg.append( + "Trying to updateHistory non-existing resource skylines for " + + recurrenceId); + LOGGER.error(errMsg.toString()); + throw new RecurrenceIdNotFoundException(errMsg.toString()); + } + } finally { + writeLock.unlock(); + } + } + + @Override public final Map> getHistory( + final RecurrenceId recurrenceId) throws SkylineStoreException { + inputValidator.validate(recurrenceId); + readLock.lock(); + try { + String pipelineId = recurrenceId.getPipelineId(); + // User tries to getHistory all resource skylines in the skylineStore + if (pipelineId.equals("*")) { + LOGGER + .info("Successfully query resource skylines for {}.", recurrenceId); + return Collections.unmodifiableMap(skylineStore); + } + String runId = recurrenceId.getRunId(); + Map> result = + new HashMap>(); + // User tries to getHistory pipelineId's all resource skylines in the + // skylineStore + if (runId.equals("*")) { + // TODO: this for loop is expensive, so we may change the type of + // skylineStore to + // speed up this loop. + for (Map.Entry> entry : skylineStore + .entrySet()) { + RecurrenceId index = entry.getKey(); + if (index.getPipelineId().equals(pipelineId)) { + result.put(index, entry.getValue()); + } + } + if (result.size() > 0) { + LOGGER.info("Successfully query resource skylines for {}.", + recurrenceId); + return Collections.unmodifiableMap(result); + } else { + LOGGER.warn( + "Trying to getHistory non-existing resource skylines for {}.", + recurrenceId); + return null; + } + } + // User tries to getHistory {pipelineId, runId}'s resource skylines + if (skylineStore.containsKey(recurrenceId)) { + result.put(recurrenceId, skylineStore.get(recurrenceId)); + } else { + LOGGER + .warn("Trying to getHistory non-existing resource skylines for {}.", + recurrenceId); + return null; + } + LOGGER.info("Successfully query resource skylines for {}.", recurrenceId); + return Collections.unmodifiableMap(result); + } finally { + readLock.unlock(); + } + } + + @Override public final RLESparseResourceAllocation getEstimation( + String pipelineId) throws SkylineStoreException { + inputValidator.validate(pipelineId); + readLock.lock(); + try { + return estimationStore.get(pipelineId); + } finally { + readLock.unlock(); + } + } +} diff --git a/hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/skylinestore/impl/package-info.java b/hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/skylinestore/impl/package-info.java new file mode 100644 index 0000000000..ffccd5de16 --- /dev/null +++ b/hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/skylinestore/impl/package-info.java @@ -0,0 +1,23 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/** + * Implementation for {@code SkylineStore}. + */ + +package org.apache.hadoop.resourceestimator.skylinestore.impl; \ No newline at end of file diff --git a/hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/skylinestore/validator/SkylineStoreValidator.java b/hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/skylinestore/validator/SkylineStoreValidator.java new file mode 100644 index 0000000000..f5f50f5eaf --- /dev/null +++ b/hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/skylinestore/validator/SkylineStoreValidator.java @@ -0,0 +1,118 @@ +/* + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package org.apache.hadoop.resourceestimator.skylinestore.validator; + +import java.util.List; + +import org.apache.hadoop.resourceestimator.common.api.RecurrenceId; +import org.apache.hadoop.resourceestimator.common.api.ResourceSkyline; +import org.apache.hadoop.resourceestimator.skylinestore.api.SkylineStore; +import org.apache.hadoop.resourceestimator.skylinestore.exceptions.NullPipelineIdException; +import org.apache.hadoop.resourceestimator.skylinestore.exceptions.NullRLESparseResourceAllocationException; +import org.apache.hadoop.resourceestimator.skylinestore.exceptions.NullRecurrenceIdException; +import org.apache.hadoop.resourceestimator.skylinestore.exceptions.NullResourceSkylineException; +import org.apache.hadoop.resourceestimator.skylinestore.exceptions.SkylineStoreException; +import org.apache.hadoop.yarn.server.resourcemanager.reservation.RLESparseResourceAllocation; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * SkylineStoreValidator validates input parameters for {@link SkylineStore}. + */ +public class SkylineStoreValidator { + private static final Logger LOGGER = + LoggerFactory.getLogger(SkylineStoreValidator.class); + + /** + * Check if recurrenceId is null. + * + * @param recurrenceId the id of the recurring pipeline job. + * @throws SkylineStoreException if input parameters are invalid. + */ + public final void validate(final RecurrenceId recurrenceId) + throws SkylineStoreException { + if (recurrenceId == null) { + StringBuilder sb = new StringBuilder(); + sb.append("Recurrence id is null, please try again by specifying" + + " a valid Recurrence id."); + LOGGER.error(sb.toString()); + throw new NullRecurrenceIdException(sb.toString()); + } + } + + /** + * Check if pipelineId is null. + * + * @param pipelineId the id of the recurring pipeline job. + * @throws SkylineStoreException if input parameters are invalid. + */ + public final void validate(final String pipelineId) + throws SkylineStoreException { + if (pipelineId == null) { + StringBuilder sb = new StringBuilder(); + sb.append("pipelineId is null, please try again by specifying" + + " a valid pipelineId."); + LOGGER.error(sb.toString()); + throw new NullPipelineIdException(sb.toString()); + } + } + + /** + * Check if recurrenceId is null or resourceSkylines is + * null. + * + * @param recurrenceId the id of the recurring pipeline job. + * @param resourceSkylines the list of {@link ResourceSkyline}s to be added. + * @throws SkylineStoreException if input parameters are invalid. + */ + public final void validate(final RecurrenceId recurrenceId, + final List resourceSkylines) + throws SkylineStoreException { + validate(recurrenceId); + if (resourceSkylines == null) { + StringBuilder sb = new StringBuilder(); + sb.append("ResourceSkylines for " + recurrenceId + + " is null, please try again by " + + "specifying valid ResourceSkylines."); + LOGGER.error(sb.toString()); + throw new NullResourceSkylineException(sb.toString()); + } + } + + /** + * Check if pipelineId is null or resourceOverTime is null. + * + * @param pipelineId the id of the recurring pipeline. + * @param resourceOverTime predicted {@code Resource} allocation to be added. + * @throws SkylineStoreException if input parameters are invalid. + */ + public final void validate(final String pipelineId, + final RLESparseResourceAllocation resourceOverTime) + throws SkylineStoreException { + validate(pipelineId); + if (resourceOverTime == null) { + StringBuilder sb = new StringBuilder(); + sb.append("Resource allocation for " + pipelineId + " is null."); + LOGGER.error(sb.toString()); + throw new NullRLESparseResourceAllocationException(sb.toString()); + } + } +} diff --git a/hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/skylinestore/validator/package-info.java b/hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/skylinestore/validator/package-info.java new file mode 100644 index 0000000000..23d67c50bc --- /dev/null +++ b/hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/skylinestore/validator/package-info.java @@ -0,0 +1,23 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/** + * Validator for {@code SkylineStore}. + */ + +package org.apache.hadoop.resourceestimator.skylinestore.validator; \ No newline at end of file diff --git a/hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/solver/api/Solver.java b/hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/solver/api/Solver.java new file mode 100644 index 0000000000..7958a6f29c --- /dev/null +++ b/hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/solver/api/Solver.java @@ -0,0 +1,76 @@ +/* + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package org.apache.hadoop.resourceestimator.solver.api; + +import java.util.List; +import java.util.Map; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.resourceestimator.common.api.RecurrenceId; +import org.apache.hadoop.resourceestimator.common.api.ResourceSkyline; +import org.apache.hadoop.resourceestimator.skylinestore.api.PredictionSkylineStore; +import org.apache.hadoop.resourceestimator.skylinestore.exceptions.SkylineStoreException; +import org.apache.hadoop.resourceestimator.solver.exceptions.SolverException; +import org.apache.hadoop.yarn.api.records.Resource; +import org.apache.hadoop.yarn.server.resourcemanager.reservation.RLESparseResourceAllocation; + +/** + * Solver takes recurring pipeline's {@link ResourceSkyline} history as input, + * predicts its {@link Resource} requirement at each time t for the next run, + * and translate them into {@link ResourceSkyline} which will be used to make + * recurring resource reservations. + */ +public interface Solver { + /** + * Initializing the Solver, including loading solver parameters from + * configuration file. + * + * @param config {@link Configuration} for the Solver. + * @param skylineStore the {@link PredictionSkylineStore} which stores + * predicted {@code Resource} allocations. + */ + void init(Configuration config, PredictionSkylineStore skylineStore); + + /** + * The Solver reads recurring pipeline's {@link ResourceSkyline} history, and + * precits its {@link ResourceSkyline} requirements for the next run. + * + * @param jobHistory the {@link ResourceSkyline}s of the recurring pipeline in + * previous runs. The {@link RecurrenceId} identifies one run of the + * recurring pipeline, and the list of {@link ResourceSkyline}s + * records the {@link ResourceSkyline} of each job within the pipeline. + * @return the amount of {@link Resource} requested by the pipeline for the + * next run (discretized by timeInterval). + * @throws SolverException if: (1) input is invalid; (2) the number of + * instances in the jobHistory is smaller than the minimum + * requirement; (3) solver runtime has unexpected behaviors; + * @throws SkylineStoreException if it fails to add predicted {@code Resource} + * allocation to the {@link PredictionSkylineStore}. + */ + RLESparseResourceAllocation solve( + Map> jobHistory) + throws SolverException, SkylineStoreException; + + /** + * Release the resource used by the Solver. + */ + void close(); +} diff --git a/hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/solver/api/package-info.java b/hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/solver/api/package-info.java new file mode 100644 index 0000000000..fc8363d851 --- /dev/null +++ b/hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/solver/api/package-info.java @@ -0,0 +1,23 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/** + * API for {@code Solver}. + */ + +package org.apache.hadoop.resourceestimator.solver.api; \ No newline at end of file diff --git a/hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/solver/exceptions/InvalidInputException.java b/hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/solver/exceptions/InvalidInputException.java new file mode 100644 index 0000000000..ff51f5fd53 --- /dev/null +++ b/hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/solver/exceptions/InvalidInputException.java @@ -0,0 +1,34 @@ +/* + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package org.apache.hadoop.resourceestimator.solver.exceptions; + +/** + * Exception thrown the {@code SkylineStore} or the {@code Estimator} tries to + * addHistory or query pipeline job's resource skylines. + */ +public class InvalidInputException extends SolverException { + + private static final long serialVersionUID = -684069387367879218L; + + public InvalidInputException(final String entity, final String reason) { + super(entity + " is " + reason + ", please try again with valid " + entity); + } +} diff --git a/hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/solver/exceptions/InvalidSolverException.java b/hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/solver/exceptions/InvalidSolverException.java new file mode 100644 index 0000000000..9b614b6fa0 --- /dev/null +++ b/hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/solver/exceptions/InvalidSolverException.java @@ -0,0 +1,34 @@ +/* + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package org.apache.hadoop.resourceestimator.solver.exceptions; + +/** + * Exception thrown the @link{SkylineStore} or the {@code Estimator} tries to + * addHistory or query pipeline job's resource skylines. + */ +public class InvalidSolverException extends SolverException { + + private static final long serialVersionUID = -684069387367879218L; + + public InvalidSolverException(final String message) { + super(message); + } +} diff --git a/hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/solver/exceptions/SolverException.java b/hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/solver/exceptions/SolverException.java new file mode 100644 index 0000000000..57507eaac0 --- /dev/null +++ b/hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/solver/exceptions/SolverException.java @@ -0,0 +1,34 @@ +/* + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package org.apache.hadoop.resourceestimator.solver.exceptions; + +/** + * Exception thrown the @link{SkylineStore} or the {@code Estimator} tries to + * addHistory or query pipeline job's resource skylines. + */ +public abstract class SolverException extends Exception { + + private static final long serialVersionUID = -684069387367879218L; + + public SolverException(final String message) { + super(message); + } +} diff --git a/hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/solver/exceptions/package-info.java b/hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/solver/exceptions/package-info.java new file mode 100644 index 0000000000..bd4532428c --- /dev/null +++ b/hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/solver/exceptions/package-info.java @@ -0,0 +1,24 @@ +/* + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +/** + * Exception module. + */ +package org.apache.hadoop.resourceestimator.solver.exceptions; diff --git a/hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/solver/impl/BaseSolver.java b/hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/solver/impl/BaseSolver.java new file mode 100644 index 0000000000..55abb1c908 --- /dev/null +++ b/hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/solver/impl/BaseSolver.java @@ -0,0 +1,94 @@ +/* + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package org.apache.hadoop.resourceestimator.solver.impl; + +import java.util.ArrayList; +import java.util.List; +import java.util.Random; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.resourceestimator.common.config.ResourceEstimatorConfiguration; +import org.apache.hadoop.yarn.api.protocolrecords.ReservationSubmissionRequest; +import org.apache.hadoop.yarn.api.records.ReservationDefinition; +import org.apache.hadoop.yarn.api.records.ReservationId; +import org.apache.hadoop.yarn.api.records.ReservationRequest; +import org.apache.hadoop.yarn.api.records.ReservationRequestInterpreter; +import org.apache.hadoop.yarn.api.records.ReservationRequests; +import org.apache.hadoop.yarn.api.records.Resource; +import org.apache.hadoop.yarn.server.resourcemanager.reservation.RLESparseResourceAllocation; + +/** + * Common functions shared by {@code Solver} (translate predicted resource + * allocation into Hadoop's {@link ReservationSubmissionRequest}. + */ +public abstract class BaseSolver { + /** + * Used to generate {@link ReservationId}. + */ + private static final Random RAND = new Random(); + + /** + * Translate the estimated {@link Resource} requirements of the pipeline to + * Hadoop's {@link ReservationSubmissionRequest}. + * + * @param containerSpec the {@link Resource} to be allocated to each + * container; + * @param containerRequests the predicted {@link Resource} to be allocated to + * the job in each discrete time intervals; + * @param config configuration file for BaseSolver. + * @return {@link ReservationSubmissionRequest} to be submitted to Hadoop to + * make recurring resource reservation for the pipeline. + */ + public final ReservationSubmissionRequest toRecurringRDL( + final Resource containerSpec, + final RLESparseResourceAllocation containerRequests, + final Configuration config) { + final int timeInterval = + config.getInt(ResourceEstimatorConfiguration.TIME_INTERVAL_KEY, 5); + long pipelineSubmissionTime = containerRequests.getEarliestStartTime(); + long pipelineFinishTime = containerRequests.getLatestNonNullTime(); + final long containerMemAlloc = containerSpec.getMemorySize(); + final long jobLen = + (pipelineFinishTime - pipelineSubmissionTime) / timeInterval; + List reservationRequestList = new ArrayList<>(); + for (int i = 0; i < jobLen; i++) { + // container spec, # of containers, concurrency, duration + ReservationRequest reservationRequest = ReservationRequest + .newInstance(containerSpec, (int) ( + containerRequests.getCapacityAtTime(i * timeInterval) + .getMemorySize() / containerMemAlloc), 1, timeInterval); + reservationRequestList.add(reservationRequest); + } + ReservationRequests reservationRequests = ReservationRequests + .newInstance(reservationRequestList, + ReservationRequestInterpreter.R_ALL); + ReservationDefinition reservationDefinition = ReservationDefinition + .newInstance(pipelineSubmissionTime, pipelineFinishTime, + reservationRequests, "LpSolver#toRecurringRDL"); + ReservationId reservationId = + ReservationId.newInstance(RAND.nextLong(), RAND.nextLong()); + ReservationSubmissionRequest reservationSubmissionRequest = + ReservationSubmissionRequest + .newInstance(reservationDefinition, "resourceestimator", + reservationId); + return reservationSubmissionRequest; + } +} diff --git a/hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/solver/impl/LpSolver.java b/hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/solver/impl/LpSolver.java new file mode 100644 index 0000000000..c944d20ae9 --- /dev/null +++ b/hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/solver/impl/LpSolver.java @@ -0,0 +1,340 @@ +/* + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package org.apache.hadoop.resourceestimator.solver.impl; + +import java.math.BigDecimal; +import java.util.List; +import java.util.Map; +import java.util.TreeMap; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.resourceestimator.common.api.RecurrenceId; +import org.apache.hadoop.resourceestimator.common.api.ResourceSkyline; +import org.apache.hadoop.resourceestimator.common.config.ResourceEstimatorConfiguration; +import org.apache.hadoop.resourceestimator.skylinestore.api.PredictionSkylineStore; +import org.apache.hadoop.resourceestimator.skylinestore.exceptions.SkylineStoreException; +import org.apache.hadoop.resourceestimator.solver.api.Solver; +import org.apache.hadoop.resourceestimator.solver.exceptions.SolverException; +import org.apache.hadoop.resourceestimator.solver.preprocess.SolverPreprocessor; +import org.apache.hadoop.yarn.api.records.Resource; +import org.apache.hadoop.yarn.server.resourcemanager.reservation.RLESparseResourceAllocation; +import org.apache.hadoop.yarn.server.resourcemanager.reservation.ReservationInterval; +import org.apache.hadoop.yarn.util.resource.DefaultResourceCalculator; +import org.ojalgo.optimisation.Expression; +import org.ojalgo.optimisation.ExpressionsBasedModel; +import org.ojalgo.optimisation.Optimisation.Result; +import org.ojalgo.optimisation.Variable; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * A LP(Linear Programming) solution to predict recurring pipeline's + * {@link Resource} requirements, and generate Hadoop {@code RDL} requests which + * will be used to make recurring resource reservation. + */ +public class LpSolver extends BaseSolver implements Solver { + private static final Logger LOGGER = LoggerFactory.getLogger(LpSolver.class); + private final SolverPreprocessor preprocessor = new SolverPreprocessor(); + /** + * Controls the balance between over-allocation and under-allocation. + */ + private double alpha; + /** + * Controls the generalization of the solver. + */ + private double beta; + /** + * The minimum number of job runs required to run the solver. + */ + private int minJobRuns; + /** + * The time interval which is used to discretize job execution. + */ + private int timeInterval; + /** + * The PredictionSkylineStore to store the predicted ResourceSkyline for new + * run. + */ + private PredictionSkylineStore predictionSkylineStore; + + @Override public final void init(final Configuration config, + PredictionSkylineStore skylineStore) { + this.alpha = + config.getDouble(ResourceEstimatorConfiguration.SOLVER_ALPHA_KEY, 0.1); + this.beta = + config.getDouble(ResourceEstimatorConfiguration.SOLVER_BETA_KEY, 0.1); + this.minJobRuns = + config.getInt(ResourceEstimatorConfiguration.SOLVER_MIN_JOB_RUN_KEY, 1); + this.timeInterval = + config.getInt(ResourceEstimatorConfiguration.TIME_INTERVAL_KEY, 5); + this.predictionSkylineStore = skylineStore; + } + + /** + * Generate over-allocation constraints. + * + * @param lpModel the LP model. + * @param cJobITimeK actual container allocation for job i in time + * interval k. + * @param oa container over-allocation. + * @param x predicted container allocation. + * @param indexJobITimeK index for job i at time interval k. + * @param timeK index for time interval k. + */ + private void generateOverAllocationConstraints( + final ExpressionsBasedModel lpModel, final double cJobITimeK, + final Variable[] oa, final Variable[] x, final int indexJobITimeK, + final int timeK) { + // oa_job_i_timeK >= x_timeK - cJobITimeK + Expression overAllocExpression = + lpModel.addExpression("over_alloc_" + indexJobITimeK); + overAllocExpression.set(oa[indexJobITimeK], 1); + overAllocExpression.set(x[timeK], -1); + overAllocExpression.lower(-cJobITimeK); // >= + } + + /** + * Generate under-allocation constraints. + * + * @param lpModel the LP model. + * @param cJobITimeK actual container allocation for job i in time + * interval k. + * @param uaPredict absolute container under-allocation. + * @param ua recursive container under-allocation. + * @param x predicted container allocation. + * @param indexJobITimeK index for job i at time interval k. + * @param timeK index for time interval k. + */ + private void generateUnderAllocationConstraints( + final ExpressionsBasedModel lpModel, final double cJobITimeK, + final Variable[] uaPredict, final Variable[] ua, final Variable[] x, + final int indexJobITimeK, final int timeK) { + // uaPredict_job_i_timeK + x_timeK >= cJobITimeK + Expression underAllocPredictExpression = + lpModel.addExpression("under_alloc_predict_" + indexJobITimeK); + underAllocPredictExpression.set(uaPredict[indexJobITimeK], 1); + underAllocPredictExpression.set(x[timeK], 1); + underAllocPredictExpression.lower(cJobITimeK); // >= + if (timeK >= 1) { + /** Recursively calculate container under-allocation. */ + // ua_job_i_timeK >= ua_job_i_time_(k-1) + cJobITimeK - x_timeK + Expression underAllocExpression = + lpModel.addExpression("under_alloc_" + indexJobITimeK); + underAllocExpression.set(ua[indexJobITimeK], 1); + underAllocExpression.set(ua[indexJobITimeK - 1], -1); + underAllocExpression.set(x[timeK], 1); + underAllocExpression.lower(cJobITimeK); // >= + } else { + /** Initial value for container under-allocation. */ + // ua_job_i_time_0 >= cJobI_time_0 - x_time_0 + Expression underAllocExpression = + lpModel.addExpression("under_alloc_" + indexJobITimeK); + underAllocExpression.set(ua[indexJobITimeK], 1); + underAllocExpression.set(x[timeK], 1); + underAllocExpression.lower(cJobITimeK); // >= + } + } + + /** + * Generate solver objective. + * + * @param objective LP solver objective. + * @param numJobs number of history runs of the recurring pipeline. + * @param jobLen (maximum) job lenght of the recurring pipeline. + * @param oa container over-allocation. + * @param ua recursive container under-allocation. + * @param eps regularization parameter. + */ + private void generateObjective(final Expression objective, final int numJobs, + final int jobLen, final Variable[] oa, final Variable[] ua, + final Variable eps) { + int indexJobITimeK; + // sum Over_Allocation + for (int indexJobI = 0; indexJobI < numJobs; indexJobI++) { + for (int timeK = 0; timeK < jobLen; timeK++) { + indexJobITimeK = indexJobI * jobLen + timeK; + objective.set(oa[indexJobITimeK], alpha / numJobs); + } + } + // sum Under_Allocation + int indexJobITimeN; + for (int indexJobI = 0; indexJobI < numJobs; indexJobI++) { + indexJobITimeN = indexJobI * jobLen + jobLen - 1; + objective.set(ua[indexJobITimeN], (1 - alpha) / numJobs); + } + objective.set(eps, beta); + objective.weight(BigDecimal.valueOf(1)); + } + + /** + * Get the job length of recurring pipeline. + * + * @param resourceSkylines the history ResourceSkylines allocated to the + * recurring pipeline. + * @param numJobs number of history runs of the recurring pipeline. + * @return length of (discretized time intervals of) the recurring pipeline. + */ + private int getJobLen(final List resourceSkylines, + final int numJobs) { + int curLen = 0; + int jobLen = 0; + for (int indexJobI = 0; indexJobI < numJobs; indexJobI++) { + curLen = (int) (resourceSkylines.get(indexJobI).getSkylineList() + .getLatestNonNullTime() - resourceSkylines.get(indexJobI) + .getSkylineList().getEarliestStartTime() + timeInterval - 1) + / timeInterval; // for round up + if (jobLen < curLen) { + jobLen = curLen; + } + } + return jobLen; + } + + @Override public final RLESparseResourceAllocation solve( + final Map> jobHistory) + throws SolverException, SkylineStoreException { + // TODO: addHistory timeout support for this function, and ideally we should + // return the confidence + // level associated with the predicted resource. + preprocessor.validate(jobHistory, timeInterval); + final List resourceSkylines = + preprocessor.aggregateSkylines(jobHistory, minJobRuns); + final int numJobs = resourceSkylines.size(); + final int jobLen = getJobLen(resourceSkylines, numJobs); + + /** Create variables. */ + final ExpressionsBasedModel lpModel = new ExpressionsBasedModel(); + + Variable[] oa = new Variable[jobLen * numJobs]; + Variable[] ua = new Variable[jobLen * numJobs]; + Variable[] uaPredict = new Variable[jobLen * numJobs]; + Variable[] x = new Variable[jobLen]; + for (int i = 0; i < jobLen * numJobs; i++) { + oa[i] = new Variable("oa" + i).lower(BigDecimal.valueOf(0)); + ua[i] = new Variable("ua" + i).lower(BigDecimal.valueOf(0)); + uaPredict[i] = new Variable("uaPredict" + i).lower(BigDecimal.valueOf(0)); + } + for (int i = 0; i < jobLen; i++) { + x[i] = new Variable("x").lower(BigDecimal.valueOf(0)); + } + lpModel.addVariables(x); + lpModel.addVariables(oa); + lpModel.addVariables(ua); + lpModel.addVariables(uaPredict); + Variable eps = new Variable("epsilon").lower(BigDecimal.valueOf(0)); + lpModel.addVariable(eps); + + /** Set constraints. */ + int indexJobITimeK = 0; + double cJobI = 0; + double cJobITimeK = 0; + ResourceSkyline resourceSkyline; + int[] containerNums; + // 1. sum(job_i){sum(timeK){1/cJobI * uaPredict_job_i_timeK}} <= numJobs + // * eps + Expression regularizationConstraint = + lpModel.addExpression("regularization"); + regularizationConstraint.set(eps, -numJobs); + regularizationConstraint.upper(BigDecimal.valueOf(0)); // <= 0 + for (int indexJobI = 0; + indexJobI < resourceSkylines.size(); indexJobI++) { + resourceSkyline = resourceSkylines.get(indexJobI); + // the # of containers consumed by job i in discretized time intervals + containerNums = preprocessor + .getDiscreteSkyline(resourceSkyline.getSkylineList(), timeInterval, + resourceSkyline.getContainerSpec().getMemorySize(), jobLen); + // the aggregated # of containers consumed by job i during its lifespan + cJobI = 0; + for (int i = 0; i < containerNums.length; i++) { + cJobI = cJobI + containerNums[i]; + } + for (int timeK = 0; timeK < jobLen; timeK++) { + indexJobITimeK = indexJobI * jobLen + timeK; + // the # of containers consumed by job i in the k-th time interval + cJobITimeK = containerNums[timeK]; + regularizationConstraint + .set(uaPredict[indexJobITimeK], 1 / cJobI); + generateOverAllocationConstraints(lpModel, cJobITimeK, oa, x, + indexJobITimeK, timeK); + generateUnderAllocationConstraints(lpModel, cJobITimeK, uaPredict, + ua, x, indexJobITimeK, timeK); + } + } + + /** Set objective. */ + Expression objective = lpModel.addExpression("objective"); + generateObjective(objective, numJobs, jobLen, oa, ua, eps); + + /** Solve the model. */ + final Result lpResult = lpModel.minimise(); + final TreeMap treeMap = new TreeMap<>(); + RLESparseResourceAllocation result = + new RLESparseResourceAllocation(treeMap, + new DefaultResourceCalculator()); + ReservationInterval riAdd; + Resource containerSpec = resourceSkylines.get(0).getContainerSpec(); + String pipelineId = + ((RecurrenceId) jobHistory.keySet().toArray()[0]).getPipelineId(); + Resource resource; + for (int indexTimeK = 0; indexTimeK < jobLen; indexTimeK++) { + riAdd = new ReservationInterval(indexTimeK * timeInterval, + (indexTimeK + 1) * timeInterval); + resource = Resource.newInstance( + containerSpec.getMemorySize() * (int) lpResult + .doubleValue(indexTimeK), + containerSpec.getVirtualCores() * (int) lpResult + .doubleValue(indexTimeK)); + result.addInterval(riAdd, resource); + LOGGER.debug("time interval: {}, container: {}.", indexTimeK, + lpResult.doubleValue(indexTimeK)); + } + + predictionSkylineStore.addEstimation(pipelineId, result); + + /** + * TODO: 1. We can calculate the estimated error (over-allocation, + * under-allocation) of our prediction which could be used to generate + * confidence level for our prediction; 2. Also, we can modify our model to + * take job input data size (and maybe stage info) into consideration; 3. We + * can also try to generate such conclusion: our prediction under-allocates + * X amount of resources from time 0 to time 100 compared with 95% of + * history runs; 4. We can build framework-specific versions of estimator + * (such as scope/spark/hive, etc.) and provides more specific suggestions. + * For example, we may say: for spark job i, its task size is X GB while the + * container memory allocation is Y GB; as a result, its shuffling stage is + * 20% slower than ideal case due to the disk spilling operations, etc. 5. + * If we have more information of jobs (other than ResourceSkyline), we may + * have such conclusion: job i is 20% slower than 90% of history runs, and + * it is because part of its tasks are running together with job j's tasks. + * In this case, we not only predict the amount of resource needed for job + * i, but also how to place the resource requirements to clusters; 6. We may + * monitor job progress, and dynamically increase/decrease container + * allocations to satisfy job deadline while minimizing the cost; 7. We may + * allow users to specify a budget (say $100 per job run), and optimize the + * resource allocation under the budget constraints. 8. ... + */ + return result; + } + + @Override public final void close() { + // TODO: currently place holder + } +} diff --git a/hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/solver/impl/package-info.java b/hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/solver/impl/package-info.java new file mode 100644 index 0000000000..3344f60ae0 --- /dev/null +++ b/hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/solver/impl/package-info.java @@ -0,0 +1,23 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/** + * Implementation for {@code Solver}. + */ + +package org.apache.hadoop.resourceestimator.solver.impl; \ No newline at end of file diff --git a/hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/solver/preprocess/SolverPreprocessor.java b/hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/solver/preprocess/SolverPreprocessor.java new file mode 100644 index 0000000000..af613346e4 --- /dev/null +++ b/hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/solver/preprocess/SolverPreprocessor.java @@ -0,0 +1,219 @@ +/* + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package org.apache.hadoop.resourceestimator.solver.preprocess; + +import static java.lang.Math.toIntExact; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.Map; +import java.util.TreeMap; + +import org.apache.hadoop.resourceestimator.common.api.RecurrenceId; +import org.apache.hadoop.resourceestimator.common.api.ResourceSkyline; +import org.apache.hadoop.resourceestimator.solver.api.Solver; +import org.apache.hadoop.resourceestimator.solver.exceptions.InvalidInputException; +import org.apache.hadoop.yarn.api.records.Resource; +import org.apache.hadoop.yarn.server.resourcemanager.reservation.RLESparseResourceAllocation; +import org.apache.hadoop.yarn.server.resourcemanager.reservation.ReservationInterval; +import org.apache.hadoop.yarn.util.resource.DefaultResourceCalculator; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Common preprocessing functions for {@link Solver}. + */ +public class SolverPreprocessor { + private static final Logger LOGGER = + LoggerFactory.getLogger(SolverPreprocessor.class); + + /** + * Check if Solver's input parameters are valid. + * + * @param jobHistory the history {@link ResourceSkyline}s of the recurring + * pipeline job. + * @param timeInterval the time interval which is used to discretize the + * history {@link ResourceSkyline}s. + * @throws InvalidInputException if: (1) jobHistory is null; + * (2) jobHistory is empty; (3) timeout is non-positive; + * (4) timeInterval is non-positive; + */ + public final void validate( + final Map> jobHistory, + final int timeInterval) throws InvalidInputException { + if ((jobHistory == null) || (jobHistory.size() == 0)) { + LOGGER.error( + "Job resource skyline history is invalid, please try again with" + + " valid resource skyline history."); + throw new InvalidInputException("Job ResourceSkyline history", "invalid"); + } + + if (timeInterval <= 0) { + LOGGER.error( + "Solver timeInterval {} is invalid, please specify a positive value.", + timeInterval); + throw new InvalidInputException("Solver timeInterval", "non-positive"); + } + } + + /** + * Return the multi-dimension resource vector consumed by the job at specified + * time. + * + * @param skyList the list of {@link Resource}s used by the job. + * @param index the discretized time index. + * @param containerMemAlloc the multi-dimension resource vector allocated to + * one container. + * @return the multi-dimension resource vector consumed by the job. + */ + public final long getResourceVector(final RLESparseResourceAllocation skyList, + final int index, final long containerMemAlloc) { + return skyList.getCapacityAtTime(index).getMemorySize() / containerMemAlloc; + } + + /** + * Discretize job's lifespan into intervals, and return the number of + * containers used by the job within each interval. + *

Note that here we assume all containers allocated to the job have the + * same {@link Resource}. This is due to the limit of + * {@link RLESparseResourceAllocation}. + * + * @param skyList the list of {@link Resource}s used by the job. + * @param timeInterval the time interval used to discretize the job's + * lifespan. + * @param containerMemAlloc the amount of memory allocated to each container. + * @param jobLen the duration of the job. + * @return the number of containers allocated to the job within discretized + * time intervals. + */ + public final int[] getDiscreteSkyline( + final RLESparseResourceAllocation skyList, final int timeInterval, + final long containerMemAlloc, final int jobLen) { + long jobLifeSpan = + skyList.getLatestNonNullTime() - skyList.getEarliestStartTime(); + int[] result = new int[jobLen]; + Arrays.fill(result, 0); + + int index = 0; + long numContainerAt = 0; + for (int i = 0; i < jobLifeSpan; i++) { + index = (int) Math.floor((double) i / timeInterval); + numContainerAt = getResourceVector(skyList, i, containerMemAlloc); + if (result[index] < numContainerAt) { + result[index] = (int) numContainerAt; + } + } + return result; + } + + /** + * Merge different jobs' resource skylines into one within the same pipeline. + * + * @param resourceSkylines different jobs' resource skylines within the same + * pipeline. + * @return an aggregated resource skyline for the pipeline. + */ + public final ResourceSkyline mergeSkyline( + final List resourceSkylines) { + // TODO: + // rewrite this function with shift and merge once YARN-5328 is committed + /** First, getHistory the pipeline submission time. */ + long pipelineSubmission = Long.MAX_VALUE; + for (int i = 0; i < resourceSkylines.size(); i++) { + long jobSubmission = resourceSkylines.get(i).getJobSubmissionTime(); + if (pipelineSubmission > jobSubmission) { + pipelineSubmission = jobSubmission; + } + } + final TreeMap resourceOverTime = new TreeMap<>(); + final RLESparseResourceAllocation skylineListAgg = + new RLESparseResourceAllocation(resourceOverTime, + new DefaultResourceCalculator()); + /** + * Second, adjust different jobs' ResourceSkyline starting time based on + * pipeline submission time, and merge them into one ResourceSkyline. + */ + for (int i = 0; i < resourceSkylines.size(); i++) { + long jobSubmission = resourceSkylines.get(i).getJobSubmissionTime(); + long diff = (jobSubmission - pipelineSubmission) / 1000; + RLESparseResourceAllocation tmp = + resourceSkylines.get(i).getSkylineList(); + Object[] timePoints = tmp.getCumulative().keySet().toArray(); + for (int j = 0; j < timePoints.length - 2; j++) { + ReservationInterval riAdd = + new ReservationInterval(toIntExact((long) timePoints[j]) + diff, + toIntExact((long) timePoints[j + 1] + diff)); + skylineListAgg.addInterval(riAdd, + tmp.getCapacityAtTime(toIntExact((long) timePoints[j]))); + } + } + ResourceSkyline skylineAgg = + new ResourceSkyline(resourceSkylines.get(0).getJobId(), + resourceSkylines.get(0).getJobInputDataSize(), + resourceSkylines.get(0).getJobSubmissionTime(), + resourceSkylines.get(0).getJobFinishTime(), + resourceSkylines.get(0).getContainerSpec(), skylineListAgg); + + return skylineAgg; + } + + /** + * Aggregate all job's {@link ResourceSkyline}s in the one run of recurring + * pipeline, and return the aggregated {@link ResourceSkyline}s in different + * runs. + * + * @param jobHistory the history {@link ResourceSkyline} of the recurring + * pipeline job. + * @param minJobRuns the minimum number of job runs required to run the + * solver. + * @return the aggregated {@link ResourceSkyline}s in different runs. + * @throws InvalidInputException if: (1) job submission time parsing fails; + * (2) jobHistory has less job runs than the minimum requirement; + */ + public final List aggregateSkylines( + final Map> jobHistory, + final int minJobRuns) throws InvalidInputException { + List resourceSkylines = new ArrayList(); + for (Map.Entry> entry : jobHistory + .entrySet()) { + // TODO: identify different jobs within the same pipeline + // right now, we do prediction at the granularity of pipeline, i.e., we + // will merge the + // resource skylines of jobs within the same pipeline into one aggregated + // resource skyline + ResourceSkyline skylineAgg = null; + skylineAgg = mergeSkyline(entry.getValue()); + resourceSkylines.add(skylineAgg); + } + int numJobs = resourceSkylines.size(); + if (numJobs < minJobRuns) { + LOGGER.error( + "Solver requires job resource skyline history for at least {} runs," + + " but it only receives history info for {} runs.", + minJobRuns, numJobs); + throw new InvalidInputException("Job ResourceSkyline history", + "containing less job runs" + " than " + minJobRuns); + } + + return resourceSkylines; + } +} diff --git a/hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/solver/preprocess/package-info.java b/hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/solver/preprocess/package-info.java new file mode 100644 index 0000000000..b58cef96de --- /dev/null +++ b/hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/solver/preprocess/package-info.java @@ -0,0 +1,23 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/** + * Preprocessor for {@code Solver}. + */ + +package org.apache.hadoop.resourceestimator.solver.preprocess; \ No newline at end of file diff --git a/hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/translator/api/JobMetaData.java b/hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/translator/api/JobMetaData.java new file mode 100644 index 0000000000..2bfab3ebf8 --- /dev/null +++ b/hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/translator/api/JobMetaData.java @@ -0,0 +1,163 @@ +/* + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package org.apache.hadoop.resourceestimator.translator.api; + +import java.util.HashMap; +import java.util.Map; +import java.util.TreeMap; + +import org.apache.hadoop.resourceestimator.common.api.RecurrenceId; +import org.apache.hadoop.resourceestimator.common.api.ResourceSkyline; +import org.apache.hadoop.yarn.api.records.Resource; +import org.apache.hadoop.yarn.server.resourcemanager.reservation.RLESparseResourceAllocation; +import org.apache.hadoop.yarn.server.resourcemanager.reservation.ReservationInterval; +import org.apache.hadoop.yarn.util.resource.DefaultResourceCalculator; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Job metadata collected when parsing the log file. + */ +public class JobMetaData { + // containerId, releaseTime + private static final Logger LOGGER = + LoggerFactory.getLogger(JobMetaData.class); + private final ResourceSkyline resourceSkyline = new ResourceSkyline(); + private final Map rawStart = new HashMap(); + // containerId, startTime + private final Map rawEnd = new HashMap(); + private RecurrenceId recurrenceId; + + /** + * Constructor. + * + * @param jobSubmissionTimeConfig job submission time. + */ + public JobMetaData(final long jobSubmissionTimeConfig) { + resourceSkyline.setJobSubmissionTime(jobSubmissionTimeConfig); + } + + /** + * Set job finish time. + * + * @param jobFinishTimeConfig job finish time. + * @return the reference to current {@link JobMetaData}. + */ + public final JobMetaData setJobFinishTime(final long jobFinishTimeConfig) { + resourceSkyline.setJobFinishTime(jobFinishTimeConfig); + return this; + } + + /** + * Add container launch time. + * + * @param containerId id of the container. + * @param time container launch time. + * @return the reference to current {@link JobMetaData}. + */ + public final JobMetaData setContainerStart(final String containerId, + final long time) { + if (rawStart.put(containerId, time) != null) { + LOGGER.warn("find duplicate container launch time for {}, so we replace" + + " it with {}.", containerId, time); + } + return this; + } + + /** + * Add container release time. + * + * @param containerId id of the container. + * @param time container release time. + * @return the reference to current {@link JobMetaData}. + */ + public final JobMetaData setContainerEnd(final String containerId, + final long time) { + if (rawEnd.put(containerId, time) != null) { + LOGGER.warn("find duplicate container release time for {}, so we replace" + + " it with {}.", containerId, time); + } + return this; + } + + /** + * Get {@link RecurrenceId}. + * + * @return {@link RecurrenceId}. + */ + public final RecurrenceId getRecurrenceId() { + return recurrenceId; + } + + /** + * Set {@link RecurrenceId}. + * + * @param recurrenceIdConfig the {@link RecurrenceId}. + * @return the reference to current {@link JobMetaData}. + */ + public final JobMetaData setRecurrenceId( + final RecurrenceId recurrenceIdConfig) { + this.recurrenceId = recurrenceIdConfig; + return this; + } + + /** + * Get {@link ResourceSkyline}. + * + * @return {@link ResourceSkyline}. + */ + public final ResourceSkyline getResourceSkyline() { + return resourceSkyline; + } + + /** + * Normalized container launch/release time, and generate the + * {@link ResourceSkyline}. + */ + public final void createSkyline() { + final long jobSubmissionTime = resourceSkyline.getJobSubmissionTime(); + Resource containerSpec = resourceSkyline.getContainerSpec(); + final TreeMap resourceOverTime = new TreeMap<>(); + final RLESparseResourceAllocation skylineList = + new RLESparseResourceAllocation(resourceOverTime, + new DefaultResourceCalculator()); + resourceSkyline.setSkylineList(skylineList); + if (containerSpec == null) { + // if RmParser fails to extract container resource spec from logs, we will + // statically set + // it to be <1core, 1GB> + containerSpec = Resource.newInstance(1024, 1); + } + resourceSkyline.setContainerSpec(containerSpec); + for (final Map.Entry entry : rawStart.entrySet()) { + final long timeStart = entry.getValue(); + final Long timeEnd = rawEnd.get(entry.getKey()); + if (timeEnd == null) { + LOGGER.warn("container release time not found for {}.", entry.getKey()); + } else { + final ReservationInterval riAdd = + new ReservationInterval((timeStart - jobSubmissionTime) / 1000, + (timeEnd - jobSubmissionTime) / 1000); + resourceSkyline.getSkylineList().addInterval(riAdd, containerSpec); + } + } + } +} diff --git a/hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/translator/api/LogParser.java b/hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/translator/api/LogParser.java new file mode 100644 index 0000000000..b7efe5adf9 --- /dev/null +++ b/hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/translator/api/LogParser.java @@ -0,0 +1,65 @@ +/* + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package org.apache.hadoop.resourceestimator.translator.api; + +import java.io.IOException; +import java.io.InputStream; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.resourceestimator.common.exception.ResourceEstimatorException; +import org.apache.hadoop.resourceestimator.skylinestore.api.HistorySkylineStore; +import org.apache.hadoop.resourceestimator.skylinestore.exceptions.SkylineStoreException; + +/** + * LogParser iterates over a stream of logs, uses {@link SingleLineParser} to + * parse each line, and adds extracted {@code ResourceSkyline}s to the + * {@code SkylineStore}. + */ +public interface LogParser extends AutoCloseable { + + /** + * Initializing the LogParser, including loading solver parameters from + * configuration file. + * + * @param config {@link Configuration} for the LogParser. + * @param skylineStore the {@link HistorySkylineStore} which stores recurring + * pipeline's {@code + * ResourceSkyline}s. + * @throws ResourceEstimatorException if initialization of a + * {@code SingleLineParser} fails. + */ + void init(Configuration config, HistorySkylineStore skylineStore) + throws ResourceEstimatorException; + + /** + * Parses each line in the log stream, and adds extracted + * {@code ResourceSkyline}s to the {@code + * SkylineStore}. + * + * @param logs the stream of input logs. + * @throws SkylineStoreException if it fails to addHistory extracted + * {@code ResourceSkyline}s to the {@code SkylineStore}. + * @throws IOException if it fails to read from the {@link InputStream}. + */ + void parseStream(InputStream logs) throws SkylineStoreException, IOException; + + @Override void close(); +} diff --git a/hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/translator/api/SingleLineParser.java b/hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/translator/api/SingleLineParser.java new file mode 100644 index 0000000000..b9607fe2c3 --- /dev/null +++ b/hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/translator/api/SingleLineParser.java @@ -0,0 +1,52 @@ +/* + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package org.apache.hadoop.resourceestimator.translator.api; + +import java.text.ParseException; +import java.util.List; +import java.util.Map; + +import org.apache.hadoop.resourceestimator.common.api.RecurrenceId; +import org.apache.hadoop.resourceestimator.common.api.ResourceSkyline; +import org.apache.hadoop.resourceestimator.translator.exceptions.DataFieldNotFoundException; + +/** + * SingleLineParser parses one line in the log file, extracts the + * {@link ResourceSkyline}s and stores them. + */ +public interface SingleLineParser { + /** + * Parse one line in the log file, extract the {@link ResourceSkyline}s and + * store them. + * + * @param logLine one line in the log file. + * @param jobMetas the job metadata collected during parsing. + * @param skylineRecords the valid {@link ResourceSkyline}s extracted from the + * log. + * @throws DataFieldNotFoundException if certain data fields are not found in + * the log. + * @throws ParseException if it fails to convert date string to + * unix timestamp successfully. + */ + void parseLine(String logLine, Map jobMetas, + Map> skylineRecords) + throws DataFieldNotFoundException, ParseException; +} diff --git a/hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/translator/api/package-info.java b/hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/translator/api/package-info.java new file mode 100644 index 0000000000..a1b79c0b06 --- /dev/null +++ b/hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/translator/api/package-info.java @@ -0,0 +1,23 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/** + * API for {@code Translator}. + */ + +package org.apache.hadoop.resourceestimator.translator.api; \ No newline at end of file diff --git a/hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/translator/exceptions/DataFieldNotFoundException.java b/hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/translator/exceptions/DataFieldNotFoundException.java new file mode 100644 index 0000000000..6eae614b9b --- /dev/null +++ b/hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/translator/exceptions/DataFieldNotFoundException.java @@ -0,0 +1,32 @@ +/* + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package org.apache.hadoop.resourceestimator.translator.exceptions; + +/** + * Exception thrown when job attributes are not found. + */ +public class DataFieldNotFoundException extends Exception { + private static final long serialVersionUID = -684069387367879218L; + + public DataFieldNotFoundException(final String log) { + super("Fail to extract data fields properly from " + log); + } +} diff --git a/hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/translator/exceptions/package-info.java b/hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/translator/exceptions/package-info.java new file mode 100644 index 0000000000..104388ddfd --- /dev/null +++ b/hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/translator/exceptions/package-info.java @@ -0,0 +1,23 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/** + * Exception for {@code Translator}. + */ + +package org.apache.hadoop.resourceestimator.translator.exceptions; \ No newline at end of file diff --git a/hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/translator/impl/BaseLogParser.java b/hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/translator/impl/BaseLogParser.java new file mode 100644 index 0000000000..50d911f988 --- /dev/null +++ b/hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/translator/impl/BaseLogParser.java @@ -0,0 +1,125 @@ +/* + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package org.apache.hadoop.resourceestimator.translator.impl; + +import java.io.BufferedReader; +import java.io.IOException; +import java.io.InputStream; +import java.io.InputStreamReader; +import java.text.ParseException; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.resourceestimator.common.api.RecurrenceId; +import org.apache.hadoop.resourceestimator.common.api.ResourceSkyline; +import org.apache.hadoop.resourceestimator.common.config.ResourceEstimatorConfiguration; +import org.apache.hadoop.resourceestimator.common.config.ResourceEstimatorUtil; +import org.apache.hadoop.resourceestimator.common.exception.ResourceEstimatorException; +import org.apache.hadoop.resourceestimator.skylinestore.api.HistorySkylineStore; +import org.apache.hadoop.resourceestimator.skylinestore.exceptions.SkylineStoreException; +import org.apache.hadoop.resourceestimator.translator.api.JobMetaData; +import org.apache.hadoop.resourceestimator.translator.api.LogParser; +import org.apache.hadoop.resourceestimator.translator.api.SingleLineParser; +import org.apache.hadoop.resourceestimator.translator.exceptions.DataFieldNotFoundException; +import org.apache.hadoop.resourceestimator.translator.validator.ParserValidator; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Base class to implement {@link LogParser}. It wraps a + * {@link SingleLineParser} from the {@link Configuration} to parse a log + * dir/file. + */ +public class BaseLogParser implements LogParser { + private static final Logger LOGGER = + LoggerFactory.getLogger(BaseLogParser.class); + private static final ParserValidator INPUT_VALIDATOR = new ParserValidator(); + private SingleLineParser singleLineParser; + private HistorySkylineStore historySkylineStore; + + @Override public void init(Configuration config, + HistorySkylineStore skylineStore) throws ResourceEstimatorException { + singleLineParser = ResourceEstimatorUtil.createProviderInstance(config, + ResourceEstimatorConfiguration.TRANSLATOR_LINE_PARSER, + ResourceEstimatorConfiguration.DEFAULT_TRANSLATOR_LINE_PARSER, + SingleLineParser.class); + this.historySkylineStore = skylineStore; + } + + /** + * Add job's {@link ResourceSkyline}s to the {@link HistorySkylineStore}. + * + * @param skylineRecords the {@link Map} which records the completed recurring + * pipeline's {@link ResourceSkyline}s. + * @throws SkylineStoreException if it failes to addHistory job's + * {@link ResourceSkyline}s to the {@link HistorySkylineStore}. + */ + private void addToSkylineStore( + final Map> skylineRecords) + throws SkylineStoreException { + for (final Map.Entry> entry : + skylineRecords.entrySet()) { + historySkylineStore.addHistory(entry.getKey(), entry.getValue()); + } + } + + public void parseLine(final String logLine, + final Map jobMetas, + final Map> skylineRecords) + throws DataFieldNotFoundException, ParseException { + singleLineParser.parseLine(logLine, jobMetas, skylineRecords); + } + + @Override public final void parseStream(final InputStream logs) + throws SkylineStoreException, IOException { + if (!INPUT_VALIDATOR.validate(logs)) { + LOGGER.error("Input validation fails, please specify with" + + " valid input parameters."); + return; + } + final Map> skylineRecords = + new HashMap<>(); + final Map jobMetas = + new HashMap(); + final BufferedReader bf = new BufferedReader(new InputStreamReader(logs)); + String line = null; + while ((line = bf.readLine()) != null) { + try { + parseLine(line, jobMetas, skylineRecords); + } catch (DataFieldNotFoundException e) { + LOGGER.debug("Data field not found", e); + } catch (ParseException e) { + LOGGER.debug("Date conversion error", e); + } + } + + addToSkylineStore(skylineRecords); + } + + /** + * Release the resource used by the ParserUtil. + */ + @Override public final void close() { + historySkylineStore = null; + } +} diff --git a/hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/translator/impl/LogParserUtil.java b/hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/translator/impl/LogParserUtil.java new file mode 100644 index 0000000000..35da799c60 --- /dev/null +++ b/hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/translator/impl/LogParserUtil.java @@ -0,0 +1,97 @@ +/* + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package org.apache.hadoop.resourceestimator.translator.impl; + +import java.io.FileInputStream; +import java.io.IOException; +import java.io.InputStream; +import java.text.DateFormat; +import java.text.ParseException; +import java.text.SimpleDateFormat; + +import org.apache.hadoop.resourceestimator.common.exception.ResourceEstimatorException; +import org.apache.hadoop.resourceestimator.skylinestore.api.SkylineStore; +import org.apache.hadoop.resourceestimator.skylinestore.exceptions.SkylineStoreException; +import org.apache.hadoop.resourceestimator.translator.api.LogParser; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Common utility functions for {@link LogParser}. + */ +public class LogParserUtil { + private static final Logger LOGGER = + LoggerFactory.getLogger(LogParserUtil.class); + private LogParser logParser; + private DateFormat format = new SimpleDateFormat("MM/dd/yyyy HH:mm:ss"); + + /** + * Set the {@link LogParser} to use. + * + * @param logParser the {@link LogParser} to use. + */ + public void setLogParser(final LogParser logParser) { + this.logParser = logParser; + } + + /** + * Set date format for the {@link LogParser}. + * + * @param datePattern the date pattern in the log. + */ + public void setDateFormat(final String datePattern) { + this.format = new SimpleDateFormat(datePattern); + } + + /** + * Converts String date to unix timestamp. Note that we assume the time in the + * logs has the same time zone with the machine which runs the + * {@link RmSingleLineParser}. + * + * @param date The String date. + * @return Unix time stamp. + * @throws ParseException if data conversion from String to unix timestamp + * fails. + */ + public long stringToUnixTimestamp(final String date) throws ParseException { + return format.parse(date).getTime(); + } + + /** + * Parse the log file/directory. + * + * @param logFile the file/directory of the log. + * @throws SkylineStoreException if fails to addHistory to + * {@link SkylineStore}. + * @throws IOException if fails to parse the log. + * @throws ResourceEstimatorException if the {@link LogParser} + * is not initialized. + */ + public final void parseLog(final String logFile) + throws SkylineStoreException, IOException, ResourceEstimatorException { + if (logParser == null) { + throw new ResourceEstimatorException("The log parser is not initialized," + + " please try again after initializing."); + } + InputStream inputStream = new FileInputStream(logFile); + logParser.parseStream(inputStream); + } +} diff --git a/hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/translator/impl/NativeSingleLineParser.java b/hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/translator/impl/NativeSingleLineParser.java new file mode 100644 index 0000000000..83557d32fe --- /dev/null +++ b/hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/translator/impl/NativeSingleLineParser.java @@ -0,0 +1,120 @@ +/* + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package org.apache.hadoop.resourceestimator.translator.impl; + +import java.text.ParseException; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.TreeMap; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.resourceestimator.common.api.RecurrenceId; +import org.apache.hadoop.resourceestimator.common.api.ResourceSkyline; +import org.apache.hadoop.resourceestimator.common.config.ResourceEstimatorConfiguration; +import org.apache.hadoop.resourceestimator.translator.api.JobMetaData; +import org.apache.hadoop.resourceestimator.translator.api.SingleLineParser; +import org.apache.hadoop.resourceestimator.translator.exceptions.DataFieldNotFoundException; +import org.apache.hadoop.yarn.api.records.Resource; +import org.apache.hadoop.yarn.server.resourcemanager.reservation.RLESparseResourceAllocation; +import org.apache.hadoop.yarn.server.resourcemanager.reservation.ReservationInterval; +import org.apache.hadoop.yarn.util.resource.DefaultResourceCalculator; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * This sample parser will parse the sample log and extract the resource + * skyline. + *

The expected log format is: NormalizedJobName NumInstances SubmitTime + * StartTime EndTime JobInstanceName memUsage coreUsage + */ +public class NativeSingleLineParser implements SingleLineParser { + private static final Logger LOGGER = + LoggerFactory.getLogger(NativeSingleLineParser.class); + + /** + * Aggregates different jobs' {@link ResourceSkyline}s within the same + * pipeline together. + * + * @param resourceSkyline newly extracted {@link ResourceSkyline}. + * @param recurrenceId the {@link RecurrenceId} which the resourceSkyline + * belongs to. + * @param skylineRecords a {@link Map} which stores the + * {@link ResourceSkyline}s for all pipelines during this parsing. + */ + private void aggregateSkyline(final ResourceSkyline resourceSkyline, + final RecurrenceId recurrenceId, + final Map> skylineRecords) { + List resourceSkylines = skylineRecords.get(recurrenceId); + if (resourceSkylines == null) { + resourceSkylines = new ArrayList(); + skylineRecords.put(recurrenceId, resourceSkylines); + } + resourceSkylines.add(resourceSkyline); + } + + @Override public void parseLine(String logLine, + Map jobMetas, + Map> skylineRecords) + throws DataFieldNotFoundException, ParseException { + Configuration config = new Configuration(); + config.addResource(new org.apache.hadoop.fs.Path( + ResourceEstimatorConfiguration.CONFIG_FILE)); + int timeInterval = + config.getInt(ResourceEstimatorConfiguration.TIME_INTERVAL_KEY, 5); + // note that for native log, we assume each container is allocated <1 core, + // 1GB RAM> + long containerMemAlloc = 1024; + int containerCpuAlloc = 1; + String[] splitString = logLine.split("\\s+"); + String pipelineId = splitString[0]; + String jobId = splitString[5]; + String[] skylineUnits = splitString[7].split("\\|"); + + JobMetaData appMeta = new JobMetaData(0); + RecurrenceId recurrenceId = new RecurrenceId(pipelineId, jobId); + appMeta.setRecurrenceId(recurrenceId); + Resource containerAlloc; + int numContainers; + ResourceSkyline resourceSkyline = appMeta.getResourceSkyline(); + final TreeMap resourceOverTime = new TreeMap<>(); + final RLESparseResourceAllocation skylineList = + new RLESparseResourceAllocation(resourceOverTime, + new DefaultResourceCalculator()); + resourceSkyline.setSkylineList(skylineList); + for (String elem : skylineUnits) { + numContainers = Integer.parseInt(elem.split("\\:")[0]); + containerAlloc = Resource.newInstance(containerMemAlloc * numContainers, + containerCpuAlloc * numContainers); + final ReservationInterval riAdd = + new ReservationInterval(Long.parseLong(elem.split("\\:")[1]), + Long.parseLong(elem.split("\\:")[1]) + timeInterval); + resourceSkyline.getSkylineList().addInterval(riAdd, containerAlloc); + } + resourceSkyline.setContainerSpec( + Resource.newInstance(containerMemAlloc, containerCpuAlloc)); + appMeta.setJobFinishTime( + appMeta.getResourceSkyline().getSkylineList().getLatestNonNullTime()); + resourceSkyline.setJobInputDataSize(0); + resourceSkyline.setJobId(jobId); + aggregateSkyline(resourceSkyline, recurrenceId, skylineRecords); + } +} diff --git a/hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/translator/impl/RmSingleLineParser.java b/hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/translator/impl/RmSingleLineParser.java new file mode 100644 index 0000000000..19049348a8 --- /dev/null +++ b/hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/translator/impl/RmSingleLineParser.java @@ -0,0 +1,203 @@ +/* + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package org.apache.hadoop.resourceestimator.translator.impl; + +import java.text.ParseException; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.regex.Matcher; +import java.util.regex.Pattern; + +import org.apache.hadoop.resourceestimator.common.api.RecurrenceId; +import org.apache.hadoop.resourceestimator.common.api.ResourceSkyline; +import org.apache.hadoop.resourceestimator.translator.api.JobMetaData; +import org.apache.hadoop.resourceestimator.translator.api.SingleLineParser; +import org.apache.hadoop.resourceestimator.translator.exceptions.DataFieldNotFoundException; +import org.apache.hadoop.yarn.api.records.Resource; + +/** + * {@link SingleLineParser} for Hadoop Resource Manager logs. + */ +public class RmSingleLineParser implements SingleLineParser { + private static final LogParserUtil PARSERUTIL = new LogParserUtil(); + private static final Pattern FILTER_PATTERN = Pattern.compile( + "(Submit Application Request|AM Allocated Container|" + + "AM Released Container|finalState=FAILED|" + + "ApplicationSummary|, Resource:)"); + private static final Pattern SUBMISSION_PATTERN = + Pattern.compile("APPID=(\\w+)"); + private static final Pattern FAIL_PATTERN = + Pattern.compile("appattempt_(\\d+_\\d+)_\\d+"); + private static final Pattern FINISH_PATTERN = + Pattern.compile("appId=(\\w+).*?name=(\\w+)\\-(\\w+)"); + private static final Pattern CONTAINER_EVENT_PATTERN = + Pattern.compile("APPID=(\\w+).*?CONTAINERID=(\\w+)"); + private static final Pattern CONTAINER_SPEC_PATTERN = Pattern.compile( + "(container_[^_]+|appattempt)_(\\d+_\\d+).*?memory:(\\d+)," + + "\\svCores:(\\d+)"); + + /** + * Aggregates different jobs' {@link ResourceSkyline}s within the same + * pipeline together. + * + * @param resourceSkyline newly extracted {@link ResourceSkyline}. + * @param recurrenceId the {@link RecurrenceId} which the resourceSkyline + * belongs to. + * @param skylineRecords a {@link Map} which stores the + * {@link ResourceSkyline}s for all pipelines during this parsing. + */ + private void aggregateSkyline(final ResourceSkyline resourceSkyline, + final RecurrenceId recurrenceId, + final Map> skylineRecords) { + List resourceSkylines = skylineRecords.get(recurrenceId); + if (resourceSkylines == null) { + resourceSkylines = new ArrayList(); + skylineRecords.put(recurrenceId, resourceSkylines); + } + resourceSkylines.add(resourceSkyline); + } + + @Override public final void parseLine(final String logLine, + final Map jobMetas, + final Map> skylineRecords) + throws DataFieldNotFoundException, ParseException { + final String[] splits = logLine.split(",", 5); // Limit the max number of 5 + // splits + if (splits.length < 5) { + return; + } + final Matcher jobEventMatcher = FILTER_PATTERN.matcher(splits[4]); // search + // only + // the + // tail + if (!jobEventMatcher.find()) { // this line of log does not contain targeted + // events + return; + } + + // now we have the match, let's do some parsing + final long date = PARSERUTIL.stringToUnixTimestamp(splits[1]); + final String tail = splits[4].split("\\s+", 4)[3]; // use the tail of the + // tail only + switch (jobEventMatcher.group(1)) { + case "Submit Application Request": { + /** Submit job. */ + final Matcher appIdMatch = SUBMISSION_PATTERN.matcher(tail); + if (appIdMatch.find()) { + final String appId = appIdMatch.group(1); + jobMetas.put(appId, new JobMetaData(date)); + } else { + throw new DataFieldNotFoundException(tail); + } + break; + } + case "AM Allocated Container": { + /** Allocate container. */ + final Matcher containerEventMatcher = + CONTAINER_EVENT_PATTERN.matcher(tail); + if (containerEventMatcher.find()) { + final String appId = containerEventMatcher.group(1); + final String containerId = containerEventMatcher.group(2); + final JobMetaData appMeta = jobMetas.get(appId); + if (appMeta != null) { + appMeta.setContainerStart(containerId, date); + } + } else { + throw new DataFieldNotFoundException(tail); + } + break; + } + case ", Resource:": { + final Matcher containerSpecMatcher = CONTAINER_SPEC_PATTERN.matcher(tail); + if (containerSpecMatcher.find()) { + final String appId = "application_" + containerSpecMatcher.group(2); + final JobMetaData appMeta = jobMetas.get(appId); + if (appMeta != null) { + final long memAlloc = Long.parseLong(containerSpecMatcher.group(3)); + final int cpuAlloc = Integer.parseInt(containerSpecMatcher.group(4)); + final Resource containerAlloc = + Resource.newInstance(memAlloc, cpuAlloc); + appMeta.getResourceSkyline().setContainerSpec(containerAlloc); + } + } else { + throw new DataFieldNotFoundException(tail); + } + break; + } + case "AM Released Container": { + final Matcher containerEventMatcher = + CONTAINER_EVENT_PATTERN.matcher(tail); + if (containerEventMatcher.find()) { + final String appId = containerEventMatcher.group(1); + final JobMetaData appMeta = jobMetas.get(appId); + if (appMeta != null) { + final String containerId = containerEventMatcher.group(2); + appMeta.setContainerEnd(containerId, date); + } + } else { + throw new DataFieldNotFoundException(tail); + } + break; + } + case "finalState=FAILED": { + /** In case of appAttempt failed: discard previous records. */ + final Matcher failMatcher = FAIL_PATTERN.matcher(tail); + if (failMatcher.find()) { + final String appId = "application_" + failMatcher.group(1); + if (jobMetas.containsKey(appId)) { + jobMetas.put(appId, new JobMetaData(date)); + } + } else { + throw new DataFieldNotFoundException(tail); + } + break; + } + case "ApplicationSummary": { + /** Finish a job. */ + final Matcher finishMatcher = FINISH_PATTERN.matcher(tail); + if (finishMatcher.find()) { + final String appId = finishMatcher.group(1); + final String pipelineId = finishMatcher.group(2); + final String runId = finishMatcher.group(3); + final RecurrenceId recurrenceId = new RecurrenceId(pipelineId, runId); + final JobMetaData appMeta = jobMetas.remove(appId); + if (appMeta != null) { + appMeta.setRecurrenceId(recurrenceId).setJobFinishTime(date) + .getResourceSkyline().setJobInputDataSize(0); // TODO: need to + // read job input + // data size from + // logs + appMeta.createSkyline(); + final ResourceSkyline resourceSkyline = appMeta.getResourceSkyline(); + resourceSkyline.setJobId(appId); + aggregateSkyline(resourceSkyline, recurrenceId, skylineRecords); + } + } else { + throw new DataFieldNotFoundException(tail); + } + break; + } + default: + break; + } + } +} diff --git a/hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/translator/impl/package-info.java b/hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/translator/impl/package-info.java new file mode 100644 index 0000000000..13b5138a26 --- /dev/null +++ b/hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/translator/impl/package-info.java @@ -0,0 +1,23 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/** + * Implementation for {@code Translator}. + */ + +package org.apache.hadoop.resourceestimator.translator.impl; \ No newline at end of file diff --git a/hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/translator/validator/ParserValidator.java b/hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/translator/validator/ParserValidator.java new file mode 100644 index 0000000000..ebf50ba8bc --- /dev/null +++ b/hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/translator/validator/ParserValidator.java @@ -0,0 +1,41 @@ +/* + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package org.apache.hadoop.resourceestimator.translator.validator; + +import java.io.InputStream; + +import org.apache.hadoop.resourceestimator.translator.api.LogParser; + +/** + * Validates the input parameters for the {@link LogParser}. + */ +public class ParserValidator { + /** + * Validates the input parameters for the {@link LogParser}. + * + * @param logs input log streams to the {@link LogParser}. + * @return whether the input parameters are valid or not. + */ + public final boolean validate(final InputStream logs) { + // TODO + return true; + } +} diff --git a/hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/translator/validator/package-info.java b/hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/translator/validator/package-info.java new file mode 100644 index 0000000000..c48b262265 --- /dev/null +++ b/hadoop-tools/hadoop-resourceestimator/src/main/java/org/apache/hadoop/resourceestimator/translator/validator/package-info.java @@ -0,0 +1,23 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/** + * Validator for {@code Translator}. + */ + +package org.apache.hadoop.resourceestimator.translator.validator; \ No newline at end of file diff --git a/hadoop-tools/hadoop-resourceestimator/src/main/resources/webapps/ResourceEstimatorServer/.gitignore b/hadoop-tools/hadoop-resourceestimator/src/main/resources/webapps/ResourceEstimatorServer/.gitignore new file mode 100644 index 0000000000..ae1e83eeb3 --- /dev/null +++ b/hadoop-tools/hadoop-resourceestimator/src/main/resources/webapps/ResourceEstimatorServer/.gitignore @@ -0,0 +1,14 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT 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/hadoop-tools/hadoop-resourceestimator/src/site/markdown/ResourceEstimator.md b/hadoop-tools/hadoop-resourceestimator/src/site/markdown/ResourceEstimator.md new file mode 100644 index 0000000000..12f8dd5ef2 --- /dev/null +++ b/hadoop-tools/hadoop-resourceestimator/src/site/markdown/ResourceEstimator.md @@ -0,0 +1,181 @@ + + +Resource Estimator Service +========================== + +* [Resource Estimator Service](#Resource_Estimator_Service) + * [Overview](#Overview) + * [Motivation](#Motivation) + * [Goals](#Goals) + * [Architecture](#Architecture) + * [Usage](#Usage) + * [Example](#Example) + * [Advanced Configuration](#AdvancedConfig) + * [Future work](#Future) + +Overview +-------- + +### Motivation +Estimating job resource requirements remains an important and challenging problem for enterprise clusters. This is amplified by the ever-increasing complexity of workloads, i.e. from traditional batch jobs to interactive queries to streaming and recently machine learning jobs. This results in jobs relying on multiple computation frameworks such as Tez, MapReduce, Spark, etc., and the problem is further compounded by sharing nature of the clusters. Current state-of-art solution relies on user expertise to make resource requirement estimations for the jobs (for e.g.: number of reducers or container memory size, etc.), which is both tedious and inefficient. + +Based on the analysis of our cluster workloads, we observe that a large portion of jobs (more than 60%) are recurring jobs, giving us the opportunity to automatically estimate job resource requirements based on job's history runs. It is worth noting that jobs usually come from different computation frameworks, and the version may change across runs as well. Therefore, we want to come up with a framework agnostic black-box solution to automatically make resource requirement estimation for the recurring jobs. + +### Goals + +* For a periodic job, analyze its history logs and predict its resource requirement for the new run. +* Support various types of job logs. +* Scale to terabytes of job logs. + +### Architecture + +The following figure illustrates the implementation architecture of the resource estimator. + +![The architecture of the resource estimator](images/resourceestimator_arch.png) + +Hadoop-resourceestimator mainly consists of three modules: Translator, SkylineStore and Estimator. + +1. `ResourceSkyline` is used to characterize job's resource utilizations during its lifespan. More specifically, it uses `RLESparseResourceAllocation` () to record the container allocation information. `RecurrenceId` is used to identify a specific run of a recurring pipeline. A pipeline could consist of multiple jobs, each has a `ResourceSkyline` to characterize its resource utilization. +2. `Translator` parses the job logs, extracts their `ResourceSkylines` and stores them to the SkylineStore. `SingleLineParser` parses one line in the log stream and extract the `ResourceSkyline`. `LogParser` recursively parses each line in the log stream using `SingleLineParser`. Note that logs could have different storage formats, so `LogParser` takes a stream of strings as input, instead of File or other formats. Since job logs may have various formats thus require different `SingleLineParser` implementations, `LogParser` initiates the `SingleLineParser` based on user configuration. Currently Hadoop-resourceestimator provides two implementations for `SingleLineParser`: `NativeSingleLineParser` supports an optimized native format, and `RMSingleLineParser` parses the YARN ResourceManager logs generated in Hadoop systems since RM logs are widely available (in production deployments). +3. `SkylineStore` serves as the storage layer for Hadoop-resourceestimator and consists of 2 parts. `HistorySkylineStore` stores the `ResourceSkylines` extracted by the `Translator`. It supports four actions: addHistory, deleteHistory, updateHistory and getHistory. addHistory appends new `ResourceSkylines` to the recurring pipelines, while updateHistory deletes all the `ResourceSkylines` of a specific recurring pipeline, and re-insert new `ResourceSkylines`. `PredictionSkylineStore` stores the predicted `RLESparseResourceAllocation` generated by the Estimator. It supports two actions: addEstimation and getEstimation. + + Currently Hadoop-resourceestimator provides in-memory implementation for the SkylineStore. +4. `Estimator` predicts recurring pipeline's resource requirements based on its history runs, stores the prediction to the `SkylineStore` and makes recurring resource reservations to YARN (YARN-5326). `Solver` reads all the history `ResourceSkylines` of a specific recurring pipeline, and predicts its new resource requirements wrapped in `RLESparseResourceAllocation`. Currently Hadoop-resourceestimator provides a `LPSOLVER` to make the prediction (the details of the Linear Programming model can be find in the paper). There is also a `BaseSolver` to translate predicted resource requirements into `ReservationSubmissionRequest` which is used by different solver implementations to make recurring resource reservations on YARN. +5. `ResourceEstimationService` wraps Hadoop-resourceestimator as a micro-service, which can be easily deployed in clusters. It provides a set of REST APIs to allow users to parse specified job logs, query pipeline's history `ResourceSkylines`, query pipeline's predicted resource requirements and run the `SOLVER` if the prediction does not exist, delete the `ResourceSkylines` in the `SkylineStore`. + +Usage +----- + +This section will guide you through the usage of resource estimator service. + +Here let `$HADOOP_ROOT` represent the Hadoop install directory. If you build Hadoop yourself, `$HADOOP_ROOT` is `hadoop-dist/target/hadoop-$VERSION`. The location of the resource estimator service, `$ResourceEstimatorServiceHome`, is `$HADOOP_ROOT/share/hadoop/tools/resourceestimator`. It contains 3 folders: `bin`, `conf` and `data`. Please note that users can use resource estimator service with the default configurations. + +* `bin` contains the running scripts for the resource estimator service. + +* `conf`: contains the configuration files for the resource estimator service. + +* `data` contains the sample log that is used to run the example of resource estimator service. + +### Step 1: Start the estimator + +First of all, copy the configuration file (located in `$ResourceEstimatorServiceHome/conf/`) to `$HADOOP_ROOT/etc/hadoop`. + +The script to start the estimator is `start-estimator.sh`. + + $ cd $ResourceEstimatorServiceHome + $ bin/start-estimator.sh + +A web server is started, and users can use the resource estimation service through REST APIs. + +### Step 2: Run the estimator + +The URI for the resource estimator sercive is `http://0.0.0.0`, and the default service port is `9998` (configured in `$ResourceEstimatorServiceHome/conf/resourceestimator-config.xml`). In `$ResourceEstimatorServiceHome/data`, there is a sample log file `resourceEstimatorService.txt` which contains the logs of tpch_q12 query job for 2 runs. + +* `parse job logs: POST http://URI:port/resourceestimator/translator/LOG_FILE_DIRECTORY` + + Send `POST http://0.0.0.0:9998/resourceestimator/translator/data/resourceEstimatorService.txt`. The underlying estimator will extract the ResourceSkylines from the log file and store them in the jobHistory SkylineStore. + +* `query job's history ResourceSkylines: GET http://URI:port/resourceestimator/skylinestore/history/{pipelineId}/{runId}` + + Send `GET http://0.0.0.0:9998/resourceestimator/skylinestore/history/*/*`, and the underlying estimator will return all the records in history SkylineStore. You should be able to see ResourceSkylines for two runs of tpch_q12: tpch_q12_0 and tpch_q12_1. Note that both `pipelineId` and `runId` fields support wildcard operations. + +* `predict job's resource skyline requirement: GET http://URI:port/resourceestimator/estimator/{pipelineId}` + + Send `http://0.0.0.0:9998/resourceestimator/estimator/tpch_q12`, and the underlying estimator will predict job's resource requirements for the new run based on its history ResourceSkylines, and store the predicted resource requirements to jobEstimation SkylineStore. + +* `query job's estimated resource skylines: GET http://URI:port/resourceestimator/skylinestore/estimation/{pipelineId}` + + Send `http://0.0.0.0:9998/resourceestimator/skylinestore/estimation/tpch_q12`, and the underlying estimator will return the history resource requirement estimation for tpch_q12 job. Note that for jobEstimation SkylineStore, it does not support wildcard operations. + +* `delete job's history resource skylines: DELETE http://URI:port/resourceestimator/skylinestore/history/{pipelineId}/{runId}` + + Send `http://0.0.0.0:9998/resourceestimator/skylinestore/history/tpch_q12/tpch_q12_0`, and the underlying estimator will delete the ResourceSkyline record for tpch_q12_0. Re-send `GET http://0.0.0.0:9998/resourceestimator/skylinestore/history/*/*`, and the underlying estimator will only return the ResourceSkyline for tpch_q12_1. + +### Step 3: Run the estimator +The script to stop the estimator is `stop-estimator.sh`. + + $ cd $ResourceEstimatorServiceHome + $ bin/stop-estimator.sh + +Example +------- + +Here we present an example for using Resource Estimator Service. + +First, we run a tpch_q12 job for 9 times, and collect job's resource skylines in each run (note that in this example, we only collect "# of allocated containers" information). + +Then, we run the log parser in Resource Estimator Service to extract the ResourceSkylines from logs and store them in the SkylineStore. The job's ResourceSkylines are plotted below for demonstration. + +![Tpch job history runs](images/tpch_history.png) + +Finally, we run the estimator in Resource Estimator Service to predict the resource requirements for the new run, which is wrapped in RLESparseResourceAllocation (). The predicted resource requirement is plotted below for demonstration. + +![Tpch job history prediction](images/tpch_predict.png) + +Advanced Configuration +---------------------- + +This section will guide you through the configuration for Resource Estimator Service. The configuration file is located at `$ResourceEstimatorServiceHome/conf/resourceestimator-config.xml`. + +* `resourceestimator.solver.lp.alpha` + + The resource estimator has an integrated Linear Programming solver to make the prediction (refer to for more details), and this parameter tunes the tradeoff between resource over-allocation and under-allocation in the Linear Programming model. This parameter varies from 0 to 1, and a larger alpha value means the model minimizes over-allocation better. Default value is 0.1. + +* `resourceestimator.solver.lp.beta` + + This parameter controls the generalization of the Linear Programming model. This parameter varies from 0 to 1. Deafult value is 0.1. + +* `resourceestimator.solver.lp.minJobRuns` + + The minimum number of job runs required in order to make the prediction. Default value is 2. + +* `resourceestimator.timeInterval` + + The time length which is used to discretize job execution into intervals. Note that the estimator makes resource allocation prediction for each interval. A smaller time interval has more fine-grained granularity for prediction, but it also takes longer time and more space for prediction. Default value is 5 (seconds). + +* `resourceestimator.skylinestore.provider` + + The class name of the skylinestore provider. Default value is `org.apache.hadoop.resourceestimator.skylinestore.impl.InMemoryStore`, which is an in-memory implementation of skylinestore. If users want to use their own skylinestore implementation, they need to change this value accordingly. + +* `resourceestimator.translator.provider` + + The class name of the translator provider. Default value is `org.apache.hadoop.resourceestimator.translator.impl.BaseLogParser`, which extracts resourceskylines from log streams. If users want to use their own translator implementation, they need to change this value accordingly. + +* `resourceestimator.translator.line-parser` + + The class name of the translator single-line parser, which parses a single line in the log. Default value is `org.apache.hadoop.resourceestimator.translator.impl.NativeSingleLineParser`, which can parse one line in the sample log. Note that if users want to parse Hadoop Resource Manager () logs, they need to set the value to be `org.apache.hadoop.resourceestimator.translator.impl.RmSingleLineParser`. If they want to implement single-line parser to parse their customized log file, they need to change this value accordingly. + +* `resourceestimator.solver.provider` + + The class name of the solver provider. Default value is `org.apache.hadoop.resourceestimator.solver.impl.LpSolver`, which incorporates a Linear Programming model to make the prediction. If users want to implement their own models, they need to change this value accordingly. + +* `resourceestimator.service-port` + + The port which ResourceEstimatorService listens to. The default value is 9998. + +Future work +----------- + +1. For SkylineStore, we plan to provide a persistent store implementation. We are considering HBase to future proof our scale requirements. + +2. For Translator module, we want to support Timeline Service v2 as the primary source as we want to rely on a stable API and logs are flaky at best. + +3. Job resource requirements could vary across runs due to skewness, contention, input data or code changes, etc. We want to design a Reprovisioner module, which dynamically monitors job progress at runtime, identifies the performance bottlenecks if the progress is slower than expectation, and dynamically adjusts job's resource allocations accordingly using ReservationUpdateRequest. + +4. When Estimator predicts job's resource requirements, we want to provide the confidence level associated with the prediction according to the estimation error (combination of over-allocation and under-allocation), etc. + +5. For Estimator module, we can integrate machine learning tools such as reinforcement learning to make better prediction. We can also integrate with domain-specific solvers such as PerfOrator to improve prediction quality. + +6. For Estimator module, we want to design incremental solver, which can incrementally update job's resource requirements only based on the new logs. diff --git a/hadoop-tools/hadoop-resourceestimator/src/site/resources/css/site.css b/hadoop-tools/hadoop-resourceestimator/src/site/resources/css/site.css new file mode 100644 index 0000000000..7315db31e5 --- /dev/null +++ b/hadoop-tools/hadoop-resourceestimator/src/site/resources/css/site.css @@ -0,0 +1,29 @@ +/* +* Licensed to the Apache Software Foundation (ASF) under one or more +* contributor license agreements. See the NOTICE file distributed with +* this work for additional information regarding copyright ownership. +* The ASF licenses this file to You under the Apache License, Version 2.0 +* (the "License"); you may not use this file except in compliance with +* the License. You may obtain a copy of the License at +* +* http://www.apache.org/licenses/LICENSE-2.0 +* +* Unless required by applicable law or agreed to in writing, software +* distributed under the License is distributed on an "AS IS" BASIS, +* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +* See the License for the specific language governing permissions and +* limitations under the License. +*/ +#banner { + height: 93px; + background: none; +} + +#bannerLeft img { + margin-left: 30px; + margin-top: 10px; +} + +#bannerRight img { + margin: 17px; +} diff --git a/hadoop-tools/hadoop-resourceestimator/src/test/java/org/apache/hadoop/resourceestimator/common/api/TestResourceSkyline.java b/hadoop-tools/hadoop-resourceestimator/src/test/java/org/apache/hadoop/resourceestimator/common/api/TestResourceSkyline.java new file mode 100644 index 0000000000..65e1c34adc --- /dev/null +++ b/hadoop-tools/hadoop-resourceestimator/src/test/java/org/apache/hadoop/resourceestimator/common/api/TestResourceSkyline.java @@ -0,0 +1,128 @@ +/* + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package org.apache.hadoop.resourceestimator.common.api; + +import java.util.TreeMap; + +import org.apache.hadoop.yarn.api.records.Resource; +import org.apache.hadoop.yarn.server.resourcemanager.reservation.RLESparseResourceAllocation; +import org.apache.hadoop.yarn.server.resourcemanager.reservation.ReservationInterval; +import org.apache.hadoop.yarn.util.resource.DefaultResourceCalculator; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +/** + * Test {@link ResourceSkyline} class. + */ +public class TestResourceSkyline { + /** + * Testing variables. + */ + private ResourceSkyline resourceSkyline; + + private Resource resource1; + private Resource resource2; + private TreeMap resourceOverTime; + private RLESparseResourceAllocation skylineList; + + @Before public final void setup() { + resourceOverTime = new TreeMap<>(); + skylineList = new RLESparseResourceAllocation(resourceOverTime, + new DefaultResourceCalculator()); + resource1 = Resource.newInstance(1024 * 100, 100); + resource2 = Resource.newInstance(1024 * 200, 200); + } + + @Test public final void testGetJobId() { + Assert.assertNull(resourceSkyline); + ReservationInterval riAdd = new ReservationInterval(0, 10); + skylineList.addInterval(riAdd, resource1); + riAdd = new ReservationInterval(10, 20); + skylineList.addInterval(riAdd, resource1); + resourceSkyline = + new ResourceSkyline("1", 1024.5, 0, 20, resource1, skylineList); + Assert.assertEquals("1", resourceSkyline.getJobId()); + } + + @Test public final void testGetJobSubmissionTime() { + Assert.assertNull(resourceSkyline); + ReservationInterval riAdd = new ReservationInterval(0, 10); + skylineList.addInterval(riAdd, resource1); + riAdd = new ReservationInterval(10, 20); + skylineList.addInterval(riAdd, resource1); + resourceSkyline = + new ResourceSkyline("1", 1024.5, 0, 20, resource1, skylineList); + Assert.assertEquals(0, resourceSkyline.getJobSubmissionTime()); + } + + @Test public final void testGetJobFinishTime() { + Assert.assertNull(resourceSkyline); + ReservationInterval riAdd = new ReservationInterval(0, 10); + skylineList.addInterval(riAdd, resource1); + riAdd = new ReservationInterval(10, 20); + skylineList.addInterval(riAdd, resource1); + resourceSkyline = + new ResourceSkyline("1", 1024.5, 0, 20, resource1, skylineList); + Assert.assertEquals(20, resourceSkyline.getJobFinishTime()); + } + + @Test public final void testGetKthResource() { + Assert.assertNull(resourceSkyline); + ReservationInterval riAdd = new ReservationInterval(10, 20); + skylineList.addInterval(riAdd, resource1); + riAdd = new ReservationInterval(20, 30); + skylineList.addInterval(riAdd, resource2); + resourceSkyline = + new ResourceSkyline("1", 1024.5, 0, 20, resource1, skylineList); + final RLESparseResourceAllocation skylineList2 = + resourceSkyline.getSkylineList(); + for (int i = 10; i < 20; i++) { + Assert.assertEquals(resource1.getMemorySize(), + skylineList2.getCapacityAtTime(i).getMemorySize()); + Assert.assertEquals(resource1.getVirtualCores(), + skylineList2.getCapacityAtTime(i).getVirtualCores()); + } + for (int i = 20; i < 30; i++) { + Assert.assertEquals(resource2.getMemorySize(), + skylineList2.getCapacityAtTime(i).getMemorySize()); + Assert.assertEquals(resource2.getVirtualCores(), + skylineList2.getCapacityAtTime(i).getVirtualCores()); + } + // test if resourceSkyline automatically extends the skyline with + // zero-resource at both ends + Assert.assertEquals(0, skylineList2.getCapacityAtTime(9).getMemorySize()); + Assert.assertEquals(0, skylineList2.getCapacityAtTime(9).getVirtualCores()); + Assert.assertEquals(0, skylineList2.getCapacityAtTime(30).getMemorySize()); + Assert + .assertEquals(0, skylineList2.getCapacityAtTime(30).getVirtualCores()); + } + + @After public final void cleanUp() { + resourceSkyline = null; + resource1 = null; + resource2 = null; + resourceOverTime.clear(); + resourceOverTime = null; + skylineList = null; + } +} diff --git a/hadoop-tools/hadoop-resourceestimator/src/test/java/org/apache/hadoop/resourceestimator/common/serialization/TestHistorySkylineSerDe.java b/hadoop-tools/hadoop-resourceestimator/src/test/java/org/apache/hadoop/resourceestimator/common/serialization/TestHistorySkylineSerDe.java new file mode 100644 index 0000000000..62743aad91 --- /dev/null +++ b/hadoop-tools/hadoop-resourceestimator/src/test/java/org/apache/hadoop/resourceestimator/common/serialization/TestHistorySkylineSerDe.java @@ -0,0 +1,134 @@ +/* + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package org.apache.hadoop.resourceestimator.common.serialization; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.TreeMap; + +import org.apache.hadoop.resourceestimator.common.api.RecurrenceId; +import org.apache.hadoop.resourceestimator.common.api.ResourceSkyline; +import org.apache.hadoop.yarn.api.records.Resource; +import org.apache.hadoop.yarn.server.resourcemanager.reservation.RLESparseResourceAllocation; +import org.apache.hadoop.yarn.server.resourcemanager.reservation.ReservationInterval; +import org.apache.hadoop.yarn.util.resource.DefaultResourceCalculator; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import com.google.gson.Gson; +import com.google.gson.GsonBuilder; +import com.google.gson.reflect.TypeToken; + +/** + * Test HistorySkylineSerDe. + */ +public class TestHistorySkylineSerDe { + /** + * Testing variables. + */ + private Gson gson; + + private ResourceSkyline resourceSkyline; + private Resource resource; + private Resource resource2; + private TreeMap resourceOverTime; + private RLESparseResourceAllocation skylineList; + + @Before public final void setup() { + resourceOverTime = new TreeMap<>(); + skylineList = new RLESparseResourceAllocation(resourceOverTime, + new DefaultResourceCalculator()); + resource = Resource.newInstance(1024 * 100, 100); + resource2 = Resource.newInstance(1024 * 200, 200); + gson = new GsonBuilder() + .registerTypeAdapter(Resource.class, new ResourceSerDe()) + .registerTypeAdapter(RLESparseResourceAllocation.class, + new RLESparseResourceAllocationSerDe()) + .enableComplexMapKeySerialization().create(); + } + + @Test public final void testSerialization() { + ReservationInterval riAdd = new ReservationInterval(0, 10); + skylineList.addInterval(riAdd, resource); + riAdd = new ReservationInterval(10, 20); + skylineList.addInterval(riAdd, resource2); + resourceSkyline = + new ResourceSkyline("1", 1024.5, 0, 20, resource, skylineList); + RecurrenceId recurrenceId = new RecurrenceId("FraudDetection", "1"); + List listSkyline = new ArrayList<>(); + listSkyline.add(resourceSkyline); + Map> historySkyline = + new HashMap>(); + historySkyline.put(recurrenceId, listSkyline); + + final String json = gson.toJson(historySkyline, + new TypeToken>>() { + }.getType()); + final Map> historySkylineDe = + gson.fromJson(json, + new TypeToken>>() { + }.getType()); + // check if the recurrenceId is correct + List resourceSkylineList = + historySkylineDe.get(recurrenceId); + Assert.assertNotNull(resourceSkylineList); + Assert.assertEquals(1, resourceSkylineList.size()); + + // check if the resourceSkyline is correct + ResourceSkyline resourceSkylineDe = resourceSkylineList.get(0); + Assert + .assertEquals(resourceSkylineDe.getJobId(), resourceSkyline.getJobId()); + Assert.assertEquals(resourceSkylineDe.getJobInputDataSize(), + resourceSkyline.getJobInputDataSize(), 0); + Assert.assertEquals(resourceSkylineDe.getJobSubmissionTime(), + resourceSkyline.getJobSubmissionTime()); + Assert.assertEquals(resourceSkylineDe.getJobFinishTime(), + resourceSkyline.getJobFinishTime()); + Assert.assertEquals(resourceSkylineDe.getContainerSpec().getMemorySize(), + resourceSkyline.getContainerSpec().getMemorySize()); + Assert.assertEquals(resourceSkylineDe.getContainerSpec().getVirtualCores(), + resourceSkyline.getContainerSpec().getVirtualCores()); + final RLESparseResourceAllocation skylineList2 = + resourceSkyline.getSkylineList(); + final RLESparseResourceAllocation skylineListDe = + resourceSkylineDe.getSkylineList(); + for (int i = 0; i < 20; i++) { + Assert.assertEquals(skylineList2.getCapacityAtTime(i).getMemorySize(), + skylineListDe.getCapacityAtTime(i).getMemorySize()); + Assert.assertEquals(skylineList2.getCapacityAtTime(i).getVirtualCores(), + skylineListDe.getCapacityAtTime(i).getVirtualCores()); + } + } + + @After public final void cleanUp() { + gson = null; + resourceSkyline = null; + resourceOverTime.clear(); + resourceOverTime = null; + resource = null; + resource2 = null; + skylineList = null; + } +} diff --git a/hadoop-tools/hadoop-resourceestimator/src/test/java/org/apache/hadoop/resourceestimator/common/serialization/TestResourceSerDe.java b/hadoop-tools/hadoop-resourceestimator/src/test/java/org/apache/hadoop/resourceestimator/common/serialization/TestResourceSerDe.java new file mode 100644 index 0000000000..8749e598f7 --- /dev/null +++ b/hadoop-tools/hadoop-resourceestimator/src/test/java/org/apache/hadoop/resourceestimator/common/serialization/TestResourceSerDe.java @@ -0,0 +1,64 @@ +/* + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package org.apache.hadoop.resourceestimator.common.serialization; + +import org.apache.hadoop.yarn.api.records.Resource; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import com.google.gson.Gson; +import com.google.gson.GsonBuilder; +import com.google.gson.reflect.TypeToken; + +/** + * Test ResourceSerDe. + */ +public class TestResourceSerDe { + /** + * Testing variables. + */ + private Gson gson; + + private Resource resource; + + @Before public final void setup() { + resource = Resource.newInstance(1024 * 100, 100); + gson = new GsonBuilder() + .registerTypeAdapter(Resource.class, new ResourceSerDe()).create(); + } + + @Test public final void testSerialization() { + final String json = gson.toJson(resource, new TypeToken() { + }.getType()); + final Resource resourceDe = gson.fromJson(json, new TypeToken() { + }.getType()); + Assert.assertEquals(resource.getMemorySize(), resourceDe.getMemorySize()); + Assert + .assertEquals(resource.getVirtualCores(), resourceDe.getVirtualCores()); + } + + @After public final void cleanUp() { + resource = null; + gson = null; + } +} diff --git a/hadoop-tools/hadoop-resourceestimator/src/test/java/org/apache/hadoop/resourceestimator/common/serialization/TestResourceSkylineSerDe.java b/hadoop-tools/hadoop-resourceestimator/src/test/java/org/apache/hadoop/resourceestimator/common/serialization/TestResourceSkylineSerDe.java new file mode 100644 index 0000000000..859ea1e06f --- /dev/null +++ b/hadoop-tools/hadoop-resourceestimator/src/test/java/org/apache/hadoop/resourceestimator/common/serialization/TestResourceSkylineSerDe.java @@ -0,0 +1,112 @@ +/* + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package org.apache.hadoop.resourceestimator.common.serialization; + +import java.util.TreeMap; + +import org.apache.hadoop.resourceestimator.common.api.ResourceSkyline; +import org.apache.hadoop.yarn.api.records.Resource; +import org.apache.hadoop.yarn.server.resourcemanager.reservation.RLESparseResourceAllocation; +import org.apache.hadoop.yarn.server.resourcemanager.reservation.ReservationInterval; +import org.apache.hadoop.yarn.util.resource.DefaultResourceCalculator; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import com.google.gson.Gson; +import com.google.gson.GsonBuilder; +import com.google.gson.reflect.TypeToken; + +/** + * Test ResourceSkylineSerDe. + */ +public class TestResourceSkylineSerDe { + /** + * Testing variables. + */ + private Gson gson; + + private ResourceSkyline resourceSkyline; + private Resource resource; + private Resource resource2; + private TreeMap resourceOverTime; + private RLESparseResourceAllocation skylineList; + + @Before public final void setup() { + resourceOverTime = new TreeMap<>(); + skylineList = new RLESparseResourceAllocation(resourceOverTime, + new DefaultResourceCalculator()); + resource = Resource.newInstance(1024 * 100, 100); + resource2 = Resource.newInstance(1024 * 200, 200); + gson = new GsonBuilder() + .registerTypeAdapter(Resource.class, new ResourceSerDe()) + .registerTypeAdapter(RLESparseResourceAllocation.class, + new RLESparseResourceAllocationSerDe()).create(); + } + + @Test public final void testSerialization() { + ReservationInterval riAdd = new ReservationInterval(0, 10); + skylineList.addInterval(riAdd, resource); + riAdd = new ReservationInterval(10, 20); + skylineList.addInterval(riAdd, resource2); + resourceSkyline = + new ResourceSkyline("1", 1024.5, 0, 20, resource, skylineList); + final String json = + gson.toJson(resourceSkyline, new TypeToken() { + }.getType()); + final ResourceSkyline resourceSkylineDe = + gson.fromJson(json, new TypeToken() { + }.getType()); + Assert + .assertEquals(resourceSkylineDe.getJobId(), resourceSkyline.getJobId()); + Assert.assertEquals(resourceSkylineDe.getJobInputDataSize(), + resourceSkyline.getJobInputDataSize(), 0); + Assert.assertEquals(resourceSkylineDe.getJobSubmissionTime(), + resourceSkyline.getJobSubmissionTime()); + Assert.assertEquals(resourceSkylineDe.getJobFinishTime(), + resourceSkyline.getJobFinishTime()); + Assert.assertEquals(resourceSkylineDe.getContainerSpec().getMemorySize(), + resourceSkyline.getContainerSpec().getMemorySize()); + Assert.assertEquals(resourceSkylineDe.getContainerSpec().getVirtualCores(), + resourceSkyline.getContainerSpec().getVirtualCores()); + final RLESparseResourceAllocation skylineList2 = + resourceSkyline.getSkylineList(); + final RLESparseResourceAllocation skylineListDe = + resourceSkylineDe.getSkylineList(); + for (int i = 0; i < 20; i++) { + Assert.assertEquals(skylineList2.getCapacityAtTime(i).getMemorySize(), + skylineListDe.getCapacityAtTime(i).getMemorySize()); + Assert.assertEquals(skylineList2.getCapacityAtTime(i).getVirtualCores(), + skylineListDe.getCapacityAtTime(i).getVirtualCores()); + } + } + + @After public final void cleanUp() { + gson = null; + resourceSkyline = null; + resourceOverTime.clear(); + resourceOverTime = null; + resource = null; + resource2 = null; + skylineList = null; + } +} diff --git a/hadoop-tools/hadoop-resourceestimator/src/test/java/org/apache/hadoop/resourceestimator/common/serialization/package-info.java b/hadoop-tools/hadoop-resourceestimator/src/test/java/org/apache/hadoop/resourceestimator/common/serialization/package-info.java new file mode 100644 index 0000000000..06957b39a7 --- /dev/null +++ b/hadoop-tools/hadoop-resourceestimator/src/test/java/org/apache/hadoop/resourceestimator/common/serialization/package-info.java @@ -0,0 +1,24 @@ +/* + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +/** + * SkylineStore serialization module. + */ +package org.apache.hadoop.resourceestimator.common.serialization; diff --git a/hadoop-tools/hadoop-resourceestimator/src/test/java/org/apache/hadoop/resourceestimator/service/GuiceServletConfig.java b/hadoop-tools/hadoop-resourceestimator/src/test/java/org/apache/hadoop/resourceestimator/service/GuiceServletConfig.java new file mode 100644 index 0000000000..b72d317402 --- /dev/null +++ b/hadoop-tools/hadoop-resourceestimator/src/test/java/org/apache/hadoop/resourceestimator/service/GuiceServletConfig.java @@ -0,0 +1,42 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + *

+ * http://www.apache.org/licenses/LICENSE-2.0 + *

+ * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.resourceestimator.service; + +import com.google.inject.Injector; +import com.google.inject.servlet.GuiceServletContextListener; + +/** + * GuiceServletConfig is a wrapper class to have a static Injector instance + * instead of having the instance inside test classes. This allow us to use + * Jersey test framework after 1.13. + * Please check test cases to know how to use this class: + * e.g. TestRMWithCSRFFilter.java + */ +public class GuiceServletConfig extends GuiceServletContextListener { + + private static Injector internalInjector = null; + + @Override protected Injector getInjector() { + return internalInjector; + } + + public static Injector setInjector(Injector in) { + internalInjector = in; + return internalInjector; + } +} \ No newline at end of file diff --git a/hadoop-tools/hadoop-resourceestimator/src/test/java/org/apache/hadoop/resourceestimator/service/TestResourceEstimatorService.java b/hadoop-tools/hadoop-resourceestimator/src/test/java/org/apache/hadoop/resourceestimator/service/TestResourceEstimatorService.java new file mode 100644 index 0000000000..d4dde7ed47 --- /dev/null +++ b/hadoop-tools/hadoop-resourceestimator/src/test/java/org/apache/hadoop/resourceestimator/service/TestResourceEstimatorService.java @@ -0,0 +1,282 @@ +/* + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package org.apache.hadoop.resourceestimator.service; + +import java.util.List; +import java.util.Map; +import java.util.TreeMap; + +import javax.ws.rs.core.MediaType; + +import org.apache.hadoop.resourceestimator.common.api.RecurrenceId; +import org.apache.hadoop.resourceestimator.common.api.ResourceSkyline; +import org.apache.hadoop.resourceestimator.common.serialization.RLESparseResourceAllocationSerDe; +import org.apache.hadoop.resourceestimator.common.serialization.ResourceSerDe; +import org.apache.hadoop.yarn.api.records.Resource; +import org.apache.hadoop.yarn.server.resourcemanager.reservation.RLESparseResourceAllocation; +import org.apache.hadoop.yarn.server.resourcemanager.reservation.ReservationInterval; +import org.apache.hadoop.yarn.util.resource.DefaultResourceCalculator; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.google.gson.Gson; +import com.google.gson.GsonBuilder; +import com.google.gson.reflect.TypeToken; +import com.google.inject.Guice; +import com.google.inject.servlet.ServletModule; +import com.sun.jersey.api.client.WebResource; +import com.sun.jersey.guice.spi.container.servlet.GuiceContainer; +import com.sun.jersey.test.framework.JerseyTest; +import com.sun.jersey.test.framework.WebAppDescriptor; + +/** + * Test ResourceEstimatorService. + */ +public class TestResourceEstimatorService extends JerseyTest { + private static final Logger LOGGER = + LoggerFactory.getLogger(TestResourceEstimatorService.class); + private final String parseLogCommand = "resourceestimator/translator/" + + "src/test/resources/resourceEstimatorService.txt"; + private final String getHistorySkylineCommand = + "resourceestimator/skylinestore/history/tpch_q12/*"; + private final String getEstimatedSkylineCommand = + "resourceestimator/skylinestore/estimation/tpch_q12"; + private final String makeEstimationCommand = + "resourceestimator/estimator/tpch_q12"; + private final String deleteHistoryCommand = + "resourceestimator/skylinestore/history/tpch_q12/tpch_q12_1"; + private static boolean setUpDone = false; + private Resource containerSpec; + private Gson gson; + private long containerMemAlloc; + private int containerCPUAlloc; + + private static class WebServletModule extends ServletModule { + @Override protected void configureServlets() { + bind(ResourceEstimatorService.class); + serve("/*").with(GuiceContainer.class); + } + } + + static { + GuiceServletConfig + .setInjector(Guice.createInjector(new WebServletModule())); + } + + public TestResourceEstimatorService() { + super(new WebAppDescriptor.Builder( + "org.apache.hadoop.resourceestimator.service") + .contextListenerClass(GuiceServletConfig.class) + .filterClass(com.google.inject.servlet.GuiceFilter.class).build()); + } + + @Before @Override public void setUp() throws Exception { + super.setUp(); + GuiceServletConfig + .setInjector(Guice.createInjector(new WebServletModule())); + containerMemAlloc = 1024; + containerCPUAlloc = 1; + containerSpec = Resource.newInstance(containerMemAlloc, containerCPUAlloc); + gson = new GsonBuilder() + .registerTypeAdapter(Resource.class, new ResourceSerDe()) + .registerTypeAdapter(RLESparseResourceAllocation.class, + new RLESparseResourceAllocationSerDe()) + .enableComplexMapKeySerialization().create(); + } + + private void compareResourceSkyline(final ResourceSkyline skyline1, + final ResourceSkyline skyline2) { + Assert.assertEquals(skyline1.getJobId(), skyline2.getJobId()); + Assert.assertEquals(skyline1.getJobInputDataSize(), + skyline2.getJobInputDataSize(), 0); + Assert.assertEquals(skyline1.getJobSubmissionTime(), + skyline2.getJobSubmissionTime()); + Assert + .assertEquals(skyline1.getJobFinishTime(), skyline2.getJobFinishTime()); + Assert.assertEquals(skyline1.getContainerSpec().getMemorySize(), + skyline2.getContainerSpec().getMemorySize()); + Assert.assertEquals(skyline1.getContainerSpec().getVirtualCores(), + skyline2.getContainerSpec().getVirtualCores()); + final RLESparseResourceAllocation skylineList1 = skyline1.getSkylineList(); + final RLESparseResourceAllocation skylineList2 = skyline2.getSkylineList(); + for (int i = (int) skylineList1.getEarliestStartTime(); + i < skylineList1.getLatestNonNullTime(); i++) { + Assert.assertEquals(skylineList1.getCapacityAtTime(i).getMemorySize(), + skylineList2.getCapacityAtTime(i).getMemorySize()); + Assert.assertEquals(skylineList1.getCapacityAtTime(i).getVirtualCores(), + skylineList2.getCapacityAtTime(i).getVirtualCores()); + } + } + + private ResourceSkyline getSkyline1() { + final TreeMap resourceOverTime = new TreeMap<>(); + ReservationInterval riAdd; + final RLESparseResourceAllocation skylineList = + new RLESparseResourceAllocation(resourceOverTime, + new DefaultResourceCalculator()); + riAdd = new ReservationInterval(0, 10); + Resource resource = + Resource.newInstance(containerMemAlloc, containerCPUAlloc); + skylineList.addInterval(riAdd, resource); + riAdd = new ReservationInterval(10, 15); + resource = Resource + .newInstance(containerMemAlloc * 1074, containerCPUAlloc * 1074); + skylineList.addInterval(riAdd, resource); + riAdd = new ReservationInterval(15, 20); + resource = Resource + .newInstance(containerMemAlloc * 2538, containerCPUAlloc * 2538); + skylineList.addInterval(riAdd, resource); + riAdd = new ReservationInterval(20, 25); + resource = Resource + .newInstance(containerMemAlloc * 2468, containerCPUAlloc * 2468); + skylineList.addInterval(riAdd, resource); + final ResourceSkyline resourceSkyline1 = + new ResourceSkyline("tpch_q12_0", 0, 0, 25, containerSpec, skylineList); + + return resourceSkyline1; + } + + private ResourceSkyline getSkyline2() { + final TreeMap resourceOverTime = new TreeMap<>(); + ReservationInterval riAdd; + final RLESparseResourceAllocation skylineList = + new RLESparseResourceAllocation(resourceOverTime, + new DefaultResourceCalculator()); + riAdd = new ReservationInterval(0, 10); + Resource resource = + Resource.newInstance(containerMemAlloc, containerCPUAlloc); + skylineList.addInterval(riAdd, resource); + riAdd = new ReservationInterval(10, 15); + resource = + Resource.newInstance(containerMemAlloc * 794, containerCPUAlloc * 794); + skylineList.addInterval(riAdd, resource); + riAdd = new ReservationInterval(15, 20); + resource = Resource + .newInstance(containerMemAlloc * 2517, containerCPUAlloc * 2517); + skylineList.addInterval(riAdd, resource); + riAdd = new ReservationInterval(20, 25); + resource = Resource + .newInstance(containerMemAlloc * 2484, containerCPUAlloc * 2484); + skylineList.addInterval(riAdd, resource); + final ResourceSkyline resourceSkyline2 = + new ResourceSkyline("tpch_q12_1", 0, 0, 25, containerSpec, skylineList); + + return resourceSkyline2; + } + + private void checkResult(final String jobId, + final Map> jobHistory) { + switch (jobId) { + case "tpch_q12_0": { + final RecurrenceId recurrenceId = + new RecurrenceId("tpch_q12", "tpch_q12_0"); + Assert.assertEquals(1, jobHistory.get(recurrenceId).size()); + ResourceSkyline skylineReceive = jobHistory.get(recurrenceId).get(0); + compareResourceSkyline(skylineReceive, getSkyline1()); + break; + } + case "tpch_q12_1": { + final RecurrenceId recurrenceId = + new RecurrenceId("tpch_q12", "tpch_q12_1"); + Assert.assertEquals(1, jobHistory.get(recurrenceId).size()); + ResourceSkyline skylineReceive = jobHistory.get(recurrenceId).get(0); + compareResourceSkyline(skylineReceive, getSkyline2()); + break; + } + default: + break; + } + } + + private void compareRLESparseResourceAllocation( + final RLESparseResourceAllocation rle1, + final RLESparseResourceAllocation rle2) { + for (int i = (int) rle1.getEarliestStartTime(); + i < rle1.getLatestNonNullTime(); i++) { + Assert.assertEquals(rle1.getCapacityAtTime(i), rle2.getCapacityAtTime(i)); + } + } + + @Test public void testGetPrediction() { + // first, parse the log + final String logFile = "resourceEstimatorService.txt"; + WebResource webResource = resource(); + webResource.path(parseLogCommand).type(MediaType.APPLICATION_XML_TYPE) + .post(logFile); + webResource = resource().path(getHistorySkylineCommand); + String response = webResource.get(String.class); + Map> jobHistory = + gson.fromJson(response, + new TypeToken>>() { + }.getType()); + checkResult("tpch_q12_0", jobHistory); + checkResult("tpch_q12_1", jobHistory); + // then, try to get estimated resource allocation from skyline store + webResource = resource().path(getEstimatedSkylineCommand); + response = webResource.get(String.class); + Assert.assertEquals("null", response); + // then, we call estimator module to make the prediction + webResource = resource().path(makeEstimationCommand); + response = webResource.get(String.class); + RLESparseResourceAllocation skylineList = + gson.fromJson(response, new TypeToken() { + }.getType()); + Assert.assertEquals(1, + skylineList.getCapacityAtTime(0).getMemorySize() / containerMemAlloc); + Assert.assertEquals(1058, + skylineList.getCapacityAtTime(10).getMemorySize() / containerMemAlloc); + Assert.assertEquals(2538, + skylineList.getCapacityAtTime(15).getMemorySize() / containerMemAlloc); + Assert.assertEquals(2484, + skylineList.getCapacityAtTime(20).getMemorySize() / containerMemAlloc); + // then, we get estimated resource allocation for tpch_q12 + webResource = resource().path(getEstimatedSkylineCommand); + response = webResource.get(String.class); + final RLESparseResourceAllocation skylineList2 = + gson.fromJson(response, new TypeToken() { + }.getType()); + compareRLESparseResourceAllocation(skylineList, skylineList2); + // then, we call estimator module again to directly get estimated resource + // allocation from skyline store + webResource = resource().path(makeEstimationCommand); + response = webResource.get(String.class); + final RLESparseResourceAllocation skylineList3 = + gson.fromJson(response, new TypeToken() { + }.getType()); + compareRLESparseResourceAllocation(skylineList, skylineList3); + // finally, test delete + webResource = resource().path(deleteHistoryCommand); + webResource.delete(); + webResource = resource().path(getHistorySkylineCommand); + response = webResource.get(String.class); + jobHistory = gson.fromJson(response, + new TypeToken>>() { + }.getType()); + // jobHistory should only have info for tpch_q12_0 + Assert.assertEquals(1, jobHistory.size()); + final String pipelineId = + ((RecurrenceId) jobHistory.keySet().toArray()[0]).getRunId(); + Assert.assertEquals("tpch_q12_0", pipelineId); + } +} diff --git a/hadoop-tools/hadoop-resourceestimator/src/test/java/org/apache/hadoop/resourceestimator/skylinestore/impl/TestInMemoryStore.java b/hadoop-tools/hadoop-resourceestimator/src/test/java/org/apache/hadoop/resourceestimator/skylinestore/impl/TestInMemoryStore.java new file mode 100644 index 0000000000..4aefab4f26 --- /dev/null +++ b/hadoop-tools/hadoop-resourceestimator/src/test/java/org/apache/hadoop/resourceestimator/skylinestore/impl/TestInMemoryStore.java @@ -0,0 +1,32 @@ +/* + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package org.apache.hadoop.resourceestimator.skylinestore.impl; + +import org.apache.hadoop.resourceestimator.skylinestore.api.SkylineStore; + +/** + * Test {@link InMemoryStore} class. + */ +public class TestInMemoryStore extends TestSkylineStore { + @Override public final SkylineStore createSkylineStore() { + return new InMemoryStore(); + } +} diff --git a/hadoop-tools/hadoop-resourceestimator/src/test/java/org/apache/hadoop/resourceestimator/skylinestore/impl/TestSkylineStore.java b/hadoop-tools/hadoop-resourceestimator/src/test/java/org/apache/hadoop/resourceestimator/skylinestore/impl/TestSkylineStore.java new file mode 100644 index 0000000000..e333e3ff26 --- /dev/null +++ b/hadoop-tools/hadoop-resourceestimator/src/test/java/org/apache/hadoop/resourceestimator/skylinestore/impl/TestSkylineStore.java @@ -0,0 +1,464 @@ +/* + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package org.apache.hadoop.resourceestimator.skylinestore.impl; + +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.TreeMap; + +import org.apache.hadoop.resourceestimator.common.api.RecurrenceId; +import org.apache.hadoop.resourceestimator.common.api.ResourceSkyline; +import org.apache.hadoop.resourceestimator.skylinestore.api.SkylineStore; +import org.apache.hadoop.resourceestimator.skylinestore.exceptions.DuplicateRecurrenceIdException; +import org.apache.hadoop.resourceestimator.skylinestore.exceptions.EmptyResourceSkylineException; +import org.apache.hadoop.resourceestimator.skylinestore.exceptions.NullPipelineIdException; +import org.apache.hadoop.resourceestimator.skylinestore.exceptions.NullRLESparseResourceAllocationException; +import org.apache.hadoop.resourceestimator.skylinestore.exceptions.NullRecurrenceIdException; +import org.apache.hadoop.resourceestimator.skylinestore.exceptions.NullResourceSkylineException; +import org.apache.hadoop.resourceestimator.skylinestore.exceptions.RecurrenceIdNotFoundException; +import org.apache.hadoop.resourceestimator.skylinestore.exceptions.SkylineStoreException; +import org.apache.hadoop.yarn.api.records.Resource; +import org.apache.hadoop.yarn.server.resourcemanager.reservation.RLESparseResourceAllocation; +import org.apache.hadoop.yarn.server.resourcemanager.reservation.ReservationInterval; +import org.apache.hadoop.yarn.util.resource.DefaultResourceCalculator; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +/** + * Test {@link SkylineStore} class. + */ +public abstract class TestSkylineStore { + /** + * Testing variables. + */ + private SkylineStore skylineStore; + + private TreeMap resourceOverTime; + private RLESparseResourceAllocation skylineList; + private ReservationInterval riAdd; + private Resource resource; + + protected abstract SkylineStore createSkylineStore(); + + @Before public final void setup() { + skylineStore = createSkylineStore(); + resourceOverTime = new TreeMap<>(); + resource = Resource.newInstance(1024 * 100, 100); + } + + private void compare(final ResourceSkyline skyline1, + final ResourceSkyline skyline2) { + Assert.assertEquals(skyline1.getJobId(), skyline2.getJobId()); + Assert.assertEquals(skyline1.getJobInputDataSize(), + skyline2.getJobInputDataSize(), 0); + Assert.assertEquals(skyline1.getJobSubmissionTime(), + skyline2.getJobSubmissionTime()); + Assert + .assertEquals(skyline1.getJobFinishTime(), skyline2.getJobFinishTime()); + Assert.assertEquals(skyline1.getContainerSpec().getMemorySize(), + skyline2.getContainerSpec().getMemorySize()); + Assert.assertEquals(skyline1.getContainerSpec().getVirtualCores(), + skyline2.getContainerSpec().getVirtualCores()); + Assert.assertEquals(true, + skyline2.getSkylineList().equals(skyline1.getSkylineList())); + } + + private void addToStore(final RecurrenceId recurrenceId, + final ResourceSkyline resourceSkyline) throws SkylineStoreException { + final List resourceSkylines = new ArrayList<>(); + resourceSkylines.add(resourceSkyline); + skylineStore.addHistory(recurrenceId, resourceSkylines); + final List resourceSkylinesGet = + skylineStore.getHistory(recurrenceId).get(recurrenceId); + Assert.assertTrue(resourceSkylinesGet.contains(resourceSkyline)); + } + + private ResourceSkyline getSkyline(final int n) { + skylineList = new RLESparseResourceAllocation(resourceOverTime, + new DefaultResourceCalculator()); + for (int i = 0; i < n; i++) { + riAdd = new ReservationInterval(i * 10, (i + 1) * 10); + skylineList.addInterval(riAdd, resource); + } + final ResourceSkyline resourceSkyline = + new ResourceSkyline(Integer.toString(n), 1024.5, 0, 20, resource, + skylineList); + + return resourceSkyline; + } + + @Test public final void testGetHistory() throws SkylineStoreException { + // addHistory first recurring pipeline + final RecurrenceId recurrenceId1 = + new RecurrenceId("FraudDetection", "17/06/20 00:00:00"); + final ResourceSkyline resourceSkyline1 = getSkyline(1); + addToStore(recurrenceId1, resourceSkyline1); + final ResourceSkyline resourceSkyline2 = getSkyline(2); + addToStore(recurrenceId1, resourceSkyline2); + final RecurrenceId recurrenceId2 = + new RecurrenceId("FraudDetection", "17/06/21 00:00:00"); + final ResourceSkyline resourceSkyline3 = getSkyline(3); + addToStore(recurrenceId2, resourceSkyline3); + final ResourceSkyline resourceSkyline4 = getSkyline(4); + addToStore(recurrenceId2, resourceSkyline4); + // addHistory second recurring pipeline + final RecurrenceId recurrenceId3 = + new RecurrenceId("Random", "17/06/20 00:00:00"); + addToStore(recurrenceId3, resourceSkyline1); + addToStore(recurrenceId3, resourceSkyline2); + // test getHistory {pipelineId, runId} + Map> jobHistory = + skylineStore.getHistory(recurrenceId1); + Assert.assertEquals(1, jobHistory.size()); + for (final Map.Entry> entry : jobHistory + .entrySet()) { + Assert.assertEquals(recurrenceId1, entry.getKey()); + final List getSkylines = entry.getValue(); + Assert.assertEquals(2, getSkylines.size()); + compare(resourceSkyline1, getSkylines.get(0)); + compare(resourceSkyline2, getSkylines.get(1)); + } + // test getHistory {pipelineId, *} + RecurrenceId recurrenceIdTest = new RecurrenceId("FraudDetection", "*"); + jobHistory = skylineStore.getHistory(recurrenceIdTest); + Assert.assertEquals(2, jobHistory.size()); + for (final Map.Entry> entry : jobHistory + .entrySet()) { + Assert.assertEquals(recurrenceId1.getPipelineId(), + entry.getKey().getPipelineId()); + final List getSkylines = entry.getValue(); + if (entry.getKey().getRunId().equals("17/06/20 00:00:00")) { + Assert.assertEquals(2, getSkylines.size()); + compare(resourceSkyline1, getSkylines.get(0)); + compare(resourceSkyline2, getSkylines.get(1)); + } else { + Assert.assertEquals(entry.getKey().getRunId(), "17/06/21 00:00:00"); + Assert.assertEquals(2, getSkylines.size()); + compare(resourceSkyline3, getSkylines.get(0)); + compare(resourceSkyline4, getSkylines.get(1)); + } + } + // test getHistory {*, runId} + recurrenceIdTest = new RecurrenceId("*", "some random runId"); + jobHistory = skylineStore.getHistory(recurrenceIdTest); + Assert.assertEquals(3, jobHistory.size()); + for (final Map.Entry> entry : jobHistory + .entrySet()) { + if (entry.getKey().getPipelineId().equals("FraudDetection")) { + final List getSkylines = entry.getValue(); + if (entry.getKey().getRunId().equals("17/06/20 00:00:00")) { + Assert.assertEquals(2, getSkylines.size()); + compare(resourceSkyline1, getSkylines.get(0)); + compare(resourceSkyline2, getSkylines.get(1)); + } else { + Assert.assertEquals(entry.getKey().getRunId(), "17/06/21 00:00:00"); + Assert.assertEquals(2, getSkylines.size()); + compare(resourceSkyline3, getSkylines.get(0)); + compare(resourceSkyline4, getSkylines.get(1)); + } + } else { + Assert.assertEquals("Random", entry.getKey().getPipelineId()); + Assert.assertEquals(entry.getKey().getRunId(), "17/06/20 00:00:00"); + final List getSkylines = entry.getValue(); + Assert.assertEquals(2, getSkylines.size()); + compare(resourceSkyline1, getSkylines.get(0)); + compare(resourceSkyline2, getSkylines.get(1)); + } + } + // test getHistory with wrong RecurrenceId + recurrenceIdTest = + new RecurrenceId("some random pipelineId", "some random runId"); + Assert.assertNull(skylineStore.getHistory(recurrenceIdTest)); + } + + @Test public final void testGetEstimation() throws SkylineStoreException { + // first, add estimation to the skyline store + final RLESparseResourceAllocation skylineList2 = + new RLESparseResourceAllocation(resourceOverTime, + new DefaultResourceCalculator()); + for (int i = 0; i < 5; i++) { + riAdd = new ReservationInterval(i * 10, (i + 1) * 10); + skylineList2.addInterval(riAdd, resource); + } + skylineStore.addEstimation("FraudDetection", skylineList2); + // then, try to get the estimation + final RLESparseResourceAllocation estimation = + skylineStore.getEstimation("FraudDetection"); + for (int i = 0; i < 50; i++) { + Assert.assertEquals(skylineList2.getCapacityAtTime(i), + estimation.getCapacityAtTime(i)); + } + } + + @Test(expected = NullRecurrenceIdException.class) + public final void testGetNullRecurrenceId() + throws SkylineStoreException { + // addHistory first recurring pipeline + final RecurrenceId recurrenceId1 = + new RecurrenceId("FraudDetection", "17/06/20 00:00:00"); + final ResourceSkyline resourceSkyline1 = getSkyline(1); + addToStore(recurrenceId1, resourceSkyline1); + final ResourceSkyline resourceSkyline2 = getSkyline(2); + addToStore(recurrenceId1, resourceSkyline2); + final RecurrenceId recurrenceId2 = + new RecurrenceId("FraudDetection", "17/06/21 00:00:00"); + final ResourceSkyline resourceSkyline3 = getSkyline(3); + addToStore(recurrenceId2, resourceSkyline3); + final ResourceSkyline resourceSkyline4 = getSkyline(4); + addToStore(recurrenceId2, resourceSkyline4); + // addHistory second recurring pipeline + final RecurrenceId recurrenceId3 = + new RecurrenceId("Random", "17/06/20 00:00:00"); + addToStore(recurrenceId3, resourceSkyline1); + addToStore(recurrenceId3, resourceSkyline2); + // try to getHistory with null recurringId + skylineStore.getHistory(null); + } + + @Test(expected = NullPipelineIdException.class) + public final void testGetNullPipelineIdException() + throws SkylineStoreException { + skylineStore.getEstimation(null); + } + + @Test public final void testAddNormal() throws SkylineStoreException { + // addHistory resource skylines to the in-memory store + final RecurrenceId recurrenceId = + new RecurrenceId("FraudDetection", "17/06/20 00:00:00"); + final ResourceSkyline resourceSkyline1 = getSkyline(1); + addToStore(recurrenceId, resourceSkyline1); + final ArrayList resourceSkylines = + new ArrayList(); + // the resource skylines to be added contain null + resourceSkylines.add(null); + final ResourceSkyline resourceSkyline2 = getSkyline(2); + resourceSkylines.add(resourceSkyline2); + skylineStore.addHistory(recurrenceId, resourceSkylines); + // query the in-memory store + final Map> jobHistory = + skylineStore.getHistory(recurrenceId); + Assert.assertEquals(1, jobHistory.size()); + for (final Map.Entry> entry : jobHistory + .entrySet()) { + Assert.assertEquals(recurrenceId, entry.getKey()); + final List getSkylines = entry.getValue(); + Assert.assertEquals(2, getSkylines.size()); + compare(resourceSkyline1, getSkylines.get(0)); + compare(resourceSkyline2, getSkylines.get(1)); + } + } + + @Test(expected = NullRecurrenceIdException.class) + public final void testAddNullRecurrenceId() + throws SkylineStoreException { + // recurrenceId is null + final RecurrenceId recurrenceIdNull = null; + final ArrayList resourceSkylines = + new ArrayList(); + final ResourceSkyline resourceSkyline1 = getSkyline(1); + resourceSkylines.add(resourceSkyline1); + skylineStore.addHistory(recurrenceIdNull, resourceSkylines); + } + + @Test(expected = NullResourceSkylineException.class) + public final void testAddNullResourceSkyline() + throws SkylineStoreException { + final RecurrenceId recurrenceId = + new RecurrenceId("FraudDetection", "17/06/20 00:00:00"); + final ArrayList resourceSkylines = + new ArrayList(); + final ResourceSkyline resourceSkyline1 = getSkyline(1); + resourceSkylines.add(resourceSkyline1); + // resourceSkylines is null + skylineStore.addHistory(recurrenceId, null); + } + + @Test(expected = DuplicateRecurrenceIdException.class) + public final void testAddDuplicateRecurrenceId() + throws SkylineStoreException { + final RecurrenceId recurrenceId = + new RecurrenceId("FraudDetection", "17/06/20 00:00:00"); + final ArrayList resourceSkylines = + new ArrayList(); + final ResourceSkyline resourceSkyline1 = getSkyline(1); + resourceSkylines.add(resourceSkyline1); + // trying to addHistory duplicate resource skylines + skylineStore.addHistory(recurrenceId, resourceSkylines); + skylineStore.addHistory(recurrenceId, resourceSkylines); + } + + @Test(expected = NullPipelineIdException.class) + public final void testAddNullPipelineIdException() + throws SkylineStoreException { + final RLESparseResourceAllocation skylineList2 = + new RLESparseResourceAllocation(resourceOverTime, + new DefaultResourceCalculator()); + for (int i = 0; i < 5; i++) { + riAdd = new ReservationInterval(i * 10, (i + 1) * 10); + skylineList2.addInterval(riAdd, resource); + } + skylineStore.addEstimation(null, skylineList2); + } + + @Test(expected = NullRLESparseResourceAllocationException.class) + public final void testAddNullRLESparseResourceAllocationExceptionException() + throws SkylineStoreException { + skylineStore.addEstimation("FraudDetection", null); + } + + @Test public final void testDeleteNormal() throws SkylineStoreException { + // addHistory first recurring pipeline + final RecurrenceId recurrenceId1 = + new RecurrenceId("FraudDetection", "17/06/20 00:00:00"); + final ResourceSkyline resourceSkyline1 = getSkyline(1); + addToStore(recurrenceId1, resourceSkyline1); + final ResourceSkyline resourceSkyline2 = getSkyline(2); + addToStore(recurrenceId1, resourceSkyline2); + // test deleteHistory function of the in-memory store + skylineStore.deleteHistory(recurrenceId1); + } + + @Test(expected = NullRecurrenceIdException.class) + public final void testDeleteNullRecurrenceId() + throws SkylineStoreException { + final RecurrenceId recurrenceId1 = + new RecurrenceId("FraudDetection", "17/06/20 00:00:00"); + final ResourceSkyline resourceSkyline1 = getSkyline(1); + addToStore(recurrenceId1, resourceSkyline1); + // try to deleteHistory with null recurringId + skylineStore.deleteHistory(null); + } + + @Test(expected = RecurrenceIdNotFoundException.class) + public final void testDeleteRecurrenceIdNotFound() + throws SkylineStoreException { + final RecurrenceId recurrenceId1 = + new RecurrenceId("FraudDetection", "17/06/20 00:00:00"); + final ResourceSkyline resourceSkyline1 = getSkyline(1); + addToStore(recurrenceId1, resourceSkyline1); + final RecurrenceId recurrenceIdInvalid = + new RecurrenceId("Some random pipelineId", "Some random runId"); + // try to deleteHistory non-existing recurringId + skylineStore.deleteHistory(recurrenceIdInvalid); + } + + @Test public final void testUpdateNormal() throws SkylineStoreException { + // addHistory first recurring pipeline + final RecurrenceId recurrenceId1 = + new RecurrenceId("FraudDetection", "17/06/20 00:00:00"); + final ResourceSkyline resourceSkyline1 = getSkyline(1); + addToStore(recurrenceId1, resourceSkyline1); + final ArrayList resourceSkylines = + new ArrayList(); + final ResourceSkyline resourceSkyline2 = getSkyline(2); + resourceSkylines.add(resourceSkyline1); + resourceSkylines.add(resourceSkyline2); + skylineStore.updateHistory(recurrenceId1, resourceSkylines); + // query the in-memory store + final Map> jobHistory = + skylineStore.getHistory(recurrenceId1); + Assert.assertEquals(1, jobHistory.size()); + for (final Map.Entry> entry : jobHistory + .entrySet()) { + Assert.assertEquals(recurrenceId1, entry.getKey()); + final List getSkylines = entry.getValue(); + Assert.assertEquals(2, getSkylines.size()); + compare(resourceSkyline1, getSkylines.get(0)); + compare(resourceSkyline2, getSkylines.get(1)); + } + } + + @Test(expected = NullRecurrenceIdException.class) + public final void testUpdateNullRecurrenceId() + throws SkylineStoreException { + final ArrayList resourceSkylines = + new ArrayList(); + final ResourceSkyline resourceSkyline1 = getSkyline(1); + resourceSkylines.add(resourceSkyline1); + final ArrayList resourceSkylinesInvalid = + new ArrayList(); + resourceSkylinesInvalid.add(null); + // try to updateHistory with null recurringId + skylineStore.updateHistory(null, resourceSkylines); + } + + @Test(expected = NullResourceSkylineException.class) + public final void testUpdateNullResourceSkyline() + throws SkylineStoreException { + final RecurrenceId recurrenceId = + new RecurrenceId("FraudDetection", "17/06/20 00:00:00"); + final ArrayList resourceSkylines = + new ArrayList(); + final ResourceSkyline resourceSkyline1 = getSkyline(1); + resourceSkylines.add(resourceSkyline1); + final ArrayList resourceSkylinesInvalid = + new ArrayList(); + resourceSkylinesInvalid.add(null); + // try to updateHistory with null resourceSkylines + skylineStore.addHistory(recurrenceId, resourceSkylines); + skylineStore.updateHistory(recurrenceId, null); + } + + @Test(expected = EmptyResourceSkylineException.class) + public final void testUpdateEmptyRecurrenceId() + throws SkylineStoreException { + final RecurrenceId recurrenceId = + new RecurrenceId("FraudDetection", "17/06/20 00:00:00"); + final ArrayList resourceSkylines = + new ArrayList(); + final ResourceSkyline resourceSkyline1 = getSkyline(1); + resourceSkylines.add(resourceSkyline1); + final ArrayList resourceSkylinesInvalid = + new ArrayList(); + resourceSkylinesInvalid.add(null); + skylineStore.addHistory(recurrenceId, resourceSkylines); + // try to updateHistory with empty resourceSkyline + skylineStore.updateHistory(recurrenceId, resourceSkylinesInvalid); + } + + @Test(expected = RecurrenceIdNotFoundException.class) + public final void testUpdateRecurrenceIdNotFound() + throws SkylineStoreException { + final ArrayList resourceSkylines = + new ArrayList(); + final ResourceSkyline resourceSkyline1 = getSkyline(1); + resourceSkylines.add(resourceSkyline1); + final RecurrenceId recurrenceIdInvalid = + new RecurrenceId("Some random pipelineId", "Some random runId"); + final ArrayList resourceSkylinesInvalid = + new ArrayList(); + resourceSkylinesInvalid.add(null); + // try to updateHistory with non-existing recurringId + skylineStore.updateHistory(recurrenceIdInvalid, resourceSkylines); + } + + @After public final void cleanUp() { + skylineStore = null; + resourceOverTime.clear(); + resourceOverTime = null; + skylineList = null; + riAdd = null; + resource = null; + } +} diff --git a/hadoop-tools/hadoop-resourceestimator/src/test/java/org/apache/hadoop/resourceestimator/solver/impl/TestLpSolver.java b/hadoop-tools/hadoop-resourceestimator/src/test/java/org/apache/hadoop/resourceestimator/solver/impl/TestLpSolver.java new file mode 100644 index 0000000000..d32f7c3592 --- /dev/null +++ b/hadoop-tools/hadoop-resourceestimator/src/test/java/org/apache/hadoop/resourceestimator/solver/impl/TestLpSolver.java @@ -0,0 +1,112 @@ +/* + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package org.apache.hadoop.resourceestimator.solver.impl; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.resourceestimator.common.api.RecurrenceId; +import org.apache.hadoop.resourceestimator.common.api.ResourceSkyline; +import org.apache.hadoop.resourceestimator.common.config.ResourceEstimatorConfiguration; +import org.apache.hadoop.resourceestimator.common.exception.ResourceEstimatorException; +import org.apache.hadoop.resourceestimator.skylinestore.exceptions.SkylineStoreException; +import org.apache.hadoop.resourceestimator.skylinestore.impl.InMemoryStore; +import org.apache.hadoop.resourceestimator.solver.api.Solver; +import org.apache.hadoop.resourceestimator.solver.exceptions.SolverException; +import org.apache.hadoop.resourceestimator.translator.api.LogParser; +import org.apache.hadoop.resourceestimator.translator.exceptions.DataFieldNotFoundException; +import org.apache.hadoop.resourceestimator.translator.impl.BaseLogParser; +import org.apache.hadoop.yarn.server.resourcemanager.reservation.RLESparseResourceAllocation; +import org.junit.Test; + +import java.io.BufferedReader; +import java.io.FileInputStream; +import java.io.IOException; +import java.io.InputStream; +import java.io.InputStreamReader; +import java.io.Reader; +import java.nio.charset.Charset; +import java.text.ParseException; +import java.util.List; +import java.util.Map; + +import static org.junit.Assert.assertEquals; + +/** + * This LPSolver class will make resource estimation using Linear Programming + * model. We use ojAlgo solver to solve the model. + */ +public class TestLpSolver extends TestSolver { + private static final LogParser SAMPLEPARSER = new BaseLogParser(); + private Solver solver; + private InMemoryStore skylineStore = new InMemoryStore(); + + private void parseLog(final String inputLog) + throws SolverException, IOException, SkylineStoreException, + DataFieldNotFoundException, ParseException { + final InputStream logs = new FileInputStream(inputLog); + SAMPLEPARSER.parseStream(logs); + } + + @Override protected Solver createSolver() throws ResourceEstimatorException { + solver = new LpSolver(); + Configuration config = new Configuration(); + config.addResource(ResourceEstimatorConfiguration.CONFIG_FILE); + solver.init(config, skylineStore); + SAMPLEPARSER.init(config, skylineStore); + return solver; + } + + @Test public void testSolve() + throws IOException, SkylineStoreException, SolverException, + ResourceEstimatorException, DataFieldNotFoundException, ParseException { + parseLog("src/test/resources/lp/tpch_q12.txt"); + RecurrenceId recurrenceId = new RecurrenceId("tpch_q12", "*"); + final Map> jobHistory = + skylineStore.getHistory(recurrenceId); + solver = createSolver(); + RLESparseResourceAllocation result = solver.solve(jobHistory); + String file = "src/test/resources/lp/answer.txt"; + Reader fileReader = new InputStreamReader(new FileInputStream(file), + Charset.forName("UTF-8")); + BufferedReader bufferedReader = new BufferedReader(fileReader); + String line = bufferedReader.readLine(); + Configuration config = new Configuration(); + config.addResource(new org.apache.hadoop.fs.Path( + ResourceEstimatorConfiguration.CONFIG_FILE)); + int timeInterval = + config.getInt(ResourceEstimatorConfiguration.TIME_INTERVAL_KEY, 5); + final long containerMemAlloc = + jobHistory.entrySet().iterator().next().getValue().get(0) + .getContainerSpec().getMemorySize(); + int count = 0; + int numContainer = 0; + while (line != null) { + numContainer = + (int) (result.getCapacityAtTime(count * timeInterval).getMemorySize() + / containerMemAlloc); + assertEquals(Integer.parseInt(line), numContainer, + 0.1 * Integer.parseInt(line)); + line = bufferedReader.readLine(); + count++; + } + fileReader.close(); + bufferedReader.close(); + } +} diff --git a/hadoop-tools/hadoop-resourceestimator/src/test/java/org/apache/hadoop/resourceestimator/solver/impl/TestSolver.java b/hadoop-tools/hadoop-resourceestimator/src/test/java/org/apache/hadoop/resourceestimator/solver/impl/TestSolver.java new file mode 100644 index 0000000000..11df6cde4c --- /dev/null +++ b/hadoop-tools/hadoop-resourceestimator/src/test/java/org/apache/hadoop/resourceestimator/solver/impl/TestSolver.java @@ -0,0 +1,73 @@ +/* + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package org.apache.hadoop.resourceestimator.solver.impl; + +import java.io.IOException; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import org.apache.hadoop.resourceestimator.common.api.RecurrenceId; +import org.apache.hadoop.resourceestimator.common.api.ResourceSkyline; +import org.apache.hadoop.resourceestimator.common.exception.ResourceEstimatorException; +import org.apache.hadoop.resourceestimator.skylinestore.exceptions.SkylineStoreException; +import org.apache.hadoop.resourceestimator.solver.api.Solver; +import org.apache.hadoop.resourceestimator.solver.exceptions.InvalidInputException; +import org.apache.hadoop.resourceestimator.solver.exceptions.SolverException; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +/** + * This LPSolver class will make resource estimation using Linear Programming + * model. We use Google Or Tool to solve the model. + */ +public abstract class TestSolver { + private Solver solver; + + protected abstract Solver createSolver() throws ResourceEstimatorException; + + @Before public void setup() + throws SolverException, IOException, SkylineStoreException, + ResourceEstimatorException { + solver = createSolver(); + } + + @Test(expected = InvalidInputException.class) public void testNullJobHistory() + throws SolverException, SkylineStoreException { + // try to solve with null jobHistory + solver.solve(null); + } + + @Test(expected = InvalidInputException.class) + public void testEmptyJobHistory() + throws SolverException, SkylineStoreException { + Map> jobHistoryInvalid = + new HashMap>(); + // try to solve with emty jobHistory + solver.solve(jobHistoryInvalid); + } + + @After public final void cleanUp() { + solver.close(); + solver = null; + } +} diff --git a/hadoop-tools/hadoop-resourceestimator/src/test/java/org/apache/hadoop/resourceestimator/translator/api/TestJobMetaData.java b/hadoop-tools/hadoop-resourceestimator/src/test/java/org/apache/hadoop/resourceestimator/translator/api/TestJobMetaData.java new file mode 100644 index 0000000000..69ba48058a --- /dev/null +++ b/hadoop-tools/hadoop-resourceestimator/src/test/java/org/apache/hadoop/resourceestimator/translator/api/TestJobMetaData.java @@ -0,0 +1,163 @@ +/* + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package org.apache.hadoop.resourceestimator.translator.api; + +import java.text.ParseException; + +import org.apache.hadoop.resourceestimator.common.api.RecurrenceId; +import org.apache.hadoop.resourceestimator.translator.impl.LogParserUtil; +import org.apache.hadoop.yarn.api.records.Resource; +import org.apache.hadoop.yarn.server.resourcemanager.reservation.RLESparseResourceAllocation; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +/** + * Test JobMetaData. + */ +public class TestJobMetaData { + /** + * TODO: parametrize this test. + */ + private LogParserUtil logParserUtil = new LogParserUtil(); + + private JobMetaData jobMetaData; + private RecurrenceId recurrenceId; + + @Before public final void setup() throws ParseException { + recurrenceId = new RecurrenceId("Fraud Detection", "17/07/16 16:27:25"); + jobMetaData = new JobMetaData( + logParserUtil.stringToUnixTimestamp("17/07/16 16:27:25")); + jobMetaData.setRecurrenceId(recurrenceId); + jobMetaData.setContainerStart("C1", + logParserUtil.stringToUnixTimestamp("17/07/16 16:27:30")); + jobMetaData.setContainerEnd("C1", + logParserUtil.stringToUnixTimestamp("17/07/16 16:37:30")); + jobMetaData.setContainerStart("C2", + logParserUtil.stringToUnixTimestamp("17/07/16 16:27:40")); + jobMetaData.setContainerEnd("C2", + logParserUtil.stringToUnixTimestamp("17/07/16 16:37:40")); + jobMetaData.setJobFinishTime( + logParserUtil.stringToUnixTimestamp("17/07/16 16:37:45")); + final Resource containerAlloc = Resource.newInstance(1, 1); + jobMetaData.getResourceSkyline().setContainerSpec(containerAlloc); + jobMetaData.getResourceSkyline().setJobInputDataSize(1024.5); + jobMetaData.createSkyline(); + } + + @Test public final void testGetContainerSpec() { + final Resource containerAlloc = + jobMetaData.getResourceSkyline().getContainerSpec(); + final Resource containerAlloc2 = Resource.newInstance(1, 1); + Assert.assertEquals(containerAlloc.getMemorySize(), + containerAlloc2.getMemorySize()); + Assert.assertEquals(containerAlloc.getVirtualCores(), + containerAlloc2.getVirtualCores()); + } + + @Test public final void testGetJobSize() { + Assert.assertEquals(jobMetaData.getResourceSkyline().getJobInputDataSize(), + 1024.5, 0); + } + + @Test public final void testGetRecurrenceeId() { + final RecurrenceId recurrenceIdTest = + new RecurrenceId("Fraud Detection", "17/07/16 16:27:25"); + Assert.assertEquals(recurrenceIdTest, jobMetaData.getRecurrenceId()); + } + + @Test public final void testStringToUnixTimestamp() throws ParseException { + final long submissionTime = + logParserUtil.stringToUnixTimestamp("17/07/16 16:27:25"); + Assert.assertEquals(jobMetaData.getResourceSkyline().getJobSubmissionTime(), + submissionTime); + } + + @Test public final void testResourceSkyline() { + final RLESparseResourceAllocation skylineList = + jobMetaData.getResourceSkyline().getSkylineList(); + final int containerCPU = + jobMetaData.getResourceSkyline().getContainerSpec().getVirtualCores(); + int k; + for (k = 0; k < 5; k++) { + Assert.assertEquals(0, + skylineList.getCapacityAtTime(k).getVirtualCores() / containerCPU); + } + for (k = 5; k < 15; k++) { + Assert.assertEquals(1, + skylineList.getCapacityAtTime(k).getVirtualCores() / containerCPU); + } + for (k = 15; k < 605; k++) { + Assert.assertEquals(2, + skylineList.getCapacityAtTime(k).getVirtualCores() / containerCPU); + } + for (k = 605; k < 615; k++) { + Assert.assertEquals(1, + skylineList.getCapacityAtTime(k).getVirtualCores() / containerCPU); + } + Assert.assertEquals(0, + skylineList.getCapacityAtTime(615).getVirtualCores() / containerCPU); + } + + @Test public final void testContainerReleaseTimeMissing() + throws ParseException { + // create an invalid JobMetaData + recurrenceId = new RecurrenceId("Fraud Detection", "17/07/16 16:27:25"); + jobMetaData = new JobMetaData( + logParserUtil.stringToUnixTimestamp("17/07/16 16:27:25")); + jobMetaData.setRecurrenceId(recurrenceId); + jobMetaData.setContainerStart("C1", + logParserUtil.stringToUnixTimestamp("17/07/16 16:27:30")); + jobMetaData.setContainerEnd("C1", + logParserUtil.stringToUnixTimestamp("17/07/16 16:37:30")); + jobMetaData.setContainerStart("C2", + logParserUtil.stringToUnixTimestamp("17/07/16 16:27:40")); + jobMetaData.setJobFinishTime( + logParserUtil.stringToUnixTimestamp("17/07/16 16:37:45")); + final Resource containerAlloc = Resource.newInstance(1, 1); + jobMetaData.getResourceSkyline().setContainerSpec(containerAlloc); + jobMetaData.getResourceSkyline().setJobInputDataSize(1024.5); + jobMetaData.createSkyline(); + // test the generated ResourceSkyline + final RLESparseResourceAllocation skylineList = + jobMetaData.getResourceSkyline().getSkylineList(); + final int containerCPU = + jobMetaData.getResourceSkyline().getContainerSpec().getVirtualCores(); + int k; + for (k = 0; k < 5; k++) { + Assert.assertEquals(0, + skylineList.getCapacityAtTime(k).getVirtualCores() / containerCPU); + } + for (k = 5; k < 605; k++) { + Assert.assertEquals(1, + skylineList.getCapacityAtTime(k).getVirtualCores() / containerCPU); + } + Assert.assertEquals(0, + skylineList.getCapacityAtTime(605).getVirtualCores() / containerCPU); + } + + @After public final void cleanUp() { + jobMetaData = null; + recurrenceId = null; + logParserUtil = null; + } +} diff --git a/hadoop-tools/hadoop-resourceestimator/src/test/java/org/apache/hadoop/resourceestimator/translator/impl/TestNativeParser.java b/hadoop-tools/hadoop-resourceestimator/src/test/java/org/apache/hadoop/resourceestimator/translator/impl/TestNativeParser.java new file mode 100644 index 0000000000..f399bf944b --- /dev/null +++ b/hadoop-tools/hadoop-resourceestimator/src/test/java/org/apache/hadoop/resourceestimator/translator/impl/TestNativeParser.java @@ -0,0 +1,115 @@ +/* + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package org.apache.hadoop.resourceestimator.translator.impl; + +import java.io.IOException; +import java.text.ParseException; +import java.util.List; +import java.util.Map; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.resourceestimator.common.api.RecurrenceId; +import org.apache.hadoop.resourceestimator.common.api.ResourceSkyline; +import org.apache.hadoop.resourceestimator.common.config.ResourceEstimatorConfiguration; +import org.apache.hadoop.resourceestimator.common.exception.ResourceEstimatorException; +import org.apache.hadoop.resourceestimator.skylinestore.api.SkylineStore; +import org.apache.hadoop.resourceestimator.skylinestore.exceptions.SkylineStoreException; +import org.apache.hadoop.resourceestimator.skylinestore.impl.InMemoryStore; +import org.apache.hadoop.resourceestimator.translator.api.LogParser; +import org.apache.hadoop.resourceestimator.translator.exceptions.DataFieldNotFoundException; +import org.apache.hadoop.yarn.server.resourcemanager.reservation.RLESparseResourceAllocation; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +/** + * This sample parser will parse the sample log and extract the resource + * skyline. + */ +public class TestNativeParser { + private LogParserUtil logParserUtil = new LogParserUtil(); + private SkylineStore skylineStore; + + @Before public final void setup() throws ResourceEstimatorException { + skylineStore = new InMemoryStore(); + final LogParser nativeParser = new BaseLogParser(); + Configuration config = new Configuration(); + config.addResource(ResourceEstimatorConfiguration.CONFIG_FILE); + nativeParser.init(config, skylineStore); + logParserUtil.setLogParser(nativeParser); + } + + private void parseFile(final String logFile) + throws SkylineStoreException, IOException, ParseException, + ResourceEstimatorException, DataFieldNotFoundException { + logParserUtil.parseLog(logFile); + } + + @Test public final void testParse() + throws SkylineStoreException, IOException, ParseException, + ResourceEstimatorException, DataFieldNotFoundException { + final String logFile = "src/test/resources/trace/nativeLog.txt"; + parseFile(logFile); + final RecurrenceId recurrenceId = + new RecurrenceId("tpch_q12", "tpch_q12_0"); + final Map> jobSkylineLists = + skylineStore.getHistory(recurrenceId); + Assert.assertEquals(1, jobSkylineLists.size()); + final List jobHistory = jobSkylineLists.get(recurrenceId); + Assert.assertEquals(1, jobHistory.size()); + final ResourceSkyline resourceSkyline = jobHistory.get(0); + Assert.assertEquals(0, resourceSkyline.getJobInputDataSize(), 0); + Assert.assertEquals("tpch_q12_0", resourceSkyline.getJobId()); + Assert.assertEquals(0, resourceSkyline.getJobSubmissionTime()); + Assert.assertEquals(25, resourceSkyline.getJobFinishTime()); + Assert + .assertEquals(1024, resourceSkyline.getContainerSpec().getMemorySize()); + Assert + .assertEquals(1, resourceSkyline.getContainerSpec().getVirtualCores()); + final RLESparseResourceAllocation skylineLists = + resourceSkyline.getSkylineList(); + int k; + for (k = 0; k < 10; k++) { + Assert.assertEquals(1, + skylineLists.getCapacityAtTime(k).getMemorySize() / 1024); + } + for (k = 10; k < 15; k++) { + Assert.assertEquals(1074, + skylineLists.getCapacityAtTime(k).getMemorySize() / 1024); + } + for (k = 15; k < 20; k++) { + Assert.assertEquals(2538, + skylineLists.getCapacityAtTime(k).getMemorySize() / 1024); + } + for (k = 20; k < 25; k++) { + Assert.assertEquals(2468, + skylineLists.getCapacityAtTime(k).getMemorySize() / 1024); + } + Assert.assertEquals(0, + skylineLists.getCapacityAtTime(25).getMemorySize() / 1024); + } + + @After public final void cleanUp() { + skylineStore = null; + logParserUtil = null; + } +} diff --git a/hadoop-tools/hadoop-resourceestimator/src/test/java/org/apache/hadoop/resourceestimator/translator/impl/TestRmParser.java b/hadoop-tools/hadoop-resourceestimator/src/test/java/org/apache/hadoop/resourceestimator/translator/impl/TestRmParser.java new file mode 100644 index 0000000000..4a39ed7cf4 --- /dev/null +++ b/hadoop-tools/hadoop-resourceestimator/src/test/java/org/apache/hadoop/resourceestimator/translator/impl/TestRmParser.java @@ -0,0 +1,239 @@ +/* + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package org.apache.hadoop.resourceestimator.translator.impl; + +import java.io.IOException; +import java.text.ParseException; +import java.util.List; +import java.util.Map; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.resourceestimator.common.api.RecurrenceId; +import org.apache.hadoop.resourceestimator.common.api.ResourceSkyline; +import org.apache.hadoop.resourceestimator.common.config.ResourceEstimatorConfiguration; +import org.apache.hadoop.resourceestimator.common.exception.ResourceEstimatorException; +import org.apache.hadoop.resourceestimator.skylinestore.api.SkylineStore; +import org.apache.hadoop.resourceestimator.skylinestore.exceptions.SkylineStoreException; +import org.apache.hadoop.resourceestimator.skylinestore.impl.InMemoryStore; +import org.apache.hadoop.resourceestimator.translator.api.LogParser; +import org.apache.hadoop.resourceestimator.translator.exceptions.DataFieldNotFoundException; +import org.apache.hadoop.yarn.api.records.Resource; +import org.apache.hadoop.yarn.server.resourcemanager.reservation.RLESparseResourceAllocation; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +/** + * This sample parser will parse the sample log and extract the resource + * skyline. + */ +public class TestRmParser { + private LogParserUtil logParserUtil = new LogParserUtil(); + private SkylineStore skylineStore; + + @Before public final void setup() throws ResourceEstimatorException { + skylineStore = new InMemoryStore(); + final LogParser rmParser = new BaseLogParser(); + Configuration config = new Configuration(); + config.addResource(new org.apache.hadoop.fs.Path( + ResourceEstimatorConfiguration.CONFIG_FILE)); + config.set(ResourceEstimatorConfiguration.TRANSLATOR_LINE_PARSER, + RmSingleLineParser.class.getName()); + rmParser.init(config, skylineStore); + logParserUtil.setLogParser(rmParser); + } + + private void parseFile(final String logFile) + throws SkylineStoreException, IOException, ParseException, + ResourceEstimatorException, DataFieldNotFoundException { + logParserUtil.parseLog(logFile); + } + + @Test public final void testParse() + throws SkylineStoreException, IOException, ParseException, + ResourceEstimatorException, DataFieldNotFoundException { + final String logFile = "src/test/resources/trace/rmLog.txt"; + parseFile(logFile); + final RecurrenceId recurrenceId = new RecurrenceId("FraudDetection", "1"); + final Map> jobSkylineLists = + skylineStore.getHistory(recurrenceId); + Assert.assertEquals(1, jobSkylineLists.size()); + final List jobHistory = jobSkylineLists.get(recurrenceId); + Assert.assertEquals(1, jobHistory.size()); + final ResourceSkyline resourceSkyline = jobHistory.get(0); + Assert.assertEquals(0, resourceSkyline.getJobInputDataSize(), 0); + Assert.assertEquals("application_1497832133857_0330", + resourceSkyline.getJobId()); + Assert.assertEquals( + logParserUtil.stringToUnixTimestamp("06/21/2017 16:10:13"), + resourceSkyline.getJobSubmissionTime()); + Assert.assertEquals( + logParserUtil.stringToUnixTimestamp("06/21/2017 16:18:35"), + resourceSkyline.getJobFinishTime()); + final Resource resource = Resource.newInstance(1800, 1); + Assert.assertEquals(resource.getMemorySize(), + resourceSkyline.getContainerSpec().getMemorySize()); + Assert.assertEquals(resource.getVirtualCores(), + resourceSkyline.getContainerSpec().getVirtualCores()); + final RLESparseResourceAllocation skylineLists = + resourceSkyline.getSkylineList(); + + int k; + for (k = 0; k < 142; k++) { + Assert.assertEquals(1, + skylineLists.getCapacityAtTime(k).getMemorySize() / resource + .getMemorySize()); + } + for (k = 142; k < 345; k++) { + Assert.assertEquals(2, + skylineLists.getCapacityAtTime(k).getMemorySize() / resource + .getMemorySize()); + } + for (k = 345; k < 502; k++) { + Assert.assertEquals(1, + skylineLists.getCapacityAtTime(k).getMemorySize() / resource + .getMemorySize()); + } + } + + @Test(expected = ParseException.class) + public final void testInvalidDateFormat() + throws ParseException { + logParserUtil.stringToUnixTimestamp("2017.07.16 16:37:45"); + } + + @Test public final void testDuplicateJobSubmissionTime() + throws SkylineStoreException, IOException, ParseException, + ResourceEstimatorException, DataFieldNotFoundException { + final String logFile = "src/test/resources/trace/invalidLog1.txt"; + parseFile(logFile); + final RecurrenceId recurrenceId = new RecurrenceId("Test", "1"); + final Map> jobSkylineLists = + skylineStore.getHistory(recurrenceId); + Assert.assertEquals( + logParserUtil.stringToUnixTimestamp("06/21/2017 16:10:23"), + jobSkylineLists.get(recurrenceId).get(0).getJobSubmissionTime()); + } + + @Test public final void testJobIdNotFoundInJobSubmission() + throws SkylineStoreException, IOException, ParseException, + ResourceEstimatorException, DataFieldNotFoundException { + final String logFile = "src/test/resources/trace/invalidLog2.txt"; + parseFile(logFile); + final RecurrenceId recurrenceId = new RecurrenceId("Test", "2"); + Assert.assertNull(skylineStore.getHistory(recurrenceId)); + } + + @Test public final void testJobIdNotFoundInContainerAlloc() + throws SkylineStoreException, IOException, ParseException, + ResourceEstimatorException, DataFieldNotFoundException { + final String logFile = "src/test/resources/trace/invalidLog3.txt"; + parseFile(logFile); + final RecurrenceId recurrenceId = new RecurrenceId("Test", "3"); + final Map> jobSkylineLists = + skylineStore.getHistory(recurrenceId); + Assert.assertEquals(0, + jobSkylineLists.get(recurrenceId).get(0).getSkylineList() + .getCumulative().size()); + } + + @Test public final void testContainerIdNotFoundInContainerAlloc() + throws SkylineStoreException, IOException, ParseException, + ResourceEstimatorException, DataFieldNotFoundException { + final String logFile = "src/test/resources/trace/invalidLog4.txt"; + parseFile(logFile); + final RecurrenceId recurrenceId = new RecurrenceId("Test", "4"); + final Map> jobSkylineLists = + skylineStore.getHistory(recurrenceId); + Assert.assertEquals(0, + jobSkylineLists.get(recurrenceId).get(0).getSkylineList() + .getCumulative().size()); + } + + @Test public final void testJobIdNotFoundInJobFailure() + throws SkylineStoreException, IOException, ParseException, + ResourceEstimatorException, DataFieldNotFoundException { + final String logFile = "src/test/resources/trace/invalidLog5.txt"; + parseFile(logFile); + final RecurrenceId recurrenceId = new RecurrenceId("Test", "5"); + final Map> jobSkylineLists = + skylineStore.getHistory(recurrenceId); + Assert.assertEquals( + logParserUtil.stringToUnixTimestamp("06/21/2017 16:10:13"), + jobSkylineLists.get(recurrenceId).get(0).getJobSubmissionTime()); + } + + @Test public final void testJobIdNotFoundInJobFinish() + throws SkylineStoreException, IOException, ParseException, + ResourceEstimatorException, DataFieldNotFoundException { + final String logFile = "src/test/resources/trace/invalidLog6.txt"; + parseFile(logFile); + final RecurrenceId recurrenceId = new RecurrenceId("Test", "6"); + Assert.assertNull(skylineStore.getHistory(recurrenceId)); + } + + @Test public final void testRecurrenceIdNotFoundInJobFinish() + throws SkylineStoreException, IOException, ParseException, + ResourceEstimatorException, DataFieldNotFoundException { + final String logFile = "src/test/resources/trace/invalidLog7.txt"; + parseFile(logFile); + final RecurrenceId recurrenceId = new RecurrenceId("Test", "7"); + Assert.assertNull(skylineStore.getHistory(recurrenceId)); + } + + @Test public final void testJobIdNotFoundInResourceSpec() + throws SkylineStoreException, IOException, ParseException, + ResourceEstimatorException, DataFieldNotFoundException { + final String logFile = "src/test/resources/trace/invalidLog8.txt"; + parseFile(logFile); + final RecurrenceId recurrenceId = new RecurrenceId("Test", "8"); + final Map> jobSkylineLists = + skylineStore.getHistory(recurrenceId); + Assert.assertEquals(1024, + jobSkylineLists.get(recurrenceId).get(0).getContainerSpec() + .getMemorySize()); + Assert.assertEquals(1, + jobSkylineLists.get(recurrenceId).get(0).getContainerSpec() + .getVirtualCores()); + } + + @Test public final void testResourceSpecNotFoundInResourceSpec() + throws SkylineStoreException, IOException, ParseException, + ResourceEstimatorException, DataFieldNotFoundException { + final String logFile = "src/test/resources/trace/invalidLog9.txt"; + parseFile(logFile); + final RecurrenceId recurrenceId = new RecurrenceId("Test", "9"); + final Map> jobSkylineLists = + skylineStore.getHistory(recurrenceId); + Assert.assertEquals(1024, + jobSkylineLists.get(recurrenceId).get(0).getContainerSpec() + .getMemorySize()); + Assert.assertEquals(1, + jobSkylineLists.get(recurrenceId).get(0).getContainerSpec() + .getVirtualCores()); + } + + @After public final void cleanUp() { + skylineStore = null; + logParserUtil = null; + } +} diff --git a/hadoop-tools/hadoop-resourceestimator/src/test/resources/log4j.properties b/hadoop-tools/hadoop-resourceestimator/src/test/resources/log4j.properties new file mode 100644 index 0000000000..60b8fbed16 --- /dev/null +++ b/hadoop-tools/hadoop-resourceestimator/src/test/resources/log4j.properties @@ -0,0 +1,25 @@ +# +# +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# +# +log4j.rootLogger=INFO, stdout +# Redirect log messages to console +log4j.appender.stdout=org.apache.log4j.ConsoleAppender +log4j.appender.stdout.Target=System.out +log4j.appender.stdout.layout=org.apache.log4j.PatternLayout +log4j.appender.stdout.layout.ConversionPattern=%d{yyyy-MM-dd HH:mm:ss} %-5p %c{1}:%L - %m%n diff --git a/hadoop-tools/hadoop-resourceestimator/src/test/resources/lp/answer.txt b/hadoop-tools/hadoop-resourceestimator/src/test/resources/lp/answer.txt new file mode 100644 index 0000000000..94565dee5c --- /dev/null +++ b/hadoop-tools/hadoop-resourceestimator/src/test/resources/lp/answer.txt @@ -0,0 +1,7 @@ +1 +1 +1738 +2535 +2484 +2478 +2468 \ No newline at end of file diff --git a/hadoop-tools/hadoop-resourceestimator/src/test/resources/lp/tinySample.txt b/hadoop-tools/hadoop-resourceestimator/src/test/resources/lp/tinySample.txt new file mode 100644 index 0000000000..15fb84e98c --- /dev/null +++ b/hadoop-tools/hadoop-resourceestimator/src/test/resources/lp/tinySample.txt @@ -0,0 +1,2 @@ +tpch_q12 50 1462148517000 1462148527345 1462148952345 tpch_q12_0 8192:0| 1:0|1:5|1074:10|2538:15|2468:20| +tpch_q12 50 1462148949000 1462148960455 1462149205455 tpch_q12_1 8192:0| 1:0|1:5|794:10|2517:15|2484:20| \ No newline at end of file diff --git a/hadoop-tools/hadoop-resourceestimator/src/test/resources/lp/tpch_q12.txt b/hadoop-tools/hadoop-resourceestimator/src/test/resources/lp/tpch_q12.txt new file mode 100644 index 0000000000..ee19625077 --- /dev/null +++ b/hadoop-tools/hadoop-resourceestimator/src/test/resources/lp/tpch_q12.txt @@ -0,0 +1,13 @@ +tpch_q12 50 1462148517000 1462148527345 1462148952345 tpch_q12_0 8192:0|8192:5|8798208:10|20791296:15|20348928:20|20275200:25|20217856:30|20160512:35|20045824:40|18554880:45|15736832:50|15302656:55|14991360:60|14737408:65|14508032:70|14360576:75|14196736:80|14049280:85|13901824:90|13795328:95|13721600:100|13664256:105|13565952:110|12877824:115|12148736:120|11476992:125|10829824:130|10444800:135|10059776:140|9633792:145|9355264:150|9125888:155|9027584:160|8937472:165|8839168:170|8757248:175|8642560:180|8527872:185|8421376:190|8396800:195|8355840:200|8323072:205|8298496:210|8282112:215|8282112:220|8273920:225|8273920:230|8273920:235|8273920:240|8273920:245|8273920:250|8273920:255|7700480:260|5160960:265|4087808:270|3907584:275|3702784:280|3407872:285|2965504:290|2686976:295|2473984:300|2244608:305|2056192:310|1933312:315|1810432:320|1613824:325|1417216:330|1277952:335|1130496:340|843776:345|679936:350|647168:355|630784:360|630784:365|630784:370|614400:375|581632:380|548864:385|540672:390|540672:395|540672:400|507904:405|491520:410|278528:415|16384:420|0:425| 1:0|1:5|1074:10|2538:15|2484:20|2475:25|2468:30| +tpch_q12 50 1462148949000 1462148960455 1462149205455 tpch_q12_1 8192:0|8192:5|6504448:10|20619264:15|20348928:20|20299776:25|20201472:30|20127744:35|20021248:40|18366464:45|15392768:50|15605760:55|15400960:60|16064512:65|13664256:70|15753216:75|15843328:80|15843328:85|13615104:90|11943936:95|11501568:100|10895360:105|10223616:110|9822208:115|9478144:120|9142272:125|8855552:130|8683520:135|8544256:140|8413184:145|8331264:150|8290304:155|8282112:160|8273920:165|8273920:170|8273920:175|8273920:180|8273920:185|8273920:190|8265728:195|4890624:200|1089536:205|671744:210|565248:215|516096:220|483328:225|360448:230|16384:235|16384:240|8192:245|0:250| 1:0|1:5|794:10|2517:15|2484:20|2478:25|2466:30| +tpch_q12 50 1462150611000 1462150620642 1462150895642 tpch_q12_7 8192:0|8192:5|11362304:10|20365312:15|20348928:20|20242432:25|20176896:30|20029440:35|19783680:40|16703488:45|15286272:50|14589952:55|14172160:60|13770752:65|13598720:70|12992512:75|12468224:80|12025856:85|11640832:90|11460608:95|11272192:100|10510336:105|9789440:110|9207808:115|8781824:120|8585216:125|8478720:130|8396800:135|8314880:140|8290304:145|8273920:150|8273920:155|8273920:160|8273920:165|8273920:170|8273920:175|8273920:180|7077888:185|4407296:190|3735552:195|3260416:200|2736128:205|2228224:210|1777664:215|1433600:220|1155072:225|983040:230|843776:235|720896:240|647168:245|630784:250|622592:255|565248:260|303104:265|16384:270|8192:275|0:280| 1:0|1:5|1387:10|2486:15|2484:20|2471:25|2463:30| +tpch_q12 50 1462150893000 1462150904200 1462151134200 tpch_q12_8 8192:0|8192:5|7946240:10|20389888:15|20348928:20|20267008:25|20160512:30|20054016:35|19824640:40|17866752:45|14041088:50|14114816:55|12615680:60|14336000:65|12771328:70|12189696:75|11796480:80|11599872:85|11517952:90|11460608:95|11403264:100|11116544:105|10010624:110|9306112:115|8847360:120|8593408:125|8404992:130|8331264:135|8298496:140|8290304:145|8282112:150|8282112:155|8282112:160|8282112:165|8282112:170|8282112:175|8273920:180|8273920:185|8273920:190|8273920:195|8273920:200|8273920:205|8118272:210|2883584:215|376832:220|16384:225|8192:230|0:235| 1:0|1:5|970:10|2489:15|2484:20|2474:25|2461:30| +tpch_q12 50 1462151132000 1462151141241 1462151421241 tpch_q12_9 8192:0|16384:5|16031744:10|20348928:15|20348928:20|20201472:25|20152320:30|20004864:35|19562496:40|16121856:45|15319040:50|14589952:55|14139392:60|13770752:65|13131776:70|12378112:75|11878400:80|11649024:85|11534336:90|11444224:95|11157504:100|10362880:105|9650176:110|9134080:115|8749056:120|8519680:125|8372224:130|8306688:135|8290304:140|8282112:145|8282112:150|8273920:155|8273920:160|8273920:165|8273920:170|8273920:175|8273920:180|8273920:185|7569408:190|4505600:195|3522560:200|3072000:205|2801664:210|2523136:215|2080768:220|1679360:225|1343488:230|1007616:235|892928:240|819200:245|688128:250|573440:255|540672:260|524288:265|491520:270|24576:275|8192:280|0:285| 1:0|2:5|1957:10|2484:15|2484:20|2466:25|2460:30| +tpch_q12 50 1462151420000 1462151429196 1462151739196 tpch_q12_10 8192:0|8192:5|10543104:10|20357120:15|20348928:20|20234240:25|20168704:30|20045824:35|19750912:40|17293312:45|15327232:50|14770176:55|14336000:60|13926400:65|13410304:70|12517376:75|12034048:80|11714560:85|11526144:90|11427840:95|11247616:100|10616832:105|9928704:110|9412608:115|8962048:120|8585216:125|8380416:130|8323072:135|8290304:140|8282112:145|8282112:150|8282112:155|8273920:160|8273920:165|8273920:170|8273920:175|8273920:180|8273920:185|8273920:190|6889472:195|4120576:200|3465216:205|3153920:210|2924544:215|2514944:220|2056192:225|1794048:230|1646592:235|1417216:240|1122304:245|1048576:250|1007616:255|892928:260|778240:265|647168:270|557056:275|532480:280|524288:285|507904:290|475136:295|229376:300|16384:305|8192:310|0:315| 1:0|1:5|1287:10|2485:15|2484:20|2470:25|2462:30| +tpch_q12 50 1462151735000 1462151744905 1462152069905 tpch_q12_11 8192:0|8192:5|9494528:10|20389888:15|20348928:20|20267008:25|20168704:30|20086784:35|19652608:40|17334272:45|15351808:50|15040512:55|14639104:60|14295040:65|13967360:70|13672448:75|12853248:80|12181504:85|11730944:90|11493376:95|11255808:100|10592256:105|10002432:110|9633792:115|9224192:120|8896512:125|8634368:130|8462336:135|8355840:140|8298496:145|8282112:150|8273920:155|8273920:160|8273920:165|8273920:170|8273920:175|8273920:180|8142848:185|5718016:190|4440064:195|3989504:200|3604480:205|3334144:210|2981888:215|2572288:220|2269184:225|1966080:230|1744896:235|1507328:240|1310720:245|1089536:250|942080:255|851968:260|794624:265|745472:270|655360:275|598016:280|565248:285|548864:290|532480:295|532480:300|532480:305|491520:310|270336:315|16384:320|8192:325|0:330| 1:0|1:5|1159:10|2489:15|2484:20|2474:25|2462:30| +tpch_q12 50 1462152067000 1462152075862 1462152365862 tpch_q12_12 8192:0|8192:5|11583488:10|20357120:15|20348928:20|20234240:25|20168704:30|20037632:35|19734528:40|16744448:45|15351808:50|14704640:55|14229504:60|13901824:65|13631488:70|13107200:75|12500992:80|12140544:85|11804672:90|11558912:95|11386880:100|10723328:105|9928704:110|9355264:115|8937472:120|8691712:125|8519680:130|8421376:135|8372224:140|8339456:145|8306688:150|8273920:155|8273920:160|8273920:165|8273920:170|8273920:175|8273920:180|8273920:185|7577600:190|4898816:195|3719168:200|3104768:205|2613248:210|2252800:215|2064384:220|1835008:225|1515520:230|1171456:235|974848:240|892928:245|802816:250|729088:255|712704:260|663552:265|557056:270|483328:275|368640:280|16384:285|8192:290|0:295| 1:0|1:5|1414:10|2485:15|2484:20|2470:25|2462:30| +tpch_q12 50 1462161466000 1462161474815 1462161689815 tpch_q12_45 8192:0|8192:5|13836288:10|20365312:15|20348928:20|20209664:25|20160512:30|20029440:35|19906560:40|18333696:45|15794176:50|16015360:55|13451264:60|15269888:65|12730368:70|11862016:75|11657216:80|11509760:85|11460608:90|11403264:95|11296768:100|10698752:105|9814016:110|9175040:115|8757248:120|8486912:125|8339456:130|8290304:135|8282112:140|8273920:145|8273920:150|8273920:155|8273920:160|8273920:165|8273920:170|8273920:175|5447680:180|1245184:185|737280:190|589824:195|483328:200|147456:205|16384:210|8192:215|0:220| 1:0|1:5|1689:10|2486:15|2484:20|2467:25|2461:30| +tpch_q12 50 1462161685000 1462161693550 1462161903550 tpch_q12_46 8192:0|8192:5|13500416:10|20348928:15|20348928:20|20201472:25|20152320:30|20037632:35|19734528:40|17047552:45|14548992:50|13778944:55|14696448:60|16064512:65|14540800:70|12468224:75|11935744:80|11640832:85|11517952:90|11436032:95|11190272:100|10379264:105|9740288:110|9199616:115|8863744:120|8585216:125|8429568:130|8331264:135|8290304:140|8282112:145|8282112:150|8282112:155|8273920:160|8273920:165|8273920:170|8273920:175|8273920:180|8273920:185|8257536:190|3710976:195|393216:200|16384:205|8192:210|0:215| 1:0|1:5|1648:10|2484:15|2484:20|2466:25|2460:30| +tpch_q12 50 1462161901000 1462161909461 1462162219461 tpch_q12_47 8192:0|8192:5|14802944:10|20348928:15|20348928:20|20217856:25|20152320:30|20013056:35|19734528:40|16285696:45|15310848:50|14573568:55|14090240:60|13639680:65|13025280:70|12271616:75|11911168:80|11673600:85|11501568:90|11411456:95|11206656:100|10493952:105|9797632:110|9175040:115|8749056:120|8503296:125|8396800:130|8347648:135|8331264:140|8290304:145|8273920:150|8273920:155|8273920:160|8273920:165|8273920:170|8273920:175|8273920:180|6823936:185|4120576:190|3342336:195|2711552:200|2170880:205|1712128:210|1466368:215|1294336:220|1187840:225|1089536:230|917504:235|770048:240|663552:245|589824:250|548864:255|540672:260|540672:265|540672:270|540672:275|540672:280|540672:285|540672:290|516096:295|409600:300|16384:305|8192:310|0:315| 1:0|1:5|1807:10|2484:15|2484:20|2468:25|2460:30| +tpch_q12 50 1462162218000 1462162226670 1462162536670 tpch_q12_48 8192:0|8192:5|13295616:10|20348928:15|20348928:20|20234240:25|20160512:30|20070400:35|19898368:40|15876096:45|15138816:50|14344192:55|14024704:60|13787136:65|13623296:70|13312000:75|12607488:80|12042240:85|11780096:90|11640832:95|11493376:100|10649600:105|9732096:110|9232384:115|8953856:120|8749056:125|8577024:130|8429568:135|8347648:140|8298496:145|8290304:150|8290304:155|8290304:160|8282112:165|8282112:170|8282112:175|8282112:180|8273920:185|8273920:190|8273920:195|8273920:200|8273920:205|8273920:210|8273920:215|7798784:220|7307264:225|4808704:230|3891200:235|3530752:240|3162112:245|2596864:250|2195456:255|1835008:260|1589248:265|1425408:270|1204224:275|974848:280|745472:285|581632:290|483328:295|237568:300|16384:305|8192:310|0:315| 1:0|1:5|1623:10|2484:15|2484:20|2470:25|2461:30| +tpch_q12 50 1462162533000 1462162541572 1462162766572 tpch_q12_49 8192:0|8192:5|13320192:10|20348928:15|20348928:20|20217856:25|20152320:30|20004864:35|19750912:40|18366464:45|18063360:50|17530880:55|12361728:60|14123008:65|12713984:70|12181504:75|11943936:80|11730944:85|11624448:90|11493376:95|11329536:100|10723328:105|9887744:110|9322496:115|8921088:120|8634368:125|8429568:130|8339456:135|8314880:140|8306688:145|8298496:150|8282112:155|8273920:160|8273920:165|8273920:170|8273920:175|8273920:180|8273920:185|8232960:190|3375104:195|696320:200|581632:205|507904:210|335872:215|16384:220|8192:225|0:230| 1:0|1:5|1626:10|2484:15|2484:20|2468:25|2460:30| diff --git a/hadoop-tools/hadoop-resourceestimator/src/test/resources/resourceEstimatorService.txt b/hadoop-tools/hadoop-resourceestimator/src/test/resources/resourceEstimatorService.txt new file mode 100644 index 0000000000..15fb84e98c --- /dev/null +++ b/hadoop-tools/hadoop-resourceestimator/src/test/resources/resourceEstimatorService.txt @@ -0,0 +1,2 @@ +tpch_q12 50 1462148517000 1462148527345 1462148952345 tpch_q12_0 8192:0| 1:0|1:5|1074:10|2538:15|2468:20| +tpch_q12 50 1462148949000 1462148960455 1462149205455 tpch_q12_1 8192:0| 1:0|1:5|794:10|2517:15|2484:20| \ No newline at end of file diff --git a/hadoop-tools/hadoop-resourceestimator/src/test/resources/resourceestimator-config.xml b/hadoop-tools/hadoop-resourceestimator/src/test/resources/resourceestimator-config.xml new file mode 100644 index 0000000000..f78085a0ab --- /dev/null +++ b/hadoop-tools/hadoop-resourceestimator/src/test/resources/resourceestimator-config.xml @@ -0,0 +1,85 @@ + + + + + + resourceestimator.solver.lp.alpha + 0.1 + + The resource estimator has an integrated Linear Programming solver to make the prediction, and this parameter tunes the tradeoff between resource over-allocation and under-allocation in the Linear Programming model. This parameter varies from 0 to 1, and a larger alpha value means the model minimizes over-allocation better. Default value is 0.1. + + + + resourceestimator.solver.lp.beta + 0.1 + + This parameter controls the generalization of the Linear Programming model. This parameter varies from 0 to 1. Deafult value is 0.1. + + + + resourceestimator.solver.lp.minJobRuns + 2 + + The minimum number of job runs required in order to make the prediction. Default value is 2. + + + + resourceestimator.timeInterval + 5 + + The time length which is used to discretize job execution into intervals. Note that the estimator makes resource allocation prediction for each interval. A smaller time interval has more fine-grained granularity for prediction, but it also takes longer time and more space for prediction. Default value is 5 (seconds). + + + + resourceestimator.translator.line-parser + org.apache.hadoop.resourceestimator.translator.impl.NativeSingleLineParser + + The class name of the translator single-line parser, which parses a single line in the log. Default value is org.apache.hadoop.resourceestimator.translator.impl.NativeSingleLineParser, which can parse one line in the sample log. Note that if users want to parse Hadoop Resource Manager logs, they need to set the value to be org.apache.hadoop.resourceestimator.translator.impl.RmSingleLineParser. If they want to implement single-line parser to parse their customized log file, they need to change this value accordingly. + + + + + diff --git a/hadoop-tools/hadoop-resourceestimator/src/test/resources/trace/invalidLog1.txt b/hadoop-tools/hadoop-resourceestimator/src/test/resources/trace/invalidLog1.txt new file mode 100644 index 0000000000..ba05a8903e --- /dev/null +++ b/hadoop-tools/hadoop-resourceestimator/src/test/resources/trace/invalidLog1.txt @@ -0,0 +1,4 @@ +// Test case 1: duplicate job submission time +252823:e,06/21/2017 16:10:13,yarn resourcemanager,DefaultTag,Pid="5156" Tid="39272" TS="0x01D2EAA8DD060D14" String1="17/06/21 16:10:13 INFO resourcemanager.RMAuditLogger: USER=hadoop IP=25.163.46.150 OPERATION=Submit Application Request TARGET=ClientRMService RESULT=SUCCESS APPID=application_1 CALLERCONTEXT=HIVE_QUERY_ID:ExoAdmin_20170621160339_7fed8f3c-3558-4abe-a0db-029eaa44d70e" +252823:e,06/21/2017 16:10:23,yarn resourcemanager,DefaultTag,Pid="5156" Tid="39272" TS="0x01D2EAA8DD060D14" String1="17/06/21 16:10:13 INFO resourcemanager.RMAuditLogger: USER=hadoop IP=25.163.46.150 OPERATION=Submit Application Request TARGET=ClientRMService RESULT=SUCCESS APPID=application_1 CALLERCONTEXT=HIVE_QUERY_ID:ExoAdmin_20170621160339_7fed8f3c-3558-4abe-a0db-029eaa44d70e" +253114:e,06/21/2017 16:18:35,yarn resourcemanager,DefaultTag,Pid="5156" Tid="19824" TS="0x01D2EAAA089569FD" String1="17/06/21 16:18:35 INFO resourcemanager.RMAppManager$ApplicationSummary: appId=application_1,name=Test-1,user=hadoop, queue=PROD,state=FINISHED,trackingUrl=http://BY1PR00OC0019.namprd00.prod.outlook.com:8088/proxy/1,appMasterHost=by2pr08mb1799.namprd08.prod.outlook.com,startTime=1498061413073,finishTime=1498061905698,finalStatus=SUCCEEDED,memorySeconds=1330655,vcoreSeconds=704,preemptedAMContainers=0,preemptedNonAMContainers=0,preemptedResources=,applicationType=MAPREDUCE" diff --git a/hadoop-tools/hadoop-resourceestimator/src/test/resources/trace/invalidLog2.txt b/hadoop-tools/hadoop-resourceestimator/src/test/resources/trace/invalidLog2.txt new file mode 100644 index 0000000000..fe24943987 --- /dev/null +++ b/hadoop-tools/hadoop-resourceestimator/src/test/resources/trace/invalidLog2.txt @@ -0,0 +1,3 @@ +// Test case 2: no jobId found in job submission log +252823:e,06/21/2017 16:10:13,yarn resourcemanager,DefaultTag,Pid="5156" Tid="39272" TS="0x01D2EAA8DD060D14" String1="17/06/21 16:10:13 INFO resourcemanager.RMAuditLogger: USER=hadoop IP=25.163.46.150 OPERATION=Submit Application Request TARGET=ClientRMService RESULT=SUCCESS APPIDs=2 CALLERCONTEXT=HIVE_QUERY_ID:ExoAdmin_20170621160339_7fed8f3c-3558-4abe-a0db-029eaa44d70e" +253114:e,06/21/2017 16:18:35,yarn resourcemanager,DefaultTag,Pid="5156" Tid="19824" TS="0x01D2EAAA089569FD" String1="17/06/21 16:18:35 INFO resourcemanager.RMAppManager$ApplicationSummary: appId=application_2,name=Test-2,user=hadoop, queue=PROD,state=FINISHED,trackingUrl=http://BY1PR00OC0019.namprd00.prod.outlook.com:8088/proxy/1,appMasterHost=by2pr08mb1799.namprd08.prod.outlook.com,startTime=1498061413073,finishTime=1498061905698,finalStatus=SUCCEEDED,memorySeconds=1330655,vcoreSeconds=704,preemptedAMContainers=0,preemptedNonAMContainers=0,preemptedResources=,applicationType=MAPREDUCE" diff --git a/hadoop-tools/hadoop-resourceestimator/src/test/resources/trace/invalidLog3.txt b/hadoop-tools/hadoop-resourceestimator/src/test/resources/trace/invalidLog3.txt new file mode 100644 index 0000000000..63d9c2c3a8 --- /dev/null +++ b/hadoop-tools/hadoop-resourceestimator/src/test/resources/trace/invalidLog3.txt @@ -0,0 +1,5 @@ +// Test case 3: no jobId found in container allocation log +252823:e,06/21/2017 16:10:13,yarn resourcemanager,DefaultTag,Pid="5156" Tid="39272" TS="0x01D2EAA8DD060D14" String1="17/06/21 16:10:13 INFO resourcemanager.RMAuditLogger: USER=hadoop IP=25.163.46.150 OPERATION=Submit Application Request TARGET=ClientRMService RESULT=SUCCESS APPID=application_3 CALLERCONTEXT=HIVE_QUERY_ID:ExoAdmin_20170621160339_7fed8f3c-3558-4abe-a0db-029eaa44d70e" +252839:e,06/21/2017 16:10:13,yarn resourcemanager,DefaultTag,Pid="5156" Tid="39272" TS="0x01D2EAA8DD11F8F0" String1="17/06/21 16:10:13 INFO resourcemanager.RMAuditLogger: USER=hadoop OPERATION=AM Allocated Container TARGET=SchedulerApp RESULT=SUCCESS APPIDs=3 CONTAINERID=container_e26_1497832133857_0330_01_000001" +253100:e,06/21/2017 16:18:35,yarn resourcemanager,DefaultTag,Pid="5156" Tid="19824" TS="0x01D2EAAA089569FD" String1="17/06/21 16:18:35 INFO resourcemanager.RMAuditLogger: USER=hadoop OPERATION=AM Released Container TARGET=SchedulerApp RESULT=SUCCESS APPID=application_3 CONTAINERID=container_e26_1497832133857_0330_01_000001" +253114:e,06/21/2017 16:18:35,yarn resourcemanager,DefaultTag,Pid="5156" Tid="19824" TS="0x01D2EAAA089569FD" String1="17/06/21 16:18:35 INFO resourcemanager.RMAppManager$ApplicationSummary: appId=application_3,name=Test-3,user=hadoop, queue=PROD,state=FINISHED,trackingUrl=http://BY1PR00OC0019.namprd00.prod.outlook.com:8088/proxy/1,appMasterHost=by2pr08mb1799.namprd08.prod.outlook.com,startTime=1498061413073,finishTime=1498061905698,finalStatus=SUCCEEDED,memorySeconds=1330655,vcoreSeconds=704,preemptedAMContainers=0,preemptedNonAMContainers=0,preemptedResources=,applicationType=MAPREDUCE" diff --git a/hadoop-tools/hadoop-resourceestimator/src/test/resources/trace/invalidLog4.txt b/hadoop-tools/hadoop-resourceestimator/src/test/resources/trace/invalidLog4.txt new file mode 100644 index 0000000000..d787319cd1 --- /dev/null +++ b/hadoop-tools/hadoop-resourceestimator/src/test/resources/trace/invalidLog4.txt @@ -0,0 +1,5 @@ +// Test case 4: no containerId found in container allocation log +252823:e,06/21/2017 16:10:13,yarn resourcemanager,DefaultTag,Pid="5156" Tid="39272" TS="0x01D2EAA8DD060D14" String1="17/06/21 16:10:13 INFO resourcemanager.RMAuditLogger: USER=hadoop IP=25.163.46.150 OPERATION=Submit Application Request TARGET=ClientRMService RESULT=SUCCESS APPID=application_4 CALLERCONTEXT=HIVE_QUERY_ID:ExoAdmin_20170621160339_7fed8f3c-3558-4abe-a0db-029eaa44d70e" +252839:e,06/21/2017 16:10:13,yarn resourcemanager,DefaultTag,Pid="5156" Tid="39272" TS="0x01D2EAA8DD11F8F0" String1="17/06/21 16:10:13 INFO resourcemanager.RMAuditLogger: USER=hadoop OPERATION=AM Allocated Container TARGET=SchedulerApp RESULT=SUCCESS APPID=application_4 CONTAINERIDs=container_e26_1497832133857_0330_01_000001" +253100:e,06/21/2017 16:18:35,yarn resourcemanager,DefaultTag,Pid="5156" Tid="19824" TS="0x01D2EAAA089569FD" String1="17/06/21 16:18:35 INFO resourcemanager.RMAuditLogger: USER=hadoop OPERATION=AM Released Container TARGET=SchedulerApp RESULT=SUCCESS APPID=application_4 CONTAINERID=container_e26_1497832133857_0330_01_000001" +253114:e,06/21/2017 16:18:35,yarn resourcemanager,DefaultTag,Pid="5156" Tid="19824" TS="0x01D2EAAA089569FD" String1="17/06/21 16:18:35 INFO resourcemanager.RMAppManager$ApplicationSummary: appId=application_4,name=Test-4,user=hadoop, queue=PROD,state=FINISHED,trackingUrl=http://BY1PR00OC0019.namprd00.prod.outlook.com:8088/proxy/1,appMasterHost=by2pr08mb1799.namprd08.prod.outlook.com,startTime=1498061413073,finishTime=1498061905698,finalStatus=SUCCEEDED,memorySeconds=1330655,vcoreSeconds=704,preemptedAMContainers=0,preemptedNonAMContainers=0,preemptedResources=,applicationType=MAPREDUCE" diff --git a/hadoop-tools/hadoop-resourceestimator/src/test/resources/trace/invalidLog5.txt b/hadoop-tools/hadoop-resourceestimator/src/test/resources/trace/invalidLog5.txt new file mode 100644 index 0000000000..2bafa67476 --- /dev/null +++ b/hadoop-tools/hadoop-resourceestimator/src/test/resources/trace/invalidLog5.txt @@ -0,0 +1,7 @@ +// Test case 5: jobId not found in app failure log +252823:e,06/21/2017 16:10:13,yarn resourcemanager,DefaultTag,Pid="5156" Tid="39272" TS="0x01D2EAA8DD060D14" String1="17/06/21 16:10:13 INFO resourcemanager.RMAuditLogger: USER=hadoop IP=25.163.46.150 OPERATION=Submit Application Request TARGET=ClientRMService RESULT=SUCCESS APPID=application_5 CALLERCONTEXT=HIVE_QUERY_ID:ExoAdmin_20170621160339_7fed8f3c-3558-4abe-a0db-029eaa44d70e" +252839:e,06/21/2017 16:10:13,yarn resourcemanager,DefaultTag,Pid="5156" Tid="39272" TS="0x01D2EAA8DD11F8F0" String1="17/06/21 16:10:13 INFO resourcemanager.RMAuditLogger: USER=hadoop OPERATION=AM Allocated Container TARGET=SchedulerApp RESULT=SUCCESS APPID=application_5 CONTAINERID=container_e26_1497832133857_0330_01_000001" +253100:e,06/21/2017 16:18:35,yarn resourcemanager,DefaultTag,Pid="5156" Tid="19824" TS="0x01D2EAAA089569FD" String1="17/06/21 16:18:35 INFO resourcemanager.RMAuditLogger: USER=hadoop OPERATION=AM Released Container TARGET=SchedulerApp RESULT=SUCCESS APPID=application_5 CONTAINERID=container_e26_1497832133857_0330_01_000001" +e,06/20/2017 22:30:21,yarn resourcemanager,DefaultTag,Pid="5156" Tid="9896" TS="0x01D2EA14CDBCF8F1" String1="17/06/20 22:30:21 INFO capacity.CapacityScheduler: Application Attempt appattempt_5 is done. finalState=FAILED" +252844:e,06/21/2017 16:10:13,yarn resourcemanager,DefaultTag,Pid="5156" Tid="39272" TS="0x01D2EAA8DD11F8F0" String1="17/06/21 16:10:13 INFO attempt.RMAppAttemptImpl: Storing attempt: AppId: application_5 AttemptId: appattempt_5_000001 MasterContainer: Container: [ContainerId: container_e26_1497832133857_0330_01_000001, NodeId: by2pr08mb1799.namprd08.prod.outlook.com:10131, NodeHttpAddress: by2pr08mb1799.namprd08.prod.outlook.com:8042, Resource: , Priority: 0, Token: Token { kind: ContainerToken, service: by2pr08mb1799.namprd08.prod.outlook.com:10131 }, ]" +253114:e,06/21/2017 16:18:35,yarn resourcemanager,DefaultTag,Pid="5156" Tid="19824" TS="0x01D2EAAA089569FD" String1="17/06/21 16:18:35 INFO resourcemanager.RMAppManager$ApplicationSummary: appId=application_5,name=Test-5,user=hadoop, queue=PROD,state=FINISHED,trackingUrl=http://BY1PR00OC0019.namprd00.prod.outlook.com:8088/proxy/1,appMasterHost=by2pr08mb1799.namprd08.prod.outlook.com,startTime=1498061413073,finishTime=1498061905698,finalStatus=SUCCEEDED,memorySeconds=1330655,vcoreSeconds=704,preemptedAMContainers=0,preemptedNonAMContainers=0,preemptedResources=,applicationType=MAPREDUCE" diff --git a/hadoop-tools/hadoop-resourceestimator/src/test/resources/trace/invalidLog6.txt b/hadoop-tools/hadoop-resourceestimator/src/test/resources/trace/invalidLog6.txt new file mode 100644 index 0000000000..bcb5112510 --- /dev/null +++ b/hadoop-tools/hadoop-resourceestimator/src/test/resources/trace/invalidLog6.txt @@ -0,0 +1,5 @@ +// Test case 6: no jobId found in job finish log +252823:e,06/21/2017 16:10:13,yarn resourcemanager,DefaultTag,Pid="5156" Tid="39272" TS="0x01D2EAA8DD060D14" String1="17/06/21 16:10:13 INFO resourcemanager.RMAuditLogger: USER=hadoop IP=25.163.46.150 OPERATION=Submit Application Request TARGET=ClientRMService RESULT=SUCCESS APPID=application_6 CALLERCONTEXT=HIVE_QUERY_ID:ExoAdmin_20170621160339_7fed8f3c-3558-4abe-a0db-029eaa44d70e" +252839:e,06/21/2017 16:10:13,yarn resourcemanager,DefaultTag,Pid="5156" Tid="39272" TS="0x01D2EAA8DD11F8F0" String1="17/06/21 16:10:13 INFO resourcemanager.RMAuditLogger: USER=hadoop OPERATION=AM Allocated Container TARGET=SchedulerApp RESULT=SUCCESS APPID=application_6 CONTAINERID=container_e26_1497832133857_0330_01_000001" +253100:e,06/21/2017 16:18:35,yarn resourcemanager,DefaultTag,Pid="5156" Tid="19824" TS="0x01D2EAAA089569FD" String1="17/06/21 16:18:35 INFO resourcemanager.RMAuditLogger: USER=hadoop OPERATION=AM Released Container TARGET=SchedulerApp RESULT=SUCCESS APPID=application_6 CONTAINERID=container_e26_1497832133857_0330_01_000001" +253114:e,06/21/2017 16:18:35,yarn resourcemanager,DefaultTag,Pid="5156" Tid="19824" TS="0x01D2EAAA089569FD" String1="17/06/21 16:18:35 INFO resourcemanager.RMAppManager$ApplicationSummary: appIds=application_6,name=Test-6,user=hadoop, queue=PROD,state=FINISHED,trackingUrl=http://BY1PR00OC0019.namprd00.prod.outlook.com:8088/proxy/1,appMasterHost=by2pr08mb1799.namprd08.prod.outlook.com,startTime=1498061413073,finishTime=1498061905698,finalStatus=SUCCEEDED,memorySeconds=1330655,vcoreSeconds=704,preemptedAMContainers=0,preemptedNonAMContainers=0,preemptedResources=,applicationType=MAPREDUCE" diff --git a/hadoop-tools/hadoop-resourceestimator/src/test/resources/trace/invalidLog7.txt b/hadoop-tools/hadoop-resourceestimator/src/test/resources/trace/invalidLog7.txt new file mode 100644 index 0000000000..46676be7e9 --- /dev/null +++ b/hadoop-tools/hadoop-resourceestimator/src/test/resources/trace/invalidLog7.txt @@ -0,0 +1,5 @@ +// Test case 7: no recurrenceId found in job finish log +252823:e,06/21/2017 16:10:13,yarn resourcemanager,DefaultTag,Pid="5156" Tid="39272" TS="0x01D2EAA8DD060D14" String1="17/06/21 16:10:13 INFO resourcemanager.RMAuditLogger: USER=hadoop IP=25.163.46.150 OPERATION=Submit Application Request TARGET=ClientRMService RESULT=SUCCESS APPID=application_7 CALLERCONTEXT=HIVE_QUERY_ID:ExoAdmin_20170621160339_7fed8f3c-3558-4abe-a0db-029eaa44d70e" +252839:e,06/21/2017 16:10:13,yarn resourcemanager,DefaultTag,Pid="5156" Tid="39272" TS="0x01D2EAA8DD11F8F0" String1="17/06/21 16:10:13 INFO resourcemanager.RMAuditLogger: USER=hadoop OPERATION=AM Allocated Container TARGET=SchedulerApp RESULT=SUCCESS APPID=application_7 CONTAINERID=container_e26_1497832133857_0330_01_000001" +253100:e,06/21/2017 16:18:35,yarn resourcemanager,DefaultTag,Pid="5156" Tid="19824" TS="0x01D2EAAA089569FD" String1="17/06/21 16:18:35 INFO resourcemanager.RMAuditLogger: USER=hadoop OPERATION=AM Released Container TARGET=SchedulerApp RESULT=SUCCESS APPID=application_7 CONTAINERID=container_e26_1497832133857_0330_01_000001" +253114:e,06/21/2017 16:18:35,yarn resourcemanager,DefaultTag,Pid="5156" Tid="19824" TS="0x01D2EAAA089569FD" String1="17/06/21 16:18:35 INFO resourcemanager.RMAppManager$ApplicationSummary: appId=application_7,name=Test:7,user=hadoop, queue=PROD,state=FINISHED,trackingUrl=http://BY1PR00OC0019.namprd00.prod.outlook.com:8088/proxy/1,appMasterHost=by2pr08mb1799.namprd08.prod.outlook.com,startTime=1498061413073,finishTime=1498061905698,finalStatus=SUCCEEDED,memorySeconds=1330655,vcoreSeconds=704,preemptedAMContainers=0,preemptedNonAMContainers=0,preemptedResources=,applicationType=MAPREDUCE" diff --git a/hadoop-tools/hadoop-resourceestimator/src/test/resources/trace/invalidLog8.txt b/hadoop-tools/hadoop-resourceestimator/src/test/resources/trace/invalidLog8.txt new file mode 100644 index 0000000000..b8a8e204fc --- /dev/null +++ b/hadoop-tools/hadoop-resourceestimator/src/test/resources/trace/invalidLog8.txt @@ -0,0 +1,6 @@ +// Test case 8: no jobId found in resource spec log +252823:e,06/21/2017 16:10:13,yarn resourcemanager,DefaultTag,Pid="5156" Tid="39272" TS="0x01D2EAA8DD060D14" String1="17/06/21 16:10:13 INFO resourcemanager.RMAuditLogger: USER=hadoop IP=25.163.46.150 OPERATION=Submit Application Request TARGET=ClientRMService RESULT=SUCCESS APPID=application_8 CALLERCONTEXT=HIVE_QUERY_ID:ExoAdmin_20170621160339_7fed8f3c-3558-4abe-a0db-029eaa44d70e" +252839:e,06/21/2017 16:10:13,yarn resourcemanager,DefaultTag,Pid="5156" Tid="39272" TS="0x01D2EAA8DD11F8F0" String1="17/06/21 16:10:13 INFO resourcemanager.RMAuditLogger: USER=hadoop OPERATION=AM Allocated Container TARGET=SchedulerApp RESULT=SUCCESS APPID=application_8 CONTAINERID=container_e26_1497832133857_0330_01_000001" +253100:e,06/21/2017 16:18:35,yarn resourcemanager,DefaultTag,Pid="5156" Tid="19824" TS="0x01D2EAAA089569FD" String1="17/06/21 16:18:35 INFO resourcemanager.RMAuditLogger: USER=hadoop OPERATION=AM Released Container TARGET=SchedulerApp RESULT=SUCCESS APPID=application_8 CONTAINERID=container_e26_1497832133857_0330_01_000001" +252844:e,06/21/2017 16:10:13,yarn resourcemanager,DefaultTag,Pid="5156" Tid="39272" TS="0x01D2EAA8DD11F8F0" String1="17/06/21 16:10:13 INFO attempt.RMAppAttemptImpl: Storing attempt: AppId: application_8 AttemptIds: appattempt_8_000001 MasterContainer: Container: [ContainerId: container_e26_1497832133857_0330_01_000001, NodeId: by2pr08mb1799.namprd08.prod.outlook.com:10131, NodeHttpAddress: by2pr08mb1799.namprd08.prod.outlook.com:8042, Resource: , Priority: 0, Token: Token { kind: ContainerToken, service: by2pr08mb1799.namprd08.prod.outlook.com:10131 }, ]" +253114:e,06/21/2017 16:18:35,yarn resourcemanager,DefaultTag,Pid="5156" Tid="19824" TS="0x01D2EAAA089569FD" String1="17/06/21 16:18:35 INFO resourcemanager.RMAppManager$ApplicationSummary: appId=application_8,name=Test-8,user=hadoop, queue=PROD,state=FINISHED,trackingUrl=http://BY1PR00OC0019.namprd00.prod.outlook.com:8088/proxy/1,appMasterHost=by2pr08mb1799.namprd08.prod.outlook.com,startTime=1498061413073,finishTime=1498061905698,finalStatus=SUCCEEDED,memorySeconds=1330655,vcoreSeconds=704,preemptedAMContainers=0,preemptedNonAMContainers=0,preemptedResources=,applicationType=MAPREDUCE" diff --git a/hadoop-tools/hadoop-resourceestimator/src/test/resources/trace/invalidLog9.txt b/hadoop-tools/hadoop-resourceestimator/src/test/resources/trace/invalidLog9.txt new file mode 100644 index 0000000000..598f935bec --- /dev/null +++ b/hadoop-tools/hadoop-resourceestimator/src/test/resources/trace/invalidLog9.txt @@ -0,0 +1,6 @@ +// Test case 9: no resource spec found in resource spec log +252823:e,06/21/2017 16:10:13,yarn resourcemanager,DefaultTag,Pid="5156" Tid="39272" TS="0x01D2EAA8DD060D14" String1="17/06/21 16:10:13 INFO resourcemanager.RMAuditLogger: USER=hadoop IP=25.163.46.150 OPERATION=Submit Application Request TARGET=ClientRMService RESULT=SUCCESS APPID=application_9 CALLERCONTEXT=HIVE_QUERY_ID:ExoAdmin_20170621160339_7fed8f3c-3558-4abe-a0db-029eaa44d70e" +252839:e,06/21/2017 16:10:13,yarn resourcemanager,DefaultTag,Pid="5156" Tid="39272" TS="0x01D2EAA8DD11F8F0" String1="17/06/21 16:10:13 INFO resourcemanager.RMAuditLogger: USER=hadoop OPERATION=AM Allocated Container TARGET=SchedulerApp RESULT=SUCCESS APPID=application_9 CONTAINERID=container_e26_1497832133857_0330_01_000001" +253100:e,06/21/2017 16:18:35,yarn resourcemanager,DefaultTag,Pid="5156" Tid="19824" TS="0x01D2EAAA089569FD" String1="17/06/21 16:18:35 INFO resourcemanager.RMAuditLogger: USER=hadoop OPERATION=AM Released Container TARGET=SchedulerApp RESULT=SUCCESS APPID=application_9 CONTAINERID=container_e26_1497832133857_0330_01_000001" +252844:e,06/21/2017 16:10:13,yarn resourcemanager,DefaultTag,Pid="5156" Tid="39272" TS="0x01D2EAA8DD11F8F0" String1="17/06/21 16:10:13 INFO attempt.RMAppAttemptImpl: Storing attempt: AppId: application_9 AttemptId: appattempt_9_000001 MasterContainer: Container: [ContainerId: container_e26_1497832133857_0330_01_000001, NodeId: by2pr08mb1799.namprd08.prod.outlook.com:10131, NodeHttpAddress: by2pr08mb1799.namprd08.prod.outlook.com:8042, Resource: , Priority: 0, Token: Token { kind: ContainerToken, service: by2pr08mb1799.namprd08.prod.outlook.com:10131 }, ]" +253114:e,06/21/2017 16:18:35,yarn resourcemanager,DefaultTag,Pid="5156" Tid="19824" TS="0x01D2EAAA089569FD" String1="17/06/21 16:18:35 INFO resourcemanager.RMAppManager$ApplicationSummary: appId=application_9,name=Test-9,user=hadoop, queue=PROD,state=FINISHED,trackingUrl=http://BY1PR00OC0019.namprd00.prod.outlook.com:8088/proxy/1,appMasterHost=by2pr08mb1799.namprd08.prod.outlook.com,startTime=1498061413073,finishTime=1498061905698,finalStatus=SUCCEEDED,memorySeconds=1330655,vcoreSeconds=704,preemptedAMContainers=0,preemptedNonAMContainers=0,preemptedResources=,applicationType=MAPREDUCE" diff --git a/hadoop-tools/hadoop-resourceestimator/src/test/resources/trace/nativeLog.txt b/hadoop-tools/hadoop-resourceestimator/src/test/resources/trace/nativeLog.txt new file mode 100644 index 0000000000..fd03ea364b --- /dev/null +++ b/hadoop-tools/hadoop-resourceestimator/src/test/resources/trace/nativeLog.txt @@ -0,0 +1 @@ +tpch_q12 50 1462148517000 1462148527345 1462148952345 tpch_q12_0 8192:0| 1:0|1:5|1074:10|2538:15|2468:20| \ No newline at end of file diff --git a/hadoop-tools/hadoop-resourceestimator/src/test/resources/trace/rmLog.txt b/hadoop-tools/hadoop-resourceestimator/src/test/resources/trace/rmLog.txt new file mode 100644 index 0000000000..7532a73dfa --- /dev/null +++ b/hadoop-tools/hadoop-resourceestimator/src/test/resources/trace/rmLog.txt @@ -0,0 +1,30 @@ +252823:e,06/21/2017 16:10:13,yarn resourcemanager,DefaultTag,Pid="5156" Tid="39272" TS="0x01D2EAA8DD060D14" String1="17/06/21 16:10:13 INFO resourcemanager.RMAuditLogger: USER=hadoop IP=25.163.46.150 OPERATION=Submit Application Request TARGET=ClientRMService RESULT=SUCCESS APPID=application_1497832133857_0330 CALLERCONTEXT=HIVE_QUERY_ID:ExoAdmin_20170621160339_7fed8f3c-3558-4abe-a0db-029eaa44d70e" +252825:e,06/21/2017 16:10:13,yarn resourcemanager,DefaultTag,Pid="5156" Tid="39272" TS="0x01D2EAA8DD060D14" String1="17/06/21 16:10:13 INFO rmapp.RMAppImpl: Storing application with id application_1497832133857_0330" +252826:e,06/21/2017 16:10:13,yarn resourcemanager,DefaultTag,Pid="5156" Tid="39272" TS="0x01D2EAA8DD060D14" String1="17/06/21 16:10:13 INFO rmapp.RMAppImpl: application_1497832133857_0330 State change from NEW to NEW_SAVING" +252827:e,06/21/2017 16:10:13,yarn resourcemanager,DefaultTag,Pid="5156" Tid="39272" TS="0x01D2EAA8DD060D14" String1="17/06/21 16:10:13 INFO recovery.RMStateStore: Storing info for app: application_1497832133857_0330" +252828:e,06/21/2017 16:10:13,yarn resourcemanager,DefaultTag,Pid="5156" Tid="39272" TS="0x01D2EAA8DD11F8F0" String1="17/06/21 16:10:13 INFO rmapp.RMAppImpl: application_1497832133857_0330 State change from NEW_SAVING to SUBMITTED" +252829:e,06/21/2017 16:10:13,yarn resourcemanager,DefaultTag,Pid="5156" Tid="39272" TS="0x01D2EAA8DD11F8F0" String1="17/06/21 16:10:13 INFO capacity.ParentQueue: Application added - appId: application_1497832133857_0330 user: hadoop leaf-queue of parent: root #applications: 3" +252830:e,06/21/2017 16:10:13,yarn resourcemanager,DefaultTag,Pid="5156" Tid="39272" TS="0x01D2EAA8DD11F8F0" String1="17/06/21 16:10:13 INFO capacity.CapacityScheduler: Accepted application application_1497832133857_0330 from user: hadoop, in queue: PROD" +252831:e,06/21/2017 16:10:13,yarn resourcemanager,DefaultTag,Pid="5156" Tid="39272" TS="0x01D2EAA8DD11F8F0" String1="17/06/21 16:10:13 INFO rmapp.RMAppImpl: application_1497832133857_0330 State change from SUBMITTED to ACCEPTED" +252834:e,06/21/2017 16:10:13,yarn resourcemanager,DefaultTag,Pid="5156" Tid="39272" TS="0x01D2EAA8DD11F8F0" String1="17/06/21 16:10:13 INFO capacity.LeafQueue: Application application_1497832133857_0330 from user: hadoop activated in queue: PROD" +252835:e,06/21/2017 16:10:13,yarn resourcemanager,DefaultTag,Pid="5156" Tid="39272" TS="0x01D2EAA8DD11F8F0" String1="17/06/21 16:10:13 INFO capacity.LeafQueue: Application added - appId: application_1497832133857_0330 user: org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.LeafQueue$User@7e2d171f, leaf-queue: PROD #user-pending-applications: 0 #user-active-applications: 1 #queue-pending-applications: 0 #queue-active-applications: 1" +252839:e,06/21/2017 16:10:13,yarn resourcemanager,DefaultTag,Pid="5156" Tid="39272" TS="0x01D2EAA8DD11F8F0" String1="17/06/21 16:10:13 INFO resourcemanager.RMAuditLogger: USER=hadoop OPERATION=AM Allocated Container TARGET=SchedulerApp RESULT=SUCCESS APPID=application_1497832133857_0330 CONTAINERID=container_e26_1497832133857_0330_01_000001" +252844:e,06/21/2017 16:10:13,yarn resourcemanager,DefaultTag,Pid="5156" Tid="39272" TS="0x01D2EAA8DD11F8F0" String1="17/06/21 16:10:13 INFO attempt.RMAppAttemptImpl: Storing attempt: AppId: application_1497832133857_0330 AttemptId: appattempt_1497832133857_0330_000001 MasterContainer: Container: [ContainerId: container_e26_1497832133857_0330_01_000001, NodeId: by2pr08mb1799.namprd08.prod.outlook.com:10131, NodeHttpAddress: by2pr08mb1799.namprd08.prod.outlook.com:8042, Resource: , Priority: 0, Token: Token { kind: ContainerToken, service: by2pr08mb1799.namprd08.prod.outlook.com:10131 }, ]" +252897:e,06/21/2017 16:12:34,yarn resourcemanager,DefaultTag,Pid="5156" Tid="41712" TS="0x01D2EAA931905154" String1="17/06/21 16:12:34 INFO resourcemanager.RMAuditLogger: USER=hadoop IP=25.163.46.25 OPERATION=Register App Master TARGET=ApplicationMasterService RESULT=SUCCESS APPID=application_1497832133857_0330 APPATTEMPTID=appattempt_1497832133857_0330_000001" +252900:e,06/21/2017 16:12:34,yarn resourcemanager,DefaultTag,Pid="5156" Tid="41712" TS="0x01D2EAA931905154" String1="17/06/21 16:12:34 INFO rmapp.RMAppImpl: application_1497832133857_0330 State change from ACCEPTED to RUNNING" +252906:e,06/21/2017 16:12:35,yarn resourcemanager,DefaultTag,Pid="5156" Tid="41712" TS="0x01D2EAA93234D4E8" String1="17/06/21 16:12:35 INFO resourcemanager.RMAuditLogger: USER=hadoop OPERATION=AM Allocated Container TARGET=SchedulerApp RESULT=SUCCESS APPID=application_1497832133857_0330 CONTAINERID=container_e26_1497832133857_0330_01_000002" +252910:e,06/21/2017 16:12:38,yarn resourcemanager,DefaultTag,Pid="5156" Tid="39272" TS="0x01D2EAA933A6643E" String1="17/06/21 16:12:38 INFO scheduler.AppSchedulingInfo: checking for deactivate of application :application_1497832133857_0330" +253016:e,06/21/2017 16:15:58,yarn resourcemanager,DefaultTag,Pid="5156" Tid="39188" TS="0x01D2EAA9AB29C5EE" String1="17/06/21 16:15:58 INFO resourcemanager.RMAuditLogger: USER=hadoop OPERATION=AM Released Container TARGET=SchedulerApp RESULT=SUCCESS APPID=application_1497832133857_0330 CONTAINERID=container_e26_1497832133857_0330_01_000002" +253086:e,06/21/2017 16:18:25,yarn resourcemanager,DefaultTag,Pid="5156" Tid="19824" TS="0x01D2EAAA02A6A418" String1="17/06/21 16:18:25 INFO rmapp.RMAppImpl: Updating application application_1497832133857_0330 with final state: FINISHING" +253087:e,06/21/2017 16:18:25,yarn resourcemanager,DefaultTag,Pid="5156" Tid="19824" TS="0x01D2EAAA02A6A418" String1="17/06/21 16:18:25 INFO rmapp.RMAppImpl: application_1497832133857_0330 State change from RUNNING to FINAL_SAVING" +253089:e,06/21/2017 16:18:25,yarn resourcemanager,DefaultTag,Pid="5156" Tid="19824" TS="0x01D2EAAA02B28FF0" String1="17/06/21 16:18:25 INFO recovery.RMStateStore: Updating info for app: application_1497832133857_0330" +253090:e,06/21/2017 16:18:25,yarn resourcemanager,DefaultTag,Pid="5156" Tid="19824" TS="0x01D2EAAA02B28FF0" String1="17/06/21 16:18:25 INFO rmapp.RMAppImpl: application_1497832133857_0330 State change from FINAL_SAVING to FINISHING" +253091:e,06/21/2017 16:18:26,yarn resourcemanager,DefaultTag,Pid="5156" Tid="19824" TS="0x01D2EAAA033F3BC8" String1="17/06/21 16:18:26 INFO resourcemanager.ApplicationMasterService: application_1497832133857_0330 unregistered successfully. " +253100:e,06/21/2017 16:18:35,yarn resourcemanager,DefaultTag,Pid="5156" Tid="19824" TS="0x01D2EAAA089569FD" String1="17/06/21 16:18:35 INFO resourcemanager.RMAuditLogger: USER=hadoop OPERATION=AM Released Container TARGET=SchedulerApp RESULT=SUCCESS APPID=application_1497832133857_0330 CONTAINERID=container_e26_1497832133857_0330_01_000001" +253105:e,06/21/2017 16:18:35,yarn resourcemanager,DefaultTag,Pid="5156" Tid="19824" TS="0x01D2EAAA089569FD" String1="17/06/21 16:18:35 INFO rmapp.RMAppImpl: application_1497832133857_0330 State change from FINISHING to FINISHED" +253110:e,06/21/2017 16:18:35,yarn resourcemanager,DefaultTag,Pid="5156" Tid="19824" TS="0x01D2EAAA089569FD" String1="17/06/21 16:18:35 INFO scheduler.AppSchedulingInfo: Application application_1497832133857_0330 requests cleared" +253111:e,06/21/2017 16:18:35,yarn resourcemanager,DefaultTag,Pid="5156" Tid="19824" TS="0x01D2EAAA089569FD" String1="17/06/21 16:18:35 INFO capacity.LeafQueue: Application removed - appId: application_1497832133857_0330 user: hadoop queue: PROD #user-pending-applications: 0 #user-active-applications: 0 #queue-pending-applications: 0 #queue-active-applications: 0" +253112:e,06/21/2017 16:18:35,yarn resourcemanager,DefaultTag,Pid="5156" Tid="19824" TS="0x01D2EAAA089569FD" String1="17/06/21 16:18:35 INFO capacity.ParentQueue: Application removed - appId: application_1497832133857_0330 user: hadoop leaf-queue of parent: root #applications: 2" +253113:e,06/21/2017 16:18:35,yarn resourcemanager,DefaultTag,Pid="5156" Tid="19824" TS="0x01D2EAAA089569FD" String1="17/06/21 16:18:35 INFO resourcemanager.RMAuditLogger: USER=hadoop OPERATION=Application Finished - Succeeded TARGET=RMAppManager RESULT=SUCCESS APPID=application_1497832133857_0330" +253114:e,06/21/2017 16:18:35,yarn resourcemanager,DefaultTag,Pid="5156" Tid="19824" TS="0x01D2EAAA089569FD" String1="17/06/21 16:18:35 INFO resourcemanager.RMAppManager$ApplicationSummary: appId=application_1497832133857_0330,name=FraudDetection-1,user=hadoop, queue=PROD,state=FINISHED,trackingUrl=http://BY1PR00OC0019.namprd00.prod.outlook.com:8088/proxy/application_1497832133857_0330/,appMasterHost=by2pr08mb1799.namprd08.prod.outlook.com,startTime=1498061413073,finishTime=1498061905698,finalStatus=SUCCEEDED,memorySeconds=1330655,vcoreSeconds=704,preemptedAMContainers=0,preemptedNonAMContainers=0,preemptedResources=,applicationType=MAPREDUCE" +e,06/20/2017 22:30:21,yarn resourcemanager,DefaultTag,Pid="5156" Tid="9896" TS="0x01D2EA14CDBCF8F1" String1="17/06/20 22:30:21 INFO capacity.CapacityScheduler: Application Attempt appattempt_1497832133857_0194_000001 is done. finalState=FAILED" diff --git a/hadoop-tools/hadoop-tools-dist/pom.xml b/hadoop-tools/hadoop-tools-dist/pom.xml index a5aa264ed9..28faa9f0ee 100644 --- a/hadoop-tools/hadoop-tools-dist/pom.xml +++ b/hadoop-tools/hadoop-tools-dist/pom.xml @@ -116,6 +116,12 @@ hadoop-sls compile + + org.apache.hadoop + hadoop-resourceestimator + ${project.version} + compile + org.apache.hadoop hadoop-azure-datalake diff --git a/hadoop-tools/pom.xml b/hadoop-tools/pom.xml index 18381a3fc4..6f95f117ef 100644 --- a/hadoop-tools/pom.xml +++ b/hadoop-tools/pom.xml @@ -42,6 +42,7 @@ hadoop-pipes hadoop-openstack hadoop-sls + hadoop-resourceestimator hadoop-azure hadoop-aws hadoop-kafka