diff --git a/dev-support/bin/dist-layout-stitching b/dev-support/bin/dist-layout-stitching index d4bfd8aaada..cc076bd9e34 100755 --- a/dev-support/bin/dist-layout-stitching +++ b/dev-support/bin/dist-layout-stitching @@ -70,6 +70,9 @@ function copyifnotexists() else for childpath in "${src}"/*; do child="${childpath##*/}" + if [ "${child}" == "*" ]; then + continue; + fi if [[ "${child}" == "doc" || "${child}" == "webapps" ]]; then mkdir -p "${dest}/${child}" diff --git a/hadoop-assemblies/src/main/resources/assemblies/hadoop-dtss.xml b/hadoop-assemblies/src/main/resources/assemblies/hadoop-dtss.xml new file mode 100644 index 00000000000..d250dd54744 --- /dev/null +++ b/hadoop-assemblies/src/main/resources/assemblies/hadoop-dtss.xml @@ -0,0 +1,44 @@ + + + hadoop-dtss + + dir + + false + + + + ${basedir}/src/main/bin + dtss/bin + 0755 + + + ${basedir}/src/main/python + dtss/python + 0755 + + + ${basedir}/src/main/sample + dtss/sample + + + + diff --git a/hadoop-assemblies/src/main/resources/assemblies/hadoop-tools.xml b/hadoop-assemblies/src/main/resources/assemblies/hadoop-tools.xml index db744f511da..6a81f2fc354 100644 --- a/hadoop-assemblies/src/main/resources/assemblies/hadoop-tools.xml +++ b/hadoop-assemblies/src/main/resources/assemblies/hadoop-tools.xml @@ -189,6 +189,17 @@ ../hadoop-sls/target/hadoop-sls-${project.version}/sls /share/hadoop/${hadoop.component}/sls + + ../hadoop-dtss/target + /share/hadoop/${hadoop.component}/sources + + *-sources.jar + + + + ../hadoop-dtss/target/hadoop-dtss-${project.version}/dtss + /share/hadoop/${hadoop.component}/dtss + ../hadoop-resourceestimator/target /share/hadoop/${hadoop.component}/sources diff --git a/hadoop-project/pom.xml b/hadoop-project/pom.xml index 90a7c4420f8..2420e729a06 100644 --- a/hadoop-project/pom.xml +++ b/hadoop-project/pom.xml @@ -685,6 +685,11 @@ hadoop-kms ${hadoop.version} + + org.apache.hadoop + hadoop-dtss + ${project.version} + org.apache.hadoop hadoop-kms @@ -1380,6 +1385,11 @@ hadoop-sls ${hadoop.version} + + org.apache.hadoop + hadoop-dtss + ${project.version} + org.apache.hadoop hadoop-cloud-storage @@ -1754,6 +1764,11 @@ lz4-java ${lz4-java.version} + + net.objecthunter + exp4j + 0.4.8 + diff --git a/hadoop-tools/hadoop-dtss/.gitignore b/hadoop-tools/hadoop-dtss/.gitignore new file mode 100644 index 00000000000..99bbb83b6e8 --- /dev/null +++ b/hadoop-tools/hadoop-dtss/.gitignore @@ -0,0 +1,177 @@ +# Created by .ignore support plugin (hsz.mobi) +### JetBrains template +# Covers JetBrains IDEs: IntelliJ, RubyMine, PhpStorm, AppCode, PyCharm, CLion, Android Studio and Webstorm +# Reference: https://intellij-support.jetbrains.com/hc/en-us/articles/206544839 + +# CMake +cmake-build-debug/ + +# Mongo Explorer plugin: +.idea/**/mongoSettings.xml + +## File-based project format: +*.iws + +## Plugin-specific files: + +# IntelliJ +.idea/ +out/ + +# mpeltonen/sbt-idea plugin +.idea_modules/ + +# JIRA plugin +atlassian-ide-plugin.xml + +# Cursive Clojure plugin +.idea/replstate.xml + +# Crashlytics plugin (for Android Studio and IntelliJ) +com_crashlytics_export_strings.xml +crashlytics.properties +crashlytics-build.properties +fabric.properties +### Java template +# Compiled class file +*.class + +# Log file +*.log + +# BlueJ files +*.ctxt + +*.iml + +# Mobile Tools for Java (J2ME) +.mtj.tmp/ + +# Package Files # +*.jar +*.war +*.ear +*.zip +*.tar.gz +*.rar + +# virtual machine crash logs, see http://www.java.com/en/download/help/error_hotspot.xml +hs_err_pid* + +target/ +pom.xml.tag +pom.xml.releaseBackup +pom.xml.versionsBackup +pom.xml.next +release.properties +dependency-reduced-pom.xml +buildNumber.properties +.mvn/timing.properties + +# Avoid ignoring Maven wrapper jar file (.jar files are usually ignored) +!/.mvn/wrapper/maven-wrapper.jar + +# Byte-compiled / optimized / DLL files +__pycache__/ +*.py[cod] +*$py.class + +# C extensions +*.so + +# Distribution / packaging +.Python +build/ +develop-eggs/ +dist/ +downloads/ +eggs/ +.eggs/ +lib/ +lib64/ +parts/ +sdist/ +var/ +wheels/ +*.egg-info/ +.installed.cfg +*.egg +MANIFEST + +# PyInstaller +# Usually these files are written by a python script from a template +# before PyInstaller builds the exe, so as to inject date/other infos into it. +*.manifest +*.spec + +# Installer logs +pip-log.txt +pip-delete-this-directory.txt + +# Unit test / coverage reports +htmlcov/ +.tox/ +.coverage +.coverage.* +.cache +nosetests.xml +coverage.xml +*.cover +.hypothesis/ + +# Translations +*.mo +*.pot + +# Django stuff: +*.log +.static_storage/ +.media/ +local_settings.py + +# Flask stuff: +.webassets-cache + +# Scrapy stuff: +.scrapy + +# Sphinx documentation +docs/_build/ + +# Jupyter Notebook +.ipynb_checkpoints + +# pyenv +.python-version + +# celery beat schedule file +celerybeat-schedule + +# SageMath parsed files +*.sage.py + +# Environments +.env +.venv +env/ +venv/ +ENV/ +env.bak/ +venv.bak/ + +# Spyder project settings +.spyderproject +.spyproject + +# Rope project settings +.ropeproject + +# mkdocs documentation +/site + +# mypy +.mypy_cache/ + +# OS X +.DS_Store + diff --git a/hadoop-tools/hadoop-dtss/pom.xml b/hadoop-tools/hadoop-dtss/pom.xml new file mode 100644 index 00000000000..6d97f5c8597 --- /dev/null +++ b/hadoop-tools/hadoop-dtss/pom.xml @@ -0,0 +1,202 @@ + + + + 4.0.0 + + org.apache.hadoop + hadoop-project + 3.4.0-SNAPSHOT + ../../hadoop-project + + hadoop-dtss + 3.4.0-SNAPSHOT + Apache Hadoop Discrete Time Scheduler Simulator + Apache Hadoop Discrete Time Scheduler Simulator + jar + + + 4.0 + + + + + junit + junit + test + + + com.google.code.gson + gson + + + com.google.inject + guice + ${guice.version} + + + com.google.inject.extensions + guice-multibindings + ${guice.version} + + + org.apache.hadoop + hadoop-common + + + org.apache.hadoop + hadoop-yarn-api + + + org.apache.hadoop + hadoop-yarn-server-timelineservice + + + org.apache.hadoop + hadoop-client + + + org.apache.hadoop + hadoop-yarn-server-resourcemanager + + + commons-lang + commons-lang + 2.6 + + + org.apache.commons + commons-lang3 + + + org.apache.commons + commons-csv + + + commons-io + commons-io + + + commons-cli + commons-cli + + + io.dropwizard.metrics + metrics-core + + + net.objecthunter + exp4j + + + org.mockito + mockito-core + + + + + + + + src/main/sample/config/yarn + + + + + org.apache.maven.plugins + maven-source-plugin + + true + + + + + jar + + + + + + org.codehaus.mojo + findbugs-maven-plugin + + true + true + ${basedir}/dev-support/findbugs-exclude.xml + Max + + + + + + + + dist + + false + + + + + org.apache.maven.plugins + maven-assembly-plugin + + + org.apache.hadoop + hadoop-assemblies + ${project.version} + + + + + dist + prepare-package + + single + + + false + false + ${project.artifactId}-${project.version} + + hadoop-dtss + + + + + + + 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-dtss/src/main/bin/dtssrun.sh b/hadoop-tools/hadoop-dtss/src/main/bin/dtssrun.sh new file mode 100755 index 00000000000..fee1c2097a4 --- /dev/null +++ b/hadoop-tools/hadoop-dtss/src/main/bin/dtssrun.sh @@ -0,0 +1,95 @@ +#!/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. +# + +function hadoop_usage() +{ + echo "Usage: dtssrun.sh tracelocation" +} + +function calculate_classpath +{ + hadoop_add_to_classpath_tools hadoop-dtss + hadoop_add_classpath "${HADOOP_YARN_HOME}/${YARN_DIR}/timelineservice/"'/*' + hadoop_add_classpath "${HADOOP_YARN_HOME}/${YARN_DIR}/timelineservice/lib/*" +} + +function run_simulation() { + hadoop_add_client_opts + hadoop_finalize + + # shellcheck disable=SC2086 + echo "Running simulation in dtss.sh!" + if [[ -z $redirect_path ]]; then + hadoop_java_exec dtss org.apache.hadoop.yarn.dtss.Runner "$@" + else + echo "Redirecting output to $redirect_path" + mkdir -p "$(dirname "$redirect_path")" + hadoop_java_exec dtss org.apache.hadoop.yarn.dtss.Runner "$@" > $redirect_path 2>&1 + fi +} + +while getopts "i:m:" opt; do + case $opt in + i) + id=${OPTARG} + ;; + m) + metric_path=${OPTARG} + ;; + \?) + echo "Invalid option: -$OPTARG" >&2 + exit 1 + ;; + esac +done + +# 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 disable=SC1090 + . "${HADOOP_LIBEXEC_DIR}/hadoop-config.sh" +else + echo "ERROR: Cannot execute ${HADOOP_LIBEXEC_DIR}/hadoop-config.sh." 2>&1 + exit 1 +fi + +calculate_classpath + +sim_args=( "${@:1:$OPTIND-1}" ) +shift $((OPTIND-1)) + +if [[ $# = 0 ]] || [[ $# -gt 2 ]]; then + hadoop_exit_with_usage 1 +fi + +sim_args+=( "$1" ) +shift 1 + +if [[ $# = 0 ]]; then + run_simulation "${sim_args[@]}" +else + redirect_path=$1 + run_simulation "${sim_args[@]}" +fi diff --git a/hadoop-tools/hadoop-dtss/src/main/bin/run-simulation.sh b/hadoop-tools/hadoop-dtss/src/main/bin/run-simulation.sh new file mode 100755 index 00000000000..1961bb55973 --- /dev/null +++ b/hadoop-tools/hadoop-dtss/src/main/bin/run-simulation.sh @@ -0,0 +1,351 @@ +#!/usr/bin/env bash + +print_usage() { + echo "usage: run-simulation.sh [-h] [-b] [-p] [-n] [-o] [-t] sim_conf_path output_base_path" + echo "" + + echo "This script should be executed from either \$HADOOP_HOME/share/hadoop/tools/dtss/bin/ OR" + echo "from \$HADOOP_SRC/hadoop-tools/hadoop-dtss/src/main/bin/." + echo "Note that we can only build if executed from the hadoop source path." + + echo "" + + echo "flags (optional):" + echo -e "\t-h\t\t\tPrints help message." + echo -e "\t-b\t\t\tBuild." + echo -e "\t-p\t\t\tPackage." + echo -e "\t-n\t\t\tPackage with native libraries." + echo -e "\t-t\t\t\tSpecifies the base directory for temporary hadoop setups. Requires an argument." + echo -e "\t-o\t\t\tSpecifies whether or not the output should be overwritten if exists." + echo "" + + echo "positional arguments:" + + echo -e "\tsim_conf_path\t\tThe simulation configuration path, can either be a directory containing many " + echo -e "\t\tconfiguration files or a single configuration file, where the name of the configuration file " + echo -e "\t\tspecifies the run ID. For each configuration file, there should be a directory of the same " + echo -e "\t\tfile name that specifies the YARN configurations used for the particular run within the same " + echo -e "\t\tdirectory as the configuration file." + + echo -e "\toutput_base_path\tSpecifies the base directory for run outputs. A run directory will be created" + echo -e "\t\tfurther within output_base_path for each run of the simulation." +} + +pushd () { + command pushd "$@" > /dev/null +} + +popd () { + command popd "$@" > /dev/null +} + +cleanup() { + local -n child_pids=$1 + local -n tmp_dirs=$2 + + echo "Killing child processes ""${child_pids[*]}""..." + for i in "${child_pids[@]}" + do : + echo "Killing child PID $i" + kill -9 "$i" + done + + wait + + echo "Cleaning up ""${tmp_dirs[*]}""..." + for i in "${tmp_dirs[@]}" + do : + rm -rf "$i" & + done + + wait + echo "Cleanup done!" +} + +analyze_simulation() { + local run_id=$1 + local output_base_path=$2 + local output_path="$output_base_path"/"$run_id" + + python_dir="$( cd "$script_dir/../python/" >/dev/null 2>&1 && pwd )" + if [ -e $python_dir ]; then + echo "Detected python script directory at $python_dir" + else + echo "Cannot detect python script directory, skipping analysis." + return + fi + + pushd $python_dir + if [[ "$VIRTUAL_ENV" = "" ]]; then + in_venv=false + venv_dir="$python_dir"/venv + if [ ! -d $venv_dir ]; then + # Create the virtual environment directory + echo "Creating the virtual environment directory at $venv_dir" + python3 -m venv $venv_dir + fi + + echo "Activating virtual environment..." + source "$venv_dir"/bin/activate + echo "Upgrading pip..." + pip3 install --upgrade pip + echo "Installing requirements..." + pip3 install -r requirements.txt + else + # Do not deactivate later if already in venv + in_venv=true + echo "Already in python virtual environment!" + fi + + echo "Running python analysis on $output_path!" + python3 sim_analysis.py analyze -i $run_id $output_path $output_base_path + echo "Done!" + if [ $in_venv = false ]; then + deactivate + fi + popd +} + +run_simulation() { + local tmp_dir=$1 + local sim_conf_path=$2 + local output_base_path=$3 + + local sim_conf_basedir="$( cd "$( dirname "$sim_conf_path" )" >/dev/null 2>&1 && pwd )" + local sim_conf_file="${sim_conf_path##*/}" + local sim_conf_id="${sim_conf_file%.*}" + local output_path="$output_base_path"/"$sim_conf_id" + local sim_conf_dir="$sim_conf_basedir"/"$sim_conf_id" + echo "Using simulation configuration directory $sim_conf_dir" + echo "Using output directory $output_path" + + if [ -d "$output_path" ]; then + if [ "$overwrite" = false ]; then + echo "$output_path exists and overwrite is not turned on! Returning..." + return + else + echo "$output_path exists and overwrite is turned on! Deleting $output_path" + rm -rf "$output_path" + fi + fi + + echo "Using temp directory $tmp_dir" + echo "Copying hadoop home to tmp dir to prevent overwriting..." + cp -r "$hadoop_home" "$tmp_dir" + local hadoop_basename="$(basename $hadoop_home)" + local local_hadoop_home="$tmp_dir/$hadoop_basename" + echo "Using tmp hadoop home $local_hadoop_home" + local local_hadoop_conf_dir="$local_hadoop_home/etc/hadoop/" + cp -r "$sim_conf_dir"/* "$local_hadoop_conf_dir" + + echo "Running simulation!" + local out_file="$output_path"/out.txt + echo "Writing output file to $out_file" + local local_dtss_run_path="$local_hadoop_home/share/hadoop/tools/dtss/bin/dtssrun.sh" + if [ ! -e "$local_dtss_run_path" ]; then + echo "The dtssrun.sh does not exist at $local_dtss_run_path!" + return + else + echo "Found dtssrun.sh at $local_dtss_run_path" + fi + + $local_dtss_run_path -i "$sim_conf_id" -m "$output_base_path" "$sim_conf_path" "$out_file" + echo "Simulation run complete!" + echo "Analyzing simulation run metrics..." + analyze_simulation "$sim_conf_id" "$output_base_path" +} + +if [[ $# = 0 ]]; then + print_usage + exit 1 +fi + +build=false +pkg=false +native=false +from_src=false +overwrite=false +tmp_dir_base="" + +while getopts ":nbphot:" opt; do + case $opt in + o) + overwrite=true + ;; + t) + if [[ $OPTARG =~ ^-[n/b/p/h/o/t]$ ]] + then + echo "-t requires an argument!" + print_usage + exit 1 + fi + tmp_dir_base=$OPTARG + + echo "Using $tmp_dir_base as temp base directory!" + if [ ! -d "$tmp_dir_base" ]; then + echo "$tmp_dir_base is not a valid directory!" + print_usage + exit 1 + fi + + if [ ! -w "$tmp_dir_base" ] || [ ! -r "$tmp_dir_base" ]; then + echo "$tmp_dir_base must both be read and writable!" + exit 1 + fi + ;; + n) + native=true + ;; + b) + build=true + ;; + p) + pkg=true + ;; + h) + print_usage + exit 0 + ;; + \?) + echo "Invalid option: -$OPTARG" >&2 + print_usage + exit 1 + ;; + :) + echo "Invalid option: $OPTARG requires an argument" >&2 + print_usage + exit 1 + esac +done + +script_dir="$( cd "$( dirname "${BASH_SOURCE[0]}" )" >/dev/null 2>&1 && pwd )" +src_root_dir="$( cd "$script_dir/../../../../.." >/dev/null 2>&1 && pwd )" + +if [ -e "$src_root_dir"/pom.xml ] && [ -e "$src_root_dir"/LICENSE.txt ]; then + echo "Detected Hadoop source root directory at $src_root_dir" + from_src=true +fi + +if [ $from_src = true ]; then + pushd $src_root_dir + # Only build/package if running from source directory + if [ $build = true ] ; then + echo "-b is turned on, building..." >&2 + mvn clean install -DskipTests + if [ $? -ne 0 ]; then + echo "Maven build failed, exiting..." + exit 1 + fi + fi + + if [ $pkg = true ] ; then + if [ $native = true ] ; then + echo "-np is turned on, packaging native..." >&2 + mvn package -Pnative -Pdist -DskipTests -nsu + if [ $? -ne 0 ]; then + echo "Maven package failed, exiting..." + exit 1 + fi + else + echo "-p is turned on, packaging..." >&2 + mvn package -Pdist -DskipTests -nsu + if [ $? -ne 0 ]; then + echo "Maven package failed, exiting..." + exit 1 + fi + fi + fi + + hadoop_home_pattern="hadoop-dist/target/hadoop-*/" + files=( $hadoop_home_pattern ) + hadoop_home_relative="${files[0]}" + if [ "$hadoop_home_relative" = "$hadoop_home_pattern" ] ; then + echo "Unable to find hadoop home! Exiting..." + exit 1 + fi + hadoop_home_relative=$src_root_dir/$hadoop_home_relative + popd +else + echo "Skipping build, since we are not running from the source directory." + hadoop_home_relative="$script_dir"/../../../../.. +fi + +build_or_pkg=$build||$pkg + +sim_conf_path_arg=${@:$OPTIND:1} +if [ -z $sim_conf_path_arg ] ; then + if [ ! $build_or_pkg ]; then + echo "Must have a simulation configuration path!" + exit 1 + else + echo "Done!" + exit 0 + fi +fi + +output_path_arg=${@:$(( $OPTIND + 1)):1} +if [ -z $output_path_arg ] ; then + if [ ! $build_or_pkg ]; then + echo "Must have an output path!" + exit 1 + else + echo "Done!" + exit 0 + fi +fi + +if [ ! -e $sim_conf_path_arg ]; then + echo "The simulation configuration path must exist!" + exit 1 +fi + +sim_conf_path="$(cd "$(dirname "$sim_conf_path_arg")"; pwd -P)/$(basename "$sim_conf_path_arg")" + +echo "Using simulation configuration path: $sim_conf_path" + +if [ ! -e $output_path_arg ]; then + echo "The output path must exist!" + exit 1 +fi + +output_base_path="$(cd "$(dirname "$output_path_arg")"; pwd -P)/$(basename "$output_path_arg")" +echo "Using output path: $output_base_path" + +hadoop_home="$( cd "$hadoop_home_relative" >/dev/null 2>&1 && pwd )" +echo "Located hadoop home directory at $hadoop_home" + +pushd "$hadoop_home"/share/hadoop/tools/dtss/bin + tmp_dirs=() + child_pids=() + if [ -f "$sim_conf_path" ] ; then + if [ -z "$tmp_dir_base" ] ; then + tmp_dir=$(mktemp -d -t sim-XXXXXXXXXX) + else + tmp_dir=$(mktemp -p "$tmp_dir_base" -d -t sim-XXXXXXXXXX) + fi + tmp_dirs+=("$tmp_dir") + run_simulation "$tmp_dir" "$sim_conf_path" "$output_base_path" & + child_pids+=("$!") + elif [ -d "$sim_conf_path" ]; then + for conf in "$sim_conf_path"/*; do + if [ ! -f "$conf" ]; then + echo "$conf is not a file, skipping..." + continue; + fi + + if [ -z "$tmp_dir_base" ] ; then + tmp_dir=$(mktemp -d -t sim-XXXXXXXXXX) + else + tmp_dir=$(mktemp -p "$tmp_dir_base" -d -t sim-XXXXXXXXXX) + fi + + tmp_dirs+=("$tmp_dir") + echo "Running simulation with configuration $conf!" + run_simulation "$tmp_dir" "$conf" "$output_base_path" & + child_pids+=("$!") + done + fi + + trap 'cleanup "${child_pids[*]}" "${tmp_dirs[*]}"' EXIT + wait +popd diff --git a/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/Launcher.java b/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/Launcher.java new file mode 100644 index 00000000000..6316af04632 --- /dev/null +++ b/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/Launcher.java @@ -0,0 +1,256 @@ +package org.apache.hadoop.yarn.dtss; + + +import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting; +import com.google.gson.GsonBuilder; +import com.google.inject.Guice; +import com.google.inject.Injector; +import com.google.inject.Key; +import com.google.inject.Module; +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.apache.hadoop.yarn.dtss.cluster.ClusterState; +import org.apache.hadoop.yarn.dtss.cluster.ClusterTopology; +import org.apache.hadoop.yarn.dtss.cluster.ClusterTopologyBuilder; +import org.apache.hadoop.yarn.dtss.cluster.NodeRepresentation; +import org.apache.hadoop.yarn.dtss.config.parameters.IsUnmanaged; +import org.apache.hadoop.yarn.dtss.lifecycle.LifeCycleState; +import org.apache.hadoop.yarn.dtss.metrics.CapacitySchedulerWrapper; +import org.apache.hadoop.yarn.dtss.metrics.FairSchedulerWrapper; +import org.apache.hadoop.yarn.dtss.metrics.MetricsConfiguration; +import org.apache.hadoop.yarn.dtss.nm.MockNM; +import org.apache.hadoop.yarn.dtss.rm.MockRM; +import org.apache.hadoop.yarn.dtss.time.Clock; +import org.apache.hadoop.yarn.dtss.trace.TraceReader; +import org.apache.hadoop.yarn.exceptions.YarnException; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FairScheduler; + +import java.io.FileWriter; +import java.io.IOException; +import java.util.UUID; +import java.util.logging.Level; +import java.util.logging.Logger; + +/** + * Launcher class for the discrete event-driven simulation. + */ +@InterfaceAudience.Private +final class Launcher { + // How often we should check traces + private static final long TRACE_CHECK_SECONDS = 60; + + // How often heartbeats between NMs and RMs should occur + private static final long MIN_RM_NM_HEARTBEAT_INTERVAL_SECONDS = 10; + + private static final Logger LOG = Logger.getLogger(Launcher.class.getName()); + + private Launcher() { + } + + /** + * Reads the YARN configuration and start the simulations. + * @param configuration The combined simulation configuration + */ + @VisibleForTesting + public static void launch(final Module configuration) { + final Injector injector = Guice.createInjector(configuration); + + final Configuration yarnConf = injector.getInstance(Configuration.class); + final String schedulerClass = yarnConf.get(YarnConfiguration.RM_SCHEDULER); + LOG.log(Level.INFO, "RM scheduler class: " + schedulerClass); + boolean isUnmanaged = injector.getInstance(Key.get(Boolean.class, IsUnmanaged.class)); + if (isUnmanaged) { + LOG.log(Level.INFO, "Using unmanaged AMs."); + } else { + LOG.log(Level.INFO, "Using managed AMs."); + } + + // Only these schedulers are supported. + // Overwrite the scheduler class with the wrapper class. + try { + if (Class.forName(schedulerClass) == CapacityScheduler.class) { + yarnConf.set(YarnConfiguration.RM_SCHEDULER, + CapacitySchedulerWrapper.class.getName()); + } else if(Class.forName(schedulerClass) == FairScheduler.class) { + yarnConf.set(YarnConfiguration.RM_SCHEDULER, + FairSchedulerWrapper.class.getName()); + } else { + throw new YarnException(schedulerClass + " is not supported yet."); + } + } catch(Exception e) { + throw new RuntimeException(e); + } + + final Clock clock = injector.getInstance(Clock.class); + // RM is created here + final MockRM rm = injector.getInstance(MockRM.class); + + // Initialize the cluster + final ClusterTopologyBuilder clusterTopologyBuilder = injector.getInstance(ClusterTopologyBuilder.class); + final ClusterState clusterState = injector.getInstance(ClusterState.class); + final ClusterTopology clusterTopology; + final TraceReader traceReader = injector.getInstance(TraceReader.class); + + try { + clusterTopology = clusterTopologyBuilder.getClusterTopology(); + + // Start + clock.start(); + rm.start(); + final UUID nmHeartbeatAlarmId = registerNodes(yarnConf, clusterTopology, clusterState, rm, clock); + + // Start trace-reading and periodic checking + traceReader.init(); + traceReader.start(); + + registerTraceCheckAlarm(clock, traceReader, clusterState, nmHeartbeatAlarmId); + + LifeCycleState clockState = clock.pollNextAlarm(); + + // Run until either no more jobs/events, or until the end time is reached. + while (shouldContinueRunning(clockState)) { + clockState = clock.pollNextAlarm(); + } + + traceReader.onStop(); + } catch (final Exception e) { + e.printStackTrace(); + throw new RuntimeException(e); + } finally { + traceReader.stop(); + rm.stop(); + clock.stop(); + } + + // Write metric results + // By now, the metrics manager has already stopped + if (rm.getMetricsManager().isMetricsOn()) { + assert rm.getMetricsManager().getState() == LifeCycleState.STOPPED; + rm.getMetricsManager().writeResults(); + final CapacitySchedulerConfiguration capacitySchedulerConf; + if (rm.getResourceScheduler() instanceof CapacityScheduler) { + capacitySchedulerConf = ((CapacityScheduler) rm.getResourceScheduler()).getConfiguration(); + } else { + capacitySchedulerConf = null; + } + try { + writeConfigs(rm.getMetricsManager().getMetricsConfig(), yarnConf, capacitySchedulerConf, configuration); + } catch (final IOException ioe) { + throw new RuntimeException(ioe); + } + } + } + + /** + * Registers a periodic alarm on the clock that periodically checks + * if all jobs in the trace are completed by checking with the {@link TraceReader}. + * Unregister alarms if completed. + * @param clock The clock + * @param traceReader The trace reader + * @param clusterState The current cluster state to unregister NMs + * @param nmHeartbeatAlarmId The heartbeat ID for periodic NM-RM heartbeats + */ + private static void registerTraceCheckAlarm(final Clock clock, + final TraceReader traceReader, + final ClusterState clusterState, + final UUID nmHeartbeatAlarmId) { + clock.schedulePeriodicAlarm(TRACE_CHECK_SECONDS, periodicClientAlarm -> { + if (traceReader.areTraceJobsDone()) { + LOG.log(Level.INFO, "Traces completed at " + clock.getInstant() + "!"); + clock.unschedulePeriodicAlarm(nmHeartbeatAlarmId); + clock.unschedulePeriodicAlarm(periodicClientAlarm.getPeriodicAlarmId()); + for (final MockNM nm : clusterState.getNMs()) { + try { + nm.unRegisterNode(); + } catch(final Exception e) { + e.printStackTrace(); + throw new RuntimeException(e); + } + } + + clock.scheduleShutdown(1); + } + }); + } + + /** + * Registers NMs wth the RM and schedule periodic heartbeats for each NM. + * @param conf The YARN configuration + * @param clusterTopology The cluster topology + * @param clusterState The state of the cluster + * @param rm The RM + * @param clock The discrete-event clock + * @return The periodic heartbeat alarm ID + * @throws Exception + */ + private static UUID registerNodes(final Configuration conf, + final ClusterTopology clusterTopology, + final ClusterState clusterState, + final MockRM rm, + final Clock clock) throws Exception { + final long heartbeatIntervalMs = conf.getLong( + YarnConfiguration.RM_NM_HEARTBEAT_INTERVAL_MS, + YarnConfiguration.DEFAULT_RM_NM_HEARTBEAT_INTERVAL_MS); + + final long heartbeatIntervalSeconds = Math.max( + MIN_RM_NM_HEARTBEAT_INTERVAL_SECONDS, (long) Math.ceil(heartbeatIntervalMs / 1000.0)); + + for (final NodeRepresentation nodeRepresentation : clusterTopology.getNodeSet()) { + final MockNM nm = new MockNM(nodeRepresentation.getNodeHostName(), + nodeRepresentation.getContainersOnNode() * clusterTopology.getContainerMB(), + nodeRepresentation.getContainersOnNode() * clusterTopology.getContainerVCores(), + rm.getResourceTrackerService()); + + clusterState.addNM(nm); + nm.registerNode(); + nm.nodeHeartbeat(true); + } + + LOG.log(Level.INFO, "NM heartbeats every " + heartbeatIntervalSeconds + " seconds."); + + return clock.schedulePeriodicAlarm(heartbeatIntervalSeconds, alarm -> { + clusterState.heartbeatAll(); + }); + } + + private static boolean shouldContinueRunning(final LifeCycleState clockState) { + return clockState != LifeCycleState.STOPPED; + } + + /** + * Write the new configs used for running simulations for diagnostic purposes. + * @param metricsConfig The metric configuration + * @param yarnConf The YARN configuration + * @param capacitySchedulerConf The CapacityScheduler configuration + * @param module the simulation configuration + * @throws IOException + */ + private static void writeConfigs( + final MetricsConfiguration metricsConfig, + final Configuration yarnConf, + final CapacitySchedulerConfiguration capacitySchedulerConf, + final Module module) throws IOException { + final String yarnConfPath = metricsConfig.getMetricsFilePath("yarn-conf.xml"); + assert yarnConfPath != null; + try (final FileWriter fw = new FileWriter(yarnConfPath)) { + yarnConf.writeXml(fw); + } + + if (capacitySchedulerConf != null) { + final String capacityConfPath = metricsConfig.getMetricsFilePath("capacity-scheduler.xml"); + assert capacityConfPath != null; + try (final FileWriter fw = new FileWriter(capacityConfPath)) { + capacitySchedulerConf.writeXml(fw); + } + } + + final String simConfPath = metricsConfig.getMetricsFilePath("sim-conf.json"); + assert simConfPath != null; + try (final FileWriter writer = new FileWriter(simConfPath)) { + new GsonBuilder().setPrettyPrinting().create().toJson(module, writer); + } + } +} diff --git a/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/Runner.java b/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/Runner.java new file mode 100644 index 00000000000..6e985fb7fb9 --- /dev/null +++ b/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/Runner.java @@ -0,0 +1,157 @@ +package org.apache.hadoop.yarn.dtss; + +import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting; +import com.google.inject.AbstractModule; +import com.google.inject.util.Modules; +import org.apache.commons.cli.*; +import org.apache.commons.lang3.builder.ReflectionToStringBuilder; +import org.apache.commons.lang3.builder.ToStringStyle; +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.yarn.dtss.config.SimulatorModule; +import org.apache.hadoop.yarn.dtss.config.parameters.runner.RunId; +import org.apache.hadoop.yarn.dtss.config.parameters.runner.SimulationConfigPath; +import org.apache.log4j.BasicConfigurator; + +import java.io.IOException; +import java.time.LocalDateTime; +import java.time.format.DateTimeFormatter; +import java.util.logging.Handler; +import java.util.logging.Level; +import java.util.logging.LogManager; +import java.util.logging.Logger; + +/** + * Houses the main class to {@link Launcher}, which begins the simulation. + */ +@InterfaceAudience.Private +public final class Runner { + private static final Logger LOG = Logger.getLogger(Runner.class.getName()); + private static String runId = DateTimeFormatter.ofPattern( + "yyyy-MM-dd-HH-mm-ss").format(LocalDateTime.now()); + + @VisibleForTesting + public static final class RunnerModule extends AbstractModule { + private final String configPath; + private final String runId; + + @VisibleForTesting + public static RunnerModule newModule(final String configPath, final String runId) { + return new RunnerModule(configPath, runId); + } + + private RunnerModule(final String configPath, final String runId) { + this.configPath = configPath; + this.runId = runId; + } + + @Override + protected void configure() { + bind(String.class).annotatedWith(SimulationConfigPath.class).toInstance(configPath); + bind(String.class).annotatedWith(RunId.class).toInstance(runId); + } + } + + private static Options commandLineOptions() { + final Options options = new Options(); + options + .addOption("i", "id", true, "The Run ID of the job.") + .addOption("m", "metrics", true, "The metrics output directory"); + + return options; + } + + @InterfaceStability.Unstable + public static void runSimulator( + final SimulatorModule simulatorModule, final RunnerModule runnerModule){ + LOG.log(Level.INFO, String.format( + "Read configuration\n%s", + ReflectionToStringBuilder.toString(simulatorModule, ToStringStyle.MULTI_LINE_STYLE) + )); + + final Level logLevel = simulatorModule.getSimulatorLogLevel(); + LOG.log(Level.INFO, "Log level set to " + logLevel + "."); + + setSimulatorLogLevel(logLevel); + + Launcher.launch(Modules.combine(simulatorModule, runnerModule)); + } + + public static void main(final String[] args) { + // Sets the Apache logger level + org.apache.log4j.Logger.getRootLogger().setLevel(org.apache.log4j.Level.WARN); + BasicConfigurator.configure(); + + final String configPath; + final String metricsOutputDirectory; + final CommandLineParser parser = new BasicParser(); + try { + final CommandLine cmd = parser.parse(commandLineOptions(), args); + + if (cmd.hasOption('i')) { + runId = cmd.getOptionValue('i'); + if (runId == null || runId.trim().isEmpty()) { + throw new IllegalArgumentException("Run ID cannot be set to empty!"); + } + } + + if (cmd.hasOption('m')) { + metricsOutputDirectory = cmd.getOptionValue('m'); + if (metricsOutputDirectory == null || metricsOutputDirectory.trim().isEmpty()){ + throw new IllegalArgumentException("Metrics output directory cannot be empty!"); + } + } else { + metricsOutputDirectory = null; + } + + final String[] restOfArgs = cmd.getArgs(); + + assert restOfArgs.length == 1 : "Should only have one command line argument holding " + + "the path to the configuration file."; + configPath = restOfArgs[0]; + } catch (final ParseException e) { + e.printStackTrace(); + throw new RuntimeException(e); + } + + LOG.log(Level.INFO, "Running simulation with ID " + runId + "."); + LOG.log(Level.INFO, String.format("Reading configuration from %s", configPath)); + + final SimulatorModule simulatorModule; + try { + simulatorModule = SimulatorModule.newReader().readConfiguration(configPath); + if (metricsOutputDirectory != null) { + LOG.log(Level.INFO, "Overwriting metrics output directory to " + metricsOutputDirectory); + simulatorModule.setMetricsOutputDirectory(metricsOutputDirectory); + } + } catch (final IOException e) { + LOG.log( + Level.SEVERE, + () -> String.format("Unable to read simulation configuration at %s", configPath) + ); + + throw new RuntimeException(e); + } + + final RunnerModule runnerModule = new RunnerModule(configPath, runId); + try { + runSimulator(simulatorModule, runnerModule); + } catch (final AssertionError assertionError) { + LOG.log(Level.SEVERE, + "Assertion failed with message: " + assertionError); + throw assertionError; + } catch (final Throwable e) { + LOG.log(Level.SEVERE, + "Simulation failed with throwable: " + e); + throw new RuntimeException(e); + } + } + + private static void setSimulatorLogLevel(final Level logLevel) { + final Logger rootLogger = LogManager.getLogManager().getLogger(""); + rootLogger.setLevel(logLevel); + for (final Handler h : rootLogger.getHandlers()) { + h.setLevel(logLevel); + } + } +} diff --git a/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/am/MockAM.java b/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/am/MockAM.java new file mode 100644 index 00000000000..0fa0c85b7b3 --- /dev/null +++ b/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/am/MockAM.java @@ -0,0 +1,453 @@ +package org.apache.hadoop.yarn.dtss.am; + +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +import org.apache.hadoop.security.UserGroupInformation; +import org.apache.hadoop.security.token.Token; +import org.apache.hadoop.yarn.api.ApplicationMasterProtocol; +import org.apache.hadoop.yarn.api.protocolrecords.*; +import org.apache.hadoop.yarn.api.records.*; +import org.apache.hadoop.yarn.api.resource.PlacementConstraint; +import org.apache.hadoop.yarn.api.resource.PlacementConstraints; +import org.apache.hadoop.yarn.dtss.nm.MockNM; +import org.apache.hadoop.yarn.dtss.rm.MockRM; +import org.apache.hadoop.yarn.security.AMRMTokenIdentifier; +import org.apache.hadoop.yarn.server.resourcemanager.RMContext; +import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp; +import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt; +import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptState; +import org.apache.hadoop.yarn.util.Records; +import org.apache.log4j.Logger; + +import java.lang.reflect.UndeclaredThrowableException; +import java.security.PrivilegedExceptionAction; +import java.util.*; +import java.util.concurrent.atomic.AtomicLong; + +/** + * This class is based on the MockAM class used in the tests + * for YARN's resourcemanager project. + */ +public final class MockAM { + + private static final Logger LOG = Logger.getLogger(MockAM.class); + + private final AtomicLong allocationRequestId = new AtomicLong(); + private volatile int responseId = 0; + private final ApplicationAttemptId attemptId; + private RMContext context; + private ApplicationMasterProtocol amRMProtocol; + private UserGroupInformation ugi; + private volatile AllocateResponse lastResponse; + private Map, PlacementConstraint> placementConstraints = + new HashMap<>(); + private List schedulingRequests = new ArrayList<>(); + + private final List requests = new ArrayList(); + private final List releases = new ArrayList(); + + public MockAM(RMContext context, ApplicationMasterProtocol amRMProtocol, + ApplicationAttemptId attemptId) { + this.context = context; + this.amRMProtocol = amRMProtocol; + this.attemptId = attemptId; + } + + public void setAMRMProtocol(ApplicationMasterProtocol amRMProtocol, + RMContext context) { + this.context = context; + this.amRMProtocol = amRMProtocol; + } + + /** + * Wait until an attempt has reached a specified state. + * The timeout is 40 seconds. + * @param finalState the attempt state waited + * @throws InterruptedException + * if interrupted while waiting for the state transition + */ + private void waitForState(RMAppAttemptState finalState) + throws InterruptedException { + RMApp app = context.getRMApps().get(attemptId.getApplicationId()); + RMAppAttempt attempt = app.getRMAppAttempt(attemptId); + MockRM.waitForState(attempt, finalState); + } + + public RegisterApplicationMasterResponse registerAppAttempt() + throws Exception { + return registerAppAttempt(true); + } + + public void addPlacementConstraint(Set tags, + PlacementConstraint constraint) { + placementConstraints.put(tags, constraint); + } + + public MockAM addSchedulingRequest(List reqs) { + schedulingRequests.addAll(reqs); + return this; + } + + public RegisterApplicationMasterResponse registerAppAttempt(boolean wait) + throws Exception { + if (wait) { + waitForState(RMAppAttemptState.LAUNCHED); + } + responseId = 0; + final RegisterApplicationMasterRequest req = + Records.newRecord(RegisterApplicationMasterRequest.class); + req.setHost(""); + req.setRpcPort(1); + req.setTrackingUrl(""); + if (!placementConstraints.isEmpty()) { + req.setPlacementConstraints(this.placementConstraints); + } + if (ugi == null) { + ugi = UserGroupInformation.createRemoteUser( + attemptId.toString()); + Token token = + context.getRMApps().get(attemptId.getApplicationId()) + .getRMAppAttempt(attemptId).getAMRMToken(); + ugi.addTokenIdentifier(token.decodeIdentifier()); + } + try { + return ugi + .doAs( + (PrivilegedExceptionAction) () -> + amRMProtocol.registerApplicationMaster(req)); + } catch (UndeclaredThrowableException e) { + throw (Exception) e.getCause(); + } + } + + public int getResponseId() { + return responseId; + } + + public void addRequests(String[] hosts, int memory, int priority, + int containers) throws Exception { + requests.addAll(createReq(hosts, memory, priority, containers)); + } + + public AllocateResponse schedule() throws Exception { + AllocateResponse response = allocate(requests, releases); + requests.clear(); + releases.clear(); + return response; + } + + public void addContainerToBeReleased(ContainerId containerId) { + releases.add(containerId); + } + + public AllocateResponse allocate( + String host, int memory, int numContainers, + List releases) throws Exception { + return allocate(host, memory, numContainers, releases, null); + } + + public AllocateResponse allocate( + String host, int memory, int numContainers, + List releases, String labelExpression) throws Exception { + return allocate(host, memory, numContainers, 1, releases, labelExpression); + } + + public AllocateResponse allocate( + String host, int memory, int numContainers, int priority, + List releases, String labelExpression) throws Exception { + List reqs = + createReq(new String[] { host }, memory, priority, numContainers, labelExpression); + return allocate(reqs, releases); + } + + public AllocateResponse allocate( + String host, Resource cap, int numContainers, + List rels, String labelExpression) throws Exception { + List reqs = new ArrayList<>(); + ResourceRequest oneReq = + createResourceReq(host, cap, numContainers, + labelExpression); + reqs.add(oneReq); + return allocate(reqs, rels); + } + + public List createReq(String[] hosts, int memory, + int priority, int containers) throws Exception { + return createReq(hosts, memory, priority, containers, null); + } + + public List createReq(String[] hosts, int memory, + int priority, int containers, String labelExpression) throws Exception { + List reqs = new ArrayList(); + if (hosts != null) { + for (String host : hosts) { + final long allocId = allocationRequestId.getAndIncrement(); + // only add host/rack request when asked host isn't ANY + if (!host.equals(ResourceRequest.ANY)) { + ResourceRequest hostReq = + createResourceReq(host, memory, priority, containers, + labelExpression); + hostReq.setAllocationRequestId(allocId); + reqs.add(hostReq); + ResourceRequest rackReq = + createResourceReq("/default-rack", memory, priority, containers, + labelExpression); + rackReq.setAllocationRequestId(allocId); + reqs.add(rackReq); + } + } + } + + ResourceRequest offRackReq = createResourceReq(ResourceRequest.ANY, memory, + priority, containers, labelExpression); + offRackReq.setAllocationRequestId(allocationRequestId.getAndIncrement()); + reqs.add(offRackReq); + return reqs; + } + + public ResourceRequest createResourceReq(String resource, int memory, int priority, + int containers) throws Exception { + return createResourceReq(resource, memory, priority, containers, null); + } + + public ResourceRequest createResourceReq(String resource, int memory, + int priority, int containers, String labelExpression) throws Exception { + return createResourceReq(resource, memory, priority, containers, + labelExpression, ExecutionTypeRequest.newInstance()); + } + + public ResourceRequest createResourceReq(String resource, int memory, + int priority, int containers, String labelExpression, + ExecutionTypeRequest executionTypeRequest) throws Exception { + ResourceRequest req = Records.newRecord(ResourceRequest.class); + req.setResourceName(resource); + req.setNumContainers(containers); + Priority pri = Records.newRecord(Priority.class); + pri.setPriority(priority); + req.setPriority(pri); + Resource capability = Records.newRecord(Resource.class); + capability.setMemorySize(memory); + req.setCapability(capability); + if (labelExpression != null) { + req.setNodeLabelExpression(labelExpression); + } + req.setExecutionTypeRequest(executionTypeRequest); + return req; + + } + + public ResourceRequest createResourceReq(String host, Resource cap, + int containers, String labelExpression) throws Exception { + ResourceRequest req = Records.newRecord(ResourceRequest.class); + req.setResourceName(host); + req.setNumContainers(containers); + Priority pri = Records.newRecord(Priority.class); + pri.setPriority(1); + req.setPriority(pri); + req.setCapability(cap); + if (labelExpression != null) { + req.setNodeLabelExpression(labelExpression); + } + req.setExecutionTypeRequest(ExecutionTypeRequest.newInstance()); + return req; + + } + + public AllocateResponse allocate( + List resourceRequest, List releases) + throws Exception { + final AllocateRequest req = + AllocateRequest.newInstance(0, 0F, resourceRequest, + releases, null); + if (!schedulingRequests.isEmpty()) { + req.setSchedulingRequests(schedulingRequests); + schedulingRequests.clear(); + } + return allocate(req); + } + + public AllocateResponse allocate(List resourceRequest, + List newSchedulingRequests, List releases) + throws Exception { + final AllocateRequest req = + AllocateRequest.newInstance(0, 0F, resourceRequest, + releases, null); + if (newSchedulingRequests != null) { + addSchedulingRequest(newSchedulingRequests); + } + if (!schedulingRequests.isEmpty()) { + req.setSchedulingRequests(schedulingRequests); + schedulingRequests.clear(); + } + return allocate(req); + } + + public AllocateResponse allocateIntraAppAntiAffinity( + ResourceSizing resourceSizing, Priority priority, long allocationId, + Set allocationTags, String... targetTags) throws Exception { + return allocateAppAntiAffinity(resourceSizing, priority, allocationId, + null, allocationTags, targetTags); + } + + public AllocateResponse allocateAppAntiAffinity( + ResourceSizing resourceSizing, Priority priority, long allocationId, + String namespace, Set allocationTags, String... targetTags) + throws Exception { + return this.allocate(null, + Arrays.asList(SchedulingRequest.newBuilder().executionType( + ExecutionTypeRequest.newInstance(ExecutionType.GUARANTEED)) + .allocationRequestId(allocationId).priority(priority) + .allocationTags(allocationTags).placementConstraintExpression( + PlacementConstraints + .targetNotIn(PlacementConstraints.NODE, + PlacementConstraints.PlacementTargets + .allocationTagWithNamespace(namespace, targetTags)) + .build()) + .resourceSizing(resourceSizing).build()), null); + } + + public AllocateResponse allocateIntraAppAntiAffinity( + String nodePartition, ResourceSizing resourceSizing, Priority priority, + long allocationId, String... tags) throws Exception { + return this.allocate(null, + Arrays.asList(SchedulingRequest.newBuilder().executionType( + ExecutionTypeRequest.newInstance(ExecutionType.GUARANTEED)) + .allocationRequestId(allocationId).priority(priority) + .placementConstraintExpression(PlacementConstraints + .targetNotIn(PlacementConstraints.NODE, + PlacementConstraints.PlacementTargets + .allocationTag(tags), + PlacementConstraints.PlacementTargets + .nodePartition(nodePartition)).build()) + .resourceSizing(resourceSizing).build()), null); + } + + public AllocateResponse sendContainerResizingRequest( + List updateRequests) throws Exception { + final AllocateRequest req = AllocateRequest.newInstance(0, 0F, null, null, + updateRequests, null); + return allocate(req); + } + + public AllocateResponse sendContainerUpdateRequest( + List updateRequests) throws Exception { + final AllocateRequest req = AllocateRequest.newInstance(0, 0F, null, null, + updateRequests, null); + return allocate(req); + } + + public AllocateResponse allocate(AllocateRequest allocateRequest) + throws Exception { + UserGroupInformation ugi = + UserGroupInformation.createRemoteUser(attemptId.toString()); + Token token = + context.getRMApps().get(attemptId.getApplicationId()) + .getRMAppAttempt(attemptId).getAMRMToken(); + ugi.addTokenIdentifier(token.decodeIdentifier()); + lastResponse = doAllocateAs(ugi, allocateRequest); + return lastResponse; + } + + public AllocateResponse doAllocateAs(UserGroupInformation ugi, + final AllocateRequest req) throws Exception { + req.setResponseId(responseId); + try { + AllocateResponse response = + ugi.doAs(new PrivilegedExceptionAction() { + @Override + public AllocateResponse run() throws Exception { + return amRMProtocol.allocate(req); + } + }); + responseId = response.getResponseId(); + return response; + } catch (UndeclaredThrowableException e) { + throw (Exception) e.getCause(); + } + } + + public AllocateResponse doHeartbeat() throws Exception { + return allocate(Collections.emptyList(), Collections.emptyList()); + } + + public void unregisterAppAttempt() throws Exception { + waitForState(RMAppAttemptState.RUNNING); + unregisterAppAttempt(true); + } + + public void unregisterAppAttempt(boolean waitForStateRunning) + throws Exception { + final FinishApplicationMasterRequest req = + FinishApplicationMasterRequest.newInstance( + FinalApplicationStatus.SUCCEEDED, "", ""); + unregisterAppAttempt(req, waitForStateRunning); + } + + public void unregisterAppAttempt(final FinishApplicationMasterRequest req, + boolean waitForStateRunning) throws Exception { + if (waitForStateRunning) { + waitForState(RMAppAttemptState.RUNNING); + } + if (ugi == null) { + ugi = UserGroupInformation.createRemoteUser(attemptId.toString()); + Token token = + context.getRMApps() + .get(attemptId.getApplicationId()) + .getRMAppAttempt(attemptId).getAMRMToken(); + ugi.addTokenIdentifier(token.decodeIdentifier()); + } + try { + ugi.doAs(new PrivilegedExceptionAction() { + @Override + public Object run() throws Exception { + amRMProtocol.finishApplicationMaster(req); + return null; + } + }); + } catch (UndeclaredThrowableException e) { + throw (Exception) e.getCause(); + } + } + + public ApplicationAttemptId getApplicationAttemptId() { + return this.attemptId; + } + + public List allocateAndWaitForContainers(int nContainer, + int memory, MockNM nm) throws Exception { + return allocateAndWaitForContainers("ANY", nContainer, memory, nm); + } + + public List allocateAndWaitForContainers(String host, + int nContainer, int memory, MockNM nm) throws Exception { + // AM request for containers + allocate(host, memory, nContainer, null); + // kick the scheduler + nm.nodeHeartbeat(true); + List conts = allocate(new ArrayList(), null) + .getAllocatedContainers(); + while (conts.size() < nContainer) { + nm.nodeHeartbeat(true); + conts.addAll(allocate(new ArrayList(), + new ArrayList()).getAllocatedContainers()); + Thread.sleep(500); + } + return conts; + } +} diff --git a/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/cluster/ClusterState.java b/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/cluster/ClusterState.java new file mode 100644 index 00000000000..7b664cdf087 --- /dev/null +++ b/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/cluster/ClusterState.java @@ -0,0 +1,95 @@ +package org.apache.hadoop.yarn.dtss.cluster; + +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +import org.apache.hadoop.thirdparty.com.google.common.collect.ImmutableList; +import com.google.inject.Inject; +import com.google.inject.Injector; +import com.google.inject.Singleton; +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.yarn.api.records.NodeId; +import org.apache.hadoop.yarn.dtss.nm.MockNM; +import org.apache.hadoop.yarn.dtss.random.RandomGenerator; +import org.apache.hadoop.yarn.dtss.random.RandomSeeder; +import org.apache.hadoop.yarn.dtss.rm.MockRM; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.AbstractYarnScheduler; + +import java.util.HashMap; +import java.util.Map; +import java.util.logging.Logger; + +/** + * This class records the current state of the cluster, + * keeping track of the RM and the NMs registered to the cluster. + */ +@InterfaceAudience.Private +@Singleton +public final class ClusterState { + private static final Logger LOG = Logger.getLogger(ClusterState.class.getName()); + + private final Map nodeManagers = new HashMap<>(); + private final RandomGenerator randomGenerator; + private final MockRM rm; + + @Inject + private ClusterState( + final RandomSeeder randomSeeder, + final Injector injector) { + this.randomGenerator = randomSeeder.newRandomGenerator(); + this.rm = injector.getInstance(MockRM.class); + } + + public MockNM getNM(final NodeId nodeId) { + return nodeManagers.get(nodeId); + } + + public MockNM getRandomNM() { + final int nmIdx = randomGenerator.randomInt(nodeManagers.size()); + return getNMs().get(nmIdx); + } + + public void addNM(final MockNM nm) { + nodeManagers.put(nm.getNodeId(), nm); + } + + public void removeNM(final MockNM nm) { + nodeManagers.remove(nm.getNodeId()); + } + + public ImmutableList getNMs() { + return ImmutableList.copyOf(nodeManagers.values()); + } + + /** + * Triggers heartbeats from the NMs to the RM. + */ + public void heartbeatAll() { + for (final MockNM nm : getNMs()) { + try { + nm.nodeHeartbeat(true); + ((AbstractYarnScheduler) rm.getRMContext().getScheduler()).update(); + } catch (final Exception e) { + e.printStackTrace(); + throw new RuntimeException(e); + } + } + + rm.drainEvents(); + } +} diff --git a/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/cluster/ClusterTopology.java b/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/cluster/ClusterTopology.java new file mode 100644 index 00000000000..1903d4005f7 --- /dev/null +++ b/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/cluster/ClusterTopology.java @@ -0,0 +1,112 @@ +package org.apache.hadoop.yarn.dtss.cluster; + +import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting; +import com.google.gson.Gson; +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.yarn.conf.YarnConfiguration; + +import java.io.FileReader; +import java.io.FileWriter; +import java.io.IOException; +import java.util.HashSet; +import java.util.Set; +import java.util.logging.Logger; + +/** + * Container for {@link ClusterTopologyProperties}, which describes the topology of the cluster, + * and for the {@link NodeRepresentation}s of each mocked node in the cluster, + * which contains information on their hostnames and containers. + */ +@InterfaceAudience.Private +@InterfaceStability.Unstable +public final class ClusterTopology { + private static final Logger LOG = Logger.getLogger(ClusterTopology.class.getName()); + // The set of nodes + private final Set nodeSet; + // The cluster topology represented as a 2D array + private final ClusterTopologyProperties topologyProperties; + + private ClusterTopology( + final Set nodeSet, + final ClusterTopologyProperties topologyProperties) { + this.nodeSet = nodeSet; + this.topologyProperties = topologyProperties; + } + + /** + * Writes the cluster topology as JSON. + * @param topologyProperties The shape of the cluster topology + * @param filePath The file path to write to + * @throws IOException + */ + @VisibleForTesting + public static void writeClusterTopologyFile( + final ClusterTopologyProperties topologyProperties, + final String filePath + ) throws IOException { + final Gson gson = new Gson(); + final FileWriter writer = new FileWriter(filePath); + gson.toJson(topologyProperties, writer); + writer.close(); + } + + /** + * Parses the cluster topology file from JSON. + * @param clusterTopologyFilePath The path to the topology file + * @return The {@link ClusterTopology} object + * @throws IOException + */ + static ClusterTopology parseTopologyFile( + final String clusterTopologyFilePath) throws IOException { + // Read topology property file + final Gson gson = new Gson(); + final ClusterTopologyProperties topologyProperties = gson.fromJson( + new FileReader(clusterTopologyFilePath), ClusterTopologyProperties.class); + + // Make container MB and Cores configurable - not necessarily as large as the YarnConfig + // Also make sure they are not 0 - fall back to default Yarn values + topologyProperties.setContainerMB(Math.max(topologyProperties.getContainerMB(), + YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_MB)); + topologyProperties.setContainerVCores(Math.max(topologyProperties.getContainerVCores(), + YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_VCORES)); + + LOG.info("Cluster Racks: "+ topologyProperties.getClusterRacks() + + " Containers: " + topologyProperties.getClusterContainers() + + " Each ContainerMB: " + topologyProperties.getContainerMB() + + " Each ContainerVCores: " + topologyProperties.getContainerVCores()); + + final Set nodeSet = new HashSet<>(); + // Create NodeRepresentations per Rack with specific number of containers + for (int rackId = 0; rackId < topologyProperties.getRackNodeContainers().size(); rackId++) { + for (int nodeId = 0; nodeId < topologyProperties.getRackNodeContainers().get(rackId).size(); nodeId++) { + final int containersOnNode = topologyProperties.getRackNodeContainers().get(rackId).get(nodeId); + final String nodeHostName = "host" + ((rackId * topologyProperties.getRackNodeContainers().get(rackId).size()) + nodeId) + ":12345"; + final NodeRepresentation nodeRep = new NodeRepresentation(nodeHostName, containersOnNode); + nodeSet.add(nodeRep); + } + } + + return new ClusterTopology(nodeSet, topologyProperties); + } + + public ClusterTopologyProperties getClusterTopologyProperties() { + return this.topologyProperties; + } + + public int getTotalContainers() { + return this.topologyProperties.getClusterContainers(); + } + + public int getContainerVCores() { + return topologyProperties.getContainerVCores(); + } + + public int getContainerMB() { + return topologyProperties.getContainerMB(); + } + + public Set getNodeSet() { + return nodeSet; + } +} diff --git a/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/cluster/ClusterTopologyBuilder.java b/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/cluster/ClusterTopologyBuilder.java new file mode 100644 index 00000000000..a5681073d12 --- /dev/null +++ b/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/cluster/ClusterTopologyBuilder.java @@ -0,0 +1,33 @@ +package org.apache.hadoop.yarn.dtss.cluster; + +import com.google.inject.Inject; +import com.google.inject.Singleton; +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.yarn.dtss.config.parameters.ClusterTopologyFilePath; + +import java.io.IOException; + +/** + * An injected object to make the same + * {@link ClusterTopology} object available to all. + */ +@Singleton +@InterfaceAudience.Private +public final class ClusterTopologyBuilder { + private final String filePath; + + private ClusterTopology topology = null; + + @Inject + private ClusterTopologyBuilder(@ClusterTopologyFilePath final String filePath) { + this.filePath = filePath; + } + + public ClusterTopology getClusterTopology() throws IOException { + if (topology == null) { + topology = ClusterTopology.parseTopologyFile(filePath); + } + + return topology; + } +} diff --git a/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/cluster/ClusterTopologyProperties.java b/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/cluster/ClusterTopologyProperties.java new file mode 100644 index 00000000000..194578c92e6 --- /dev/null +++ b/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/cluster/ClusterTopologyProperties.java @@ -0,0 +1,84 @@ +package org.apache.hadoop.yarn.dtss.cluster; + +import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting; +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; + +import java.util.List; + +/** + * Represents the cluster topology as a 2D array. + * Each row in the 2D array represents a rack, and + * each entry in each row represents a machine. + * The value of each entry represents the number + * of containers available on the machine. + * Serializes to and from JSON. + * Also contains information on how a container is defined. + */ +@InterfaceAudience.Public +@InterfaceStability.Unstable +public final class ClusterTopologyProperties { + private List> rackNodeContainers; + private int containerMB; + private int containerVCores; + private int containerCount; + + private ClusterTopologyProperties() { + } + + @InterfaceAudience.Private + @VisibleForTesting + public ClusterTopologyProperties( + final List> rackNodeContainers, + final int containerMB, + final int containerVCores + ) { + this.rackNodeContainers = rackNodeContainers; + this.containerMB = containerMB; + this.containerVCores = containerVCores; + int count = 0; + for (final List machineOnRack : rackNodeContainers) { + for (final int containersOnMachine : machineOnRack) { + count += containersOnMachine; + } + } + + this.containerCount = count; + } + + public int getTotalContainers() { + return containerCount; + } + + public List> getRackNodeContainers() { + return rackNodeContainers; + } + + public int getClusterRacks() { + return rackNodeContainers.size(); + } + + public int getClusterNodes() { + return rackNodeContainers.stream().map(List::size).mapToInt(Integer::intValue).sum(); + } + + public int getClusterContainers() { + return rackNodeContainers.stream().flatMapToInt(l->l.stream().mapToInt(Integer::new)).sum(); + } + + public void setContainerMB(int containerMB) { + this.containerMB = containerMB; + } + + public int getContainerMB() { + return containerMB; + } + + public void setContainerVCores(int containerVCores) { + this.containerVCores = containerVCores; + } + + public int getContainerVCores() { + return containerVCores; + } +} diff --git a/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/cluster/NodeRepresentation.java b/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/cluster/NodeRepresentation.java new file mode 100644 index 00000000000..fad4fc54e5e --- /dev/null +++ b/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/cluster/NodeRepresentation.java @@ -0,0 +1,26 @@ +package org.apache.hadoop.yarn.dtss.cluster; + +import org.apache.hadoop.classification.InterfaceAudience; + +/** + * Represents a mocked node in the cluster. + * Stores its host name and the number of containers registered on the node. + */ +@InterfaceAudience.Private +public final class NodeRepresentation { + private final int containersOnNode; + private final String nodeHostName; + + NodeRepresentation(final String nodeHostName, final int containersOnNode) { + this.nodeHostName = nodeHostName; + this.containersOnNode = containersOnNode; + } + + public int getContainersOnNode() { + return containersOnNode; + } + + public String getNodeHostName() { + return nodeHostName; + } +} diff --git a/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/config/SimulatorModule.java b/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/config/SimulatorModule.java new file mode 100644 index 00000000000..c7f1f596a89 --- /dev/null +++ b/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/config/SimulatorModule.java @@ -0,0 +1,233 @@ +package org.apache.hadoop.yarn.dtss.config; + +import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting; +import com.google.gson.Gson; +import com.google.inject.AbstractModule; +import com.google.inject.multibindings.Multibinder; +import com.google.inject.util.Providers; +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem; +import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.apache.hadoop.yarn.dtss.config.parameters.*; +import org.apache.hadoop.yarn.dtss.time.Clock; +import org.apache.hadoop.yarn.dtss.time.SimulatedClock; +import org.apache.hadoop.yarn.dtss.trace.sls.config.SLSTraceModule; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration; + +import java.io.FileReader; +import java.io.IOException; +import java.text.MessageFormat; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.logging.Level; +import java.util.logging.Logger; + +/** + * The module configuration for running discrete event-based simulation. + * Serialized from JSON. + */ +@InterfaceAudience.Public +@InterfaceStability.Unstable +public final class SimulatorModule extends AbstractModule { + public static final class Constants { + public static final String TRACE_TYPE = "traceType"; + + final static HashMap> TRACE_MODULE_MAP = + new HashMap>() {{ + put(SLSTraceModule.TRACE_TYPE, SLSTraceModule.class); + }}; + } + + private static final Logger LOG = Logger.getLogger(SimulatorModule.class.getName()); + + private static final Gson GSON = new Gson(); + + // The ordering policy to use + private String scheduler = CapacitySchedulerConfiguration.FIFO_APP_ORDERING_POLICY; + + private String clusterTopologyFilePath; + + private Long randomSeed = null; + + private boolean isMetricsOn = false; + + private int metricsTimerWindowSize = 100; + + private int metricsRecordIntervalSeconds = 5; + + private String metricsOutputDirectory = null; + + private Long simulationTraceStartTime = 0L; + + private Long simulationDurationMinutes = null; + + private String simulatorLogLevel = Level.INFO.getName(); + + private boolean isUnmanaged = true; + + // Initialized by traceConfig, available to be injected in test + private AbstractModule traceModule = null; + + private Map traceConfig = new HashMap<>(); + + private List metricsSet = new ArrayList<>(); + + /** + * @return a new {@link Reader} + */ + public static Reader newReader() { + return new SimulatorModule().new Reader(); + } + + private SimulatorModule() { + } + + @InterfaceAudience.Private + @VisibleForTesting + public static SimulatorModule newModule( + final long randomSeed, + final long simulationDurationMinutes, + final String clusterTopologyFilePath, + final String metricsOutputDirectory, + final AbstractModule traceModule, + final boolean isUnmanaged, + final List metricsSet + ){ + final SimulatorModule m = new SimulatorModule(); + m.randomSeed = randomSeed; + m.simulationDurationMinutes = simulationDurationMinutes; + m.clusterTopologyFilePath = clusterTopologyFilePath; + m.metricsOutputDirectory = metricsOutputDirectory; + m.metricsSet = metricsSet; + m.traceModule = traceModule; + m.isMetricsOn = true; + m.isUnmanaged = isUnmanaged; + return m; + } + + /** + * Binds configurations to annotations for dependency injection. + */ + @Override + protected void configure() { + bind(Clock.class).to(SimulatedClock.class); + + bind(Long.class).annotatedWith(SimulationDurationMinutes.class) + .toProvider(Providers.of(simulationDurationMinutes)); + + bind(Long.class).annotatedWith(RandomSeed.class) + .toProvider(Providers.of(randomSeed)); + + bind(String.class).annotatedWith(ClusterTopologyFilePath.class) + .toInstance(clusterTopologyFilePath); + + bind(Boolean.class).annotatedWith(IsMetricsOn.class) + .toInstance(isMetricsOn); + + bind(String.class).annotatedWith(RootMetricsOutputDirectory.class) + .toProvider(Providers.of(metricsOutputDirectory)); + + bind(Integer.class).annotatedWith(MetricsTimerWindowSize.class) + .toProvider(Providers.of(metricsTimerWindowSize)); + + bind(Integer.class).annotatedWith(MetricsRecordIntervalSeconds.class) + .toProvider(Providers.of(metricsRecordIntervalSeconds)); + + bind(Boolean.class).annotatedWith(IsUnmanaged.class) + .toInstance(isUnmanaged); + + bind(Long.class).annotatedWith(SimulationTraceStartTime.class) + .toProvider(Providers.of(simulationTraceStartTime)); + + final Multibinder multibinder = Multibinder.newSetBinder(binder(), String.class, MetricsSet.class); + for (final String metric : metricsSet) { + multibinder.addBinding().toProvider(Providers.of(metric)); + } + + LOG.log(Level.INFO, clusterTopologyFilePath); + + final YarnConfiguration conf = new YarnConfiguration(); + if (scheduler != null) { + setSchedulerConfigs(conf); + } + + conf.set(CapacitySchedulerConfiguration.NODE_LOCALITY_DELAY, "-1"); + conf.set(CapacitySchedulerConfiguration.MAX_ASSIGN_PER_HEARTBEAT, "-1"); + conf.set(CapacitySchedulerConfiguration.OFFSWITCH_PER_HEARTBEAT_LIMIT, "10000"); + + bind(Configuration.class).toInstance(conf); + + QueueMetrics.clearQueueMetrics(); + DefaultMetricsSystem.setMiniClusterMode(true); + + if (traceModule != null) { + LOG.log(Level.INFO, "Trace module is available!"); + install(traceModule); + } else { + LOG.log(Level.INFO, "Trace module is not available, initializing through parsed value..."); + install(parseTraceConfig()); + } + } + + private void setSchedulerConfigs(final YarnConfiguration conf) { + final CapacitySchedulerConfiguration capConf = new CapacitySchedulerConfiguration(); + for (final String queue : capConf.getQueues("root")) { + final String queuePath = String.join(".", "root", queue); + final String orderingPolicyPrefix = CapacitySchedulerConfiguration.PREFIX + queuePath + "." + + CapacitySchedulerConfiguration.ORDERING_POLICY; + conf.set(orderingPolicyPrefix, scheduler); + } + } + + public Level getSimulatorLogLevel() { + return Level.parse(simulatorLogLevel); + } + + public void setMetricsOutputDirectory(final String metricsOutputDirectory) { + this.metricsOutputDirectory = metricsOutputDirectory; + } + + /** + * Parses the trace module to use, currently only SLS trace available. + * @return An {@link AbstractModule} describing parameters for trace objects + */ + private AbstractModule parseTraceConfig() { + // Get the module class + final Class moduleClass = + Constants.TRACE_MODULE_MAP.getOrDefault( + traceConfig.getOrDefault(Constants.TRACE_TYPE, SLSTraceModule.TRACE_TYPE), + SLSTraceModule.class + ); + + LOG.log(Level.INFO, MessageFormat.format( + "Initializing the trace module: {0}", moduleClass.getName())); + + // Convert the traceConfig dictionary to JSON then to the specified module class + return GSON.fromJson(GSON.toJsonTree(traceConfig), moduleClass); + } + + /** + * This is the reader that reads in the JSON configuration for simulations. + */ + public final class Reader { + private Reader() { + } + + /** + * Reads in the JSON configuration at the specified {@code configurationPath}. + * + * @param configurationPath the configuration path + * @return {@link SimulatorModule} object + * @throws IOException when the reader fails to read the file + */ + public SimulatorModule readConfiguration(final String configurationPath) + throws IOException { + return GSON.fromJson(new FileReader(configurationPath), SimulatorModule.class); + } + } +} diff --git a/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/config/parameters/ClusterTopologyFilePath.java b/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/config/parameters/ClusterTopologyFilePath.java new file mode 100644 index 00000000000..460d94f62c3 --- /dev/null +++ b/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/config/parameters/ClusterTopologyFilePath.java @@ -0,0 +1,19 @@ +package org.apache.hadoop.yarn.dtss.config.parameters; + + +import com.google.inject.BindingAnnotation; + +import java.lang.annotation.Retention; +import java.lang.annotation.Target; + +import static java.lang.annotation.ElementType.*; +import static java.lang.annotation.RetentionPolicy.RUNTIME; + +/** + * Annotation for the cluster topology file path. + */ +@BindingAnnotation +@Target({FIELD, PARAMETER, METHOD}) +@Retention(RUNTIME) +public @interface ClusterTopologyFilePath { +} diff --git a/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/config/parameters/IsMetricsOn.java b/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/config/parameters/IsMetricsOn.java new file mode 100644 index 00000000000..a7c12272d03 --- /dev/null +++ b/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/config/parameters/IsMetricsOn.java @@ -0,0 +1,19 @@ +package org.apache.hadoop.yarn.dtss.config.parameters; + + +import com.google.inject.BindingAnnotation; + +import java.lang.annotation.Retention; +import java.lang.annotation.Target; + +import static java.lang.annotation.ElementType.*; +import static java.lang.annotation.RetentionPolicy.RUNTIME; + +/** + * Annotation to determine whether or not metrics should be turned on. + */ +@BindingAnnotation +@Target({FIELD, PARAMETER, METHOD}) +@Retention(RUNTIME) +public @interface IsMetricsOn { +} diff --git a/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/config/parameters/IsUnmanaged.java b/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/config/parameters/IsUnmanaged.java new file mode 100644 index 00000000000..1dc94ca507a --- /dev/null +++ b/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/config/parameters/IsUnmanaged.java @@ -0,0 +1,19 @@ +package org.apache.hadoop.yarn.dtss.config.parameters; + + +import com.google.inject.BindingAnnotation; + +import java.lang.annotation.Retention; +import java.lang.annotation.Target; + +import static java.lang.annotation.ElementType.*; +import static java.lang.annotation.RetentionPolicy.RUNTIME; + +/** + * Annotation to determine whether or not unmanaged AMs should be used. + */ +@BindingAnnotation +@Target({FIELD, PARAMETER, METHOD}) +@Retention(RUNTIME) +public @interface IsUnmanaged { +} diff --git a/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/config/parameters/MetricsRecordIntervalSeconds.java b/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/config/parameters/MetricsRecordIntervalSeconds.java new file mode 100644 index 00000000000..b3c198613d6 --- /dev/null +++ b/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/config/parameters/MetricsRecordIntervalSeconds.java @@ -0,0 +1,21 @@ +package org.apache.hadoop.yarn.dtss.config.parameters; + +import com.google.inject.BindingAnnotation; + +import java.lang.annotation.Retention; +import java.lang.annotation.Target; + +import static java.lang.annotation.ElementType.FIELD; +import static java.lang.annotation.ElementType.METHOD; +import static java.lang.annotation.ElementType.PARAMETER; +import static java.lang.annotation.RetentionPolicy.RUNTIME; + +/** + * Annotation to determine how often metrics should be recorded. + */ +@BindingAnnotation +@Target({FIELD, PARAMETER, METHOD}) +@Retention(RUNTIME) +public @interface MetricsRecordIntervalSeconds { +} + diff --git a/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/config/parameters/MetricsSet.java b/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/config/parameters/MetricsSet.java new file mode 100644 index 00000000000..a7b28e3e09f --- /dev/null +++ b/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/config/parameters/MetricsSet.java @@ -0,0 +1,21 @@ +package org.apache.hadoop.yarn.dtss.config.parameters; + +import com.google.inject.BindingAnnotation; + +import java.lang.annotation.Retention; +import java.lang.annotation.Target; + +import static java.lang.annotation.ElementType.FIELD; +import static java.lang.annotation.ElementType.METHOD; +import static java.lang.annotation.ElementType.PARAMETER; +import static java.lang.annotation.RetentionPolicy.RUNTIME; + +/** + * Annotation to determine the set of metrics to record. + */ +@BindingAnnotation +@Target({FIELD, PARAMETER, METHOD}) +@Retention(RUNTIME) +public @interface MetricsSet { +} + diff --git a/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/config/parameters/MetricsTimerWindowSize.java b/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/config/parameters/MetricsTimerWindowSize.java new file mode 100644 index 00000000000..af0964d34f4 --- /dev/null +++ b/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/config/parameters/MetricsTimerWindowSize.java @@ -0,0 +1,20 @@ +package org.apache.hadoop.yarn.dtss.config.parameters; + +import com.google.inject.BindingAnnotation; + +import java.lang.annotation.Retention; +import java.lang.annotation.Target; + +import static java.lang.annotation.ElementType.FIELD; +import static java.lang.annotation.ElementType.METHOD; +import static java.lang.annotation.ElementType.PARAMETER; +import static java.lang.annotation.RetentionPolicy.RUNTIME; + +/** + * Annotation to determine the window size of gathering metrics. + */ +@BindingAnnotation +@Target({FIELD, PARAMETER, METHOD}) +@Retention(RUNTIME) +public @interface MetricsTimerWindowSize { +} diff --git a/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/config/parameters/RandomSeed.java b/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/config/parameters/RandomSeed.java new file mode 100644 index 00000000000..23a8c6c5383 --- /dev/null +++ b/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/config/parameters/RandomSeed.java @@ -0,0 +1,18 @@ +package org.apache.hadoop.yarn.dtss.config.parameters; + +import com.google.inject.BindingAnnotation; + +import java.lang.annotation.Retention; +import java.lang.annotation.Target; + +import static java.lang.annotation.ElementType.*; +import static java.lang.annotation.RetentionPolicy.RUNTIME; + +/** + * Annotation to determine the random seed to use. + */ +@BindingAnnotation +@Target({FIELD, PARAMETER, METHOD}) +@Retention(RUNTIME) +public @interface RandomSeed { +} \ No newline at end of file diff --git a/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/config/parameters/RootMetricsOutputDirectory.java b/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/config/parameters/RootMetricsOutputDirectory.java new file mode 100644 index 00000000000..80af4f63058 --- /dev/null +++ b/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/config/parameters/RootMetricsOutputDirectory.java @@ -0,0 +1,21 @@ +package org.apache.hadoop.yarn.dtss.config.parameters; + +import com.google.inject.BindingAnnotation; + +import java.lang.annotation.Retention; +import java.lang.annotation.Target; + +import static java.lang.annotation.ElementType.FIELD; +import static java.lang.annotation.ElementType.METHOD; +import static java.lang.annotation.ElementType.PARAMETER; +import static java.lang.annotation.RetentionPolicy.RUNTIME; + +/** + * Annotation on where to put metrics. + */ +@BindingAnnotation +@Target({FIELD, PARAMETER, METHOD}) +@Retention(RUNTIME) +public @interface RootMetricsOutputDirectory { +} + diff --git a/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/config/parameters/SimulationDurationMinutes.java b/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/config/parameters/SimulationDurationMinutes.java new file mode 100644 index 00000000000..aaeb58fe80e --- /dev/null +++ b/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/config/parameters/SimulationDurationMinutes.java @@ -0,0 +1,18 @@ +package org.apache.hadoop.yarn.dtss.config.parameters; + +import com.google.inject.BindingAnnotation; + +import java.lang.annotation.Retention; +import java.lang.annotation.Target; + +import static java.lang.annotation.ElementType.*; +import static java.lang.annotation.RetentionPolicy.RUNTIME; + +/** + * Annotation to determine how long (in discrete event time) the simulation should run for. + */ +@BindingAnnotation +@Target({FIELD, PARAMETER, METHOD}) +@Retention(RUNTIME) +public @interface SimulationDurationMinutes { +} \ No newline at end of file diff --git a/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/config/parameters/SimulationTraceStartTime.java b/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/config/parameters/SimulationTraceStartTime.java new file mode 100644 index 00000000000..d1a70149416 --- /dev/null +++ b/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/config/parameters/SimulationTraceStartTime.java @@ -0,0 +1,18 @@ +package org.apache.hadoop.yarn.dtss.config.parameters; + +import com.google.inject.BindingAnnotation; + +import java.lang.annotation.Retention; +import java.lang.annotation.Target; + +import static java.lang.annotation.ElementType.*; +import static java.lang.annotation.RetentionPolicy.RUNTIME; + +/** + * Annotation to determine the UNIX start time stamp of the simulation. + */ +@BindingAnnotation +@Target({FIELD, PARAMETER, METHOD}) +@Retention(RUNTIME) +public @interface SimulationTraceStartTime { +} \ No newline at end of file diff --git a/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/config/parameters/runner/RunId.java b/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/config/parameters/runner/RunId.java new file mode 100644 index 00000000000..972b8f16a36 --- /dev/null +++ b/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/config/parameters/runner/RunId.java @@ -0,0 +1,19 @@ +package org.apache.hadoop.yarn.dtss.config.parameters.runner; + + +import com.google.inject.BindingAnnotation; + +import java.lang.annotation.Retention; +import java.lang.annotation.Target; + +import static java.lang.annotation.ElementType.*; +import static java.lang.annotation.RetentionPolicy.RUNTIME; + +/** + * Annotation for the ID of the current simulation run. + */ +@BindingAnnotation +@Target({FIELD, PARAMETER, METHOD}) +@Retention(RUNTIME) +public @interface RunId { +} diff --git a/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/config/parameters/runner/SimulationConfigPath.java b/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/config/parameters/runner/SimulationConfigPath.java new file mode 100644 index 00000000000..27798fca5d8 --- /dev/null +++ b/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/config/parameters/runner/SimulationConfigPath.java @@ -0,0 +1,18 @@ +package org.apache.hadoop.yarn.dtss.config.parameters.runner; + +import com.google.inject.BindingAnnotation; + +import java.lang.annotation.Retention; +import java.lang.annotation.Target; + +import static java.lang.annotation.ElementType.*; +import static java.lang.annotation.RetentionPolicy.RUNTIME; + +/** + * Annotation for the path to the simulation configuration file. + */ +@BindingAnnotation +@Target({FIELD, PARAMETER, METHOD}) +@Retention(RUNTIME) +public @interface SimulationConfigPath { +} diff --git a/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/exceptions/EndOfExperimentNotificationException.java b/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/exceptions/EndOfExperimentNotificationException.java new file mode 100644 index 00000000000..fc3cb10e5cc --- /dev/null +++ b/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/exceptions/EndOfExperimentNotificationException.java @@ -0,0 +1,9 @@ +package org.apache.hadoop.yarn.dtss.exceptions; + +/** + * This Exception is not really an error, but rather + * a notification used to deliver the signal that + * the experiment end time has been reached. + */ +public final class EndOfExperimentNotificationException extends Exception { +} diff --git a/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/exceptions/OperationNotSupportedException.java b/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/exceptions/OperationNotSupportedException.java new file mode 100644 index 00000000000..bcc65ff9a0f --- /dev/null +++ b/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/exceptions/OperationNotSupportedException.java @@ -0,0 +1,16 @@ +package org.apache.hadoop.yarn.dtss.exceptions; + +/** + * The {@link Exception} thrown when an operation is not supported. + */ +public final class OperationNotSupportedException extends RuntimeException { + public OperationNotSupportedException(final String message) { + super(message); + } + + public OperationNotSupportedException( + final String message, final Throwable throwable) { + super(message, throwable); + } +} + diff --git a/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/exceptions/StateException.java b/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/exceptions/StateException.java new file mode 100644 index 00000000000..0e48935308c --- /dev/null +++ b/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/exceptions/StateException.java @@ -0,0 +1,15 @@ +package org.apache.hadoop.yarn.dtss.exceptions; + +/** + * The {@link Exception} thrown when a lifeCycleState machine is in an invalid lifeCycleState. + */ +public final class StateException extends RuntimeException { + public StateException(final String message) { + super(message); + } + + public StateException( + final String message, final Throwable throwable) { + super(message, throwable); + } +} diff --git a/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/job/ExperimentJobEndState.java b/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/job/ExperimentJobEndState.java new file mode 100644 index 00000000000..aa65dffafc1 --- /dev/null +++ b/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/job/ExperimentJobEndState.java @@ -0,0 +1,14 @@ +package org.apache.hadoop.yarn.dtss.job; + +/** + * The end states of jobs at the end of a simulation experiment. + */ +public enum ExperimentJobEndState { + READ, + NOT_SUBMITTED, + SUBMITTED, + STARTED, + COMPLETED, + CANCELLED, + FAILED, +} diff --git a/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/job/JobSubmissionParameters.java b/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/job/JobSubmissionParameters.java new file mode 100644 index 00000000000..606af06bacb --- /dev/null +++ b/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/job/JobSubmissionParameters.java @@ -0,0 +1,98 @@ +package org.apache.hadoop.yarn.dtss.job; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.yarn.api.records.Priority; + +/** + * Object used when submitting jobs. + */ +@InterfaceAudience.Public +@InterfaceStability.Unstable +public final class JobSubmissionParameters { + private final Builder builder; + + private JobSubmissionParameters(final Builder builder) { + this.builder = builder; + } + + boolean getIsUnmanaged() { + return builder.isUnmanaged; + } + + int getMasterMemory() { + return builder.masterMemory; + } + + String getName() { + return builder.name; + } + + String getUser() { + return builder.user; + } + + String getQueue() { + return builder.queue; + } + + Priority getPriority() { + return builder.priority; + } + + public static final class Builder { + private boolean isUnmanaged = false; + private int masterMemory = 1; + private String name; + private String user; + private String queue = null; + private Priority priority = null; + + private Builder() { } + + static Builder newInstance() { + return new Builder(); + } + + Builder setQueue(final String queue) { + this.queue = queue; + return this; + } + + Builder setUser(final String user) { + this.user = user; + return this; + } + + Builder setName(final String name) { + this.name = name; + return this; + } + + Builder setIsUnmanaged(final boolean isUnmanaged) { + this.isUnmanaged = isUnmanaged; + return this; + } + + Builder setMasterMemory(final int masterMemory) { + this.masterMemory = masterMemory; + return this; + } + + Builder setPriority(final Priority priority) { + this.priority = priority; + return this; + } + + JobSubmissionParameters build() { + return new JobSubmissionParameters(new Builder() + .setMasterMemory(masterMemory) + .setIsUnmanaged(isUnmanaged) + .setName(name) + .setUser(user) + .setQueue(queue) + .setPriority(priority) + ); + } + } +} diff --git a/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/job/SimulatedAllocateResponse.java b/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/job/SimulatedAllocateResponse.java new file mode 100644 index 00000000000..95b869e8e12 --- /dev/null +++ b/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/job/SimulatedAllocateResponse.java @@ -0,0 +1,38 @@ +package org.apache.hadoop.yarn.dtss.job; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse; +import org.apache.hadoop.yarn.api.records.Container; +import org.apache.hadoop.yarn.api.records.ContainerStatus; + +import java.util.ArrayList; +import java.util.List; + +/** + * A mocked wrapper for allocation responses from the RM. + */ +@InterfaceAudience.Private +final class SimulatedAllocateResponse { + private final List containersAllocated = new ArrayList<>(); + private final List containersCompleted = new ArrayList<>(); + + static SimulatedAllocateResponse fromAllocateResponse(final AllocateResponse response) { + return new SimulatedAllocateResponse(response); + } + + private SimulatedAllocateResponse(final AllocateResponse response) { + for (final Container container : response.getAllocatedContainers()) { + containersAllocated.add(SimulatedContainer.fromContainer(container)); + } + + containersCompleted.addAll(response.getCompletedContainersStatuses()); + } + + public List getContainersAllocated() { + return containersAllocated; + } + + public List getContainersCompleted() { + return containersCompleted; + } +} \ No newline at end of file diff --git a/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/job/SimulatedContainer.java b/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/job/SimulatedContainer.java new file mode 100644 index 00000000000..0bc07b8238b --- /dev/null +++ b/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/job/SimulatedContainer.java @@ -0,0 +1,62 @@ +package org.apache.hadoop.yarn.dtss.job; + +import org.apache.hadoop.yarn.api.records.Container; +import org.apache.hadoop.yarn.api.records.ContainerId; +import org.apache.hadoop.yarn.api.records.NodeId; +import org.apache.hadoop.yarn.api.records.Resource; + +import javax.annotation.Nullable; +import java.time.Instant; + +/** + * This is a simulated container allocated by the RM. + */ +final class SimulatedContainer implements Comparable { + private Instant startTime = null; + private Instant endTime = null; + private final Container container; + private boolean isComplete = false; + + static SimulatedContainer fromContainer(final Container container) { + return new SimulatedContainer(container); + } + + ContainerId getId() { + return container.getId(); + } + + NodeId getNodeId() { + return container.getNodeId(); + } + + Resource getResource() { + return container.getResource(); + } + + int getPriority() { + return container.getPriority().getPriority(); + } + + @Nullable + public Instant getStartTime() { + return startTime; + } + + public void setStartTime(final Instant startTime) { + this.startTime = startTime; + } + + private SimulatedContainer(final Container container) { + this.container = container; + } + + @Override + public int compareTo(final SimulatedContainer o) { + return this.container.compareTo(o.container); + } + + @Override + public int hashCode() { + return this.container.hashCode(); + } +} diff --git a/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/job/SimulatedJob.java b/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/job/SimulatedJob.java new file mode 100644 index 00000000000..87130438cca --- /dev/null +++ b/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/job/SimulatedJob.java @@ -0,0 +1,38 @@ +package org.apache.hadoop.yarn.dtss.job; + +import com.google.inject.Injector; +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; + +import java.util.List; + +/** + * This is the abstraction for a simulated job. + * Users should implement this class when reading their own custom trace, + * or convert the read-in trace into a class implementing this abstraction. + * An example for this class is {@link org.apache.hadoop.yarn.dtss.trace.sls.SLSJob}. + */ +@InterfaceAudience.Public +@InterfaceStability.Unstable +public abstract class SimulatedJob { + + public abstract String getTraceJobId(); + + public abstract String getUser(); + + public abstract String getJobName(); + + public abstract String getQueueName(); + + public abstract Integer getPriority(); + + public abstract List getContainers(); + + public abstract SimulatedJobMaster createJobMaster(Injector injector); + + public abstract Long getSubmitTimeSeconds(); + + public abstract Long getStartTimeSeconds(); + + public abstract Long getEndTimeSeconds(); +} diff --git a/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/job/SimulatedJobMaster.java b/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/job/SimulatedJobMaster.java new file mode 100644 index 00000000000..e2d11f5e5e4 --- /dev/null +++ b/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/job/SimulatedJobMaster.java @@ -0,0 +1,466 @@ +package org.apache.hadoop.yarn.dtss.job; + +import com.google.inject.Injector; +import com.google.inject.Key; +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse; +import org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationMasterRequest; +import org.apache.hadoop.yarn.api.records.*; +import org.apache.hadoop.yarn.dtss.am.MockAM; +import org.apache.hadoop.yarn.dtss.cluster.ClusterState; +import org.apache.hadoop.yarn.dtss.cluster.ClusterTopology; +import org.apache.hadoop.yarn.dtss.cluster.ClusterTopologyBuilder; +import org.apache.hadoop.yarn.dtss.config.parameters.IsUnmanaged; +import org.apache.hadoop.yarn.dtss.exceptions.EndOfExperimentNotificationException; +import org.apache.hadoop.yarn.dtss.job.exceptions.SimulationJobFailedException; +import org.apache.hadoop.yarn.dtss.lifecycle.LifeCycle; +import org.apache.hadoop.yarn.dtss.metrics.job.JobHistoryManager; +import org.apache.hadoop.yarn.dtss.nm.MockNM; +import org.apache.hadoop.yarn.dtss.rm.MockRM; +import org.apache.hadoop.yarn.dtss.time.Clock; +import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp; +import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt; +import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptState; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.AbstractYarnScheduler; + +import java.io.IOException; +import java.text.MessageFormat; +import java.time.Instant; +import java.util.Collection; +import java.util.HashMap; +import java.util.Map; +import java.util.UUID; +import java.util.concurrent.atomic.AtomicLong; +import java.util.logging.Level; +import java.util.logging.Logger; + +/** + * This is the simulated job master, or application master, for a YARN job. + * Users should implement this abstraction to customize the behavior of their + * own application logic. + */ +@InterfaceAudience.Public +@InterfaceStability.Unstable +public abstract class SimulatedJobMaster extends LifeCycle { + private static final Logger LOG = Logger.getLogger(SimulatedJobMaster.class.getName()); + + final Clock clock; + final SimulatedJob job; + final boolean isUnmanaged; + + // How long it takes for the job to spin up in seconds + protected final long jobSpinUpTimeSeconds; + protected final MockRM rm; + protected boolean stopScheduled = false; + + private final JobHistoryManager jobHistoryManager; + + private NodeId masterNode; + private MockAM am; + private RMApp app; + private UUID heartbeatAlarmId; + private boolean isLoggedTryStart = false; + + private final ClusterTopology topology; + private final ClusterState clusterState; + + private final AtomicLong executedContainerTimeSeconds = new AtomicLong(0L); + // This map is used to keep track of allocated containers + private final Map containerTaskMap = new HashMap<>(); + private final long heartbeatSeconds; + + private Long jobStartTime = null; + + SimulatedJobMaster( + final Injector injector, final SimulatedJob job, final long heartbeatSeconds, final long jobSpinUpTimeSeconds) { + this.heartbeatSeconds = heartbeatSeconds; + this.job = job; + this.rm = injector.getInstance(MockRM.class); + this.clock = injector.getInstance(Clock.class); + this.clusterState = injector.getInstance(ClusterState.class); + this.jobHistoryManager = injector.getInstance(JobHistoryManager.class); + this.isUnmanaged = injector.getInstance(Key.get(Boolean.class, IsUnmanaged.class)); + this.jobSpinUpTimeSeconds = jobSpinUpTimeSeconds; + + final ClusterTopologyBuilder topologyBuilder = injector.getInstance(ClusterTopologyBuilder.class); + try { + this.topology = topologyBuilder.getClusterTopology(); + } catch (final IOException e) { + e.printStackTrace(); + throw new RuntimeException(e); + } + } + + /** + * Initializes the job, submits the job to the RM, and schedule a job start. + */ + @Override + public void init() { + super.init(); + try { + LOG.log(Level.INFO, MessageFormat.format( + "Submitted job {0} at {1}.", job.getTraceJobId(), clock.getInstant())); + app = submitApp(); + jobHistoryManager.onJobSubmitted(job, app); + LOG.log(Level.INFO, MessageFormat.format("Job {0} assigned ID {1}.", job.getTraceJobId(), + app.getApplicationId())); + } catch (final Exception e) { + e.printStackTrace(); + throw new RuntimeException(e); + } + + final AtomicLong tryStartCount = new AtomicLong(); + // Spin up the job, unschedule to Alarm if spin up fails + clock.schedulePeriodicAlarm(jobSpinUpTimeSeconds, periodicClientAlarm -> { + if (tryStart(tryStartCount)) { + clock.unschedulePeriodicAlarm(periodicClientAlarm.getPeriodicAlarmId()); + } + }); + } + + public String getTraceJobId() { + return job.getTraceJobId(); + } + + /** + * Tries to start the job by checking its state with the RM. + * @param tryStartCount How many times the job has tried to start + * @return Whether or not the job has started successfully + */ + private boolean tryStart(final AtomicLong tryStartCount) { + final RMAppAttempt rmAppAttempt = app.getCurrentAppAttempt(); + + if (!isLoggedTryStart) { + LOG.log(Level.INFO, "Trying to start application " + job.getTraceJobId() + "..."); + isLoggedTryStart = true; + } + + // Kick the scheduler... + clusterState.heartbeatAll(); + ((AbstractYarnScheduler)rm.getResourceScheduler()).update(); + try { + final RMAppAttemptState stateToWait = isUnmanaged ? RMAppAttemptState.LAUNCHED : RMAppAttemptState.ALLOCATED; + if (!MockRM.waitForState(rmAppAttempt, stateToWait, 5)) { + return false; + } + } catch (InterruptedException e) { + e.printStackTrace(); + throw new RuntimeException(e); + } + + start(); + return true; + } + + /** + * Starts the job and register the application by allocating the AM. + * Initializes the periodic heartbeat callbacks on the simulated clock. + */ + @Override + public void start() { + super.start(); + + // Create the AM container and regsiter application with RM + try { + final Container masterContainer = app.getCurrentAppAttempt().getMasterContainer(); + if (masterContainer != null) { + masterNode = masterContainer.getNodeId(); + } else { + if (!isUnmanaged) { + throw new IllegalArgumentException("Expected managed AMs to have a master container!"); + } + + masterNode = null; + } + + am = new MockAM( + rm.getRMContext(), + rm.getApplicationMasterService(), + app.getCurrentAppAttempt().getAppAttemptId()); + + // The AM is launched by default if the AM is unmanaged + if (!isUnmanaged) { + rm.launchAM(am); + } + + LOG.log(Level.INFO, MessageFormat.format( + "Job {0} launched at {1}!", app.getCurrentAppAttempt().getAppAttemptId(), clock.getInstant())); + + am.registerAppAttempt(); + + LOG.log(Level.FINE, MessageFormat.format( + "Job {0} registered with RM!", app.getCurrentAppAttempt().getAppAttemptId())); + } catch (final Exception e) { + e.printStackTrace(); + throw new RuntimeException(e); + } + + try { + onHeartbeat(SimulatedAllocateResponse.fromAllocateResponse(am.doHeartbeat())); + } catch (final SimulationJobFailedException e) { + tearDownAndScheduleFailure(e); + return; + } catch (final Exception e) { + e.printStackTrace(); + throw new RuntimeException(e); + } + + // Schedule heartbeat alarms for the AM + heartbeatAlarmId = clock.schedulePeriodicAlarm(heartbeatSeconds, periodicClientAlarm -> { + try { + LOG.log(Level.FINER, MessageFormat.format("{0} heartbeat!", am.getApplicationAttemptId())); + final AllocateResponse response = am.doHeartbeat(); + onHeartbeat(SimulatedAllocateResponse.fromAllocateResponse(response)); + } catch(final SimulationJobFailedException e) { + tearDownAndScheduleFailure(e); + } catch (final Exception e) { + e.printStackTrace(); + throw new RuntimeException(e); + } + }); + + // Record in job history that the job has started + jobHistoryManager.onJobStarted(job); + } + + String getAttemptId() { + if (app == null) { + return null; + } + + return app.getCurrentAppAttempt().getAppAttemptId().toString(); + } + + /** + * Request containers from the resource manager by adding requests to the heartbeat. + * Call this from implementing classes. + * @param tasks The simulated tasks of the job + */ + void requestContainers(final Collection tasks) { + clock.scheduleAlarm(heartbeatSeconds, alarm-> { + try{ + am.addRequests(new String[] { "*" }, topology.getContainerMB(), 0, tasks.size()); + onHeartbeat(SimulatedAllocateResponse.fromAllocateResponse(am.schedule())); + } catch (Exception e) { + e.printStackTrace(); + throw new RuntimeException(e); + } + }); + } + + /** + * Simulate the run time of the container by specifying when it finishes. + * Call this from implementing classes. + * @param container The container allocated to the simulated job master. + * @param task The task assigned to the container by the job. + */ + void runContainer(final SimulatedContainer container, final SimulatedTask task) { + final MockNM containerNM = clusterState.getNM(container.getNodeId()); + if (jobStartTime == null) { + final Instant jobStartInstant = clock.getInstant(); + jobStartTime = jobStartInstant.getEpochSecond(); + LOG.log(Level.INFO, "Job " + job.getTraceJobId() + "'s first container started at " + jobStartInstant + "!"); + } + + try { + LOG.log(Level.FINER, MessageFormat.format( + "Trigger run for container {0}!", container.getId().getContainerId())); + + // Tell the node that the task is running + containerNM.nodeHeartbeat( + app.getCurrentAppAttempt().getAppAttemptId(), + container.getId().getContainerId(), + ContainerState.RUNNING); + + rm.drainEvents(); + + // Set the start time of the container + container.setStartTime(clock.getInstant()); + // Mark that the container has been allocated + containerTaskMap.put(container, task); + + LOG.log(Level.FINER, "Container " + container.getId() + " running at " + clock.getInstant()); + + } catch (final Exception e) { + e.printStackTrace(); + throw new RuntimeException(e); + } + + // Schedule an alarm later at the completion time of the task + // Currently we don't fully support failed tasks + clock.scheduleAlarm(task.getDurationSeconds(), alarm -> { + try { + containerNM.nodeHeartbeat( + app.getCurrentAppAttempt().getAppAttemptId(), + container.getId().getContainerId(), + ContainerState.COMPLETE); + + rm.drainEvents(); + + final SimulatedTask removedTask = containerTaskMap.remove(container); + if (container.getStartTime() == null) { + throw new IllegalStateException("Container must have started to complete!"); + } + + final long executionSeconds = alarm.getInstant().getEpochSecond() - + container.getStartTime().getEpochSecond(); + + executedContainerTimeSeconds.addAndGet(executionSeconds); + assert removedTask == task; + + LOG.log(Level.FINER, MessageFormat.format( + "Container {0} completed at {1}!", container.getId(), clock.getInstant())); + } catch (final Exception e) { + e.printStackTrace(); + throw new RuntimeException(e); + } + }); + } + + abstract JobSubmissionParameters getJobSubmissionParameters(); + + abstract void onHeartbeat(final SimulatedAllocateResponse simulatedAllocateResponse); + + RMApp submitApp() throws Exception { + final JobSubmissionParameters jobSubmissionParameters = getJobSubmissionParameters(); + return rm.submitApp( + jobSubmissionParameters.getMasterMemory(), + jobSubmissionParameters.getName(), + jobSubmissionParameters.getUser(), + jobSubmissionParameters.getIsUnmanaged(), + jobSubmissionParameters.getQueue(), + jobSubmissionParameters.getPriority() + ); + } + + void tearDownAndScheduleFailure(final Exception e) { + if (stopScheduled) { + return; + } + + stopScheduled = true; + LOG.log(Level.INFO, MessageFormat.format( + "Job {0} failed due to exception {1} at {2}! Tearing down job...", + getTraceJobId(), e.getMessage(), clock.getInstant())); + + if (heartbeatAlarmId != null) { + clock.unschedulePeriodicAlarm(heartbeatAlarmId); + } + + clock.scheduleAlarm(1, alarm -> stop(e)); + } + + void teardownAndScheduleStop() { + if (stopScheduled) { + return; + } + + stopScheduled = true; + clock.unschedulePeriodicAlarm(heartbeatAlarmId); + clock.scheduleAlarm(1, alarm -> stop()); + } + + @Override + public void stop(final Exception e) { + final String traceJobId = job.getTraceJobId(); + final boolean isEndOfExperiment = e instanceof EndOfExperimentNotificationException; + if (am == null && !isEndOfExperiment) { + throw new RuntimeException("AM not assigned to a valid job!"); + } + + if (am != null) { + final ApplicationAttemptId attemptId = am.getApplicationAttemptId(); + + try { + if (!isEndOfExperiment) { + LOG.log(Level.INFO, MessageFormat.format("Failing job {0}...", attemptId)); + + final FinishApplicationMasterRequest req = + FinishApplicationMasterRequest.newInstance( + FinalApplicationStatus.FAILED, "", ""); + + am.unregisterAppAttempt(req, false); + + rm.drainEvents(); + + if (!isUnmanaged) { + // Tell the RM that the AM container is complete if AM is managed by RM + clusterState.getNM(masterNode).nodeHeartbeat( + am.getApplicationAttemptId(), 1, ContainerState.COMPLETE); + } + + rm.drainEvents(); + + LOG.log(Level.INFO, MessageFormat.format("Job {0} containers released!", attemptId)); + + rm.waitForState(attemptId, RMAppAttemptState.FINISHED); + + LOG.log(Level.INFO, MessageFormat.format("Job {0} FAILED!", traceJobId)); + } else { + LOG.log(Level.INFO, "Job " + traceJobId + " STOPPED at end of experiment"); + } + } catch (final Exception innerE) { + e.printStackTrace(); + throw new RuntimeException(innerE); + } + } + + if (e instanceof EndOfExperimentNotificationException) { + super.stop(); + + for (final SimulatedContainer container : containerTaskMap.keySet()) { + final Instant containerStartTime = container.getStartTime(); + final long executionSeconds; + if (containerStartTime != null) { + executionSeconds = clock.getInstant().getEpochSecond() - + container.getStartTime().getEpochSecond(); + } else { + executionSeconds = 0L; + } + + executedContainerTimeSeconds.addAndGet(executionSeconds); + } + + jobHistoryManager.onJobNotCompletedByEndOfExperiment(job); + } else { + // TODO: Handle retries --- a job can have multiple attempts + super.stop(e); + jobHistoryManager.onJobFailed(job); + } + } + + @Override + public void stop() { + final ApplicationAttemptId attemptId = am.getApplicationAttemptId(); + + try { + am.unregisterAppAttempt(); + + if (!isUnmanaged) { + // Release the AM container if managed by RM + rm.waitForState(am.getApplicationAttemptId(), RMAppAttemptState.FINISHING); + LOG.log(Level.INFO, MessageFormat.format("Job {0} containers released!", attemptId)); + clusterState.getNM(masterNode).nodeHeartbeat(am.getApplicationAttemptId(), 1, ContainerState.COMPLETE); + } + + rm.waitForState(attemptId, RMAppAttemptState.FINISHED); + + LOG.log(Level.INFO, MessageFormat.format( + "Job " + job.getTraceJobId() + " FINISHED at {1}!", attemptId, clock.getInstant()) + ); + } catch (final Exception e) { + e.printStackTrace(); + throw new RuntimeException(e); + } + + super.stop(); + jobHistoryManager.onJobCompleted(job); + } + + public abstract String getQueueName(); + + public SimulatedJob getSimulatedJob() { + return job; + } +} diff --git a/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/job/SimulatedMRJobMaster.java b/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/job/SimulatedMRJobMaster.java new file mode 100644 index 00000000000..3d120652265 --- /dev/null +++ b/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/job/SimulatedMRJobMaster.java @@ -0,0 +1,216 @@ +package org.apache.hadoop.yarn.dtss.job; + +import com.google.inject.Injector; +import org.apache.commons.lang.NotImplementedException; +import org.apache.hadoop.yarn.api.records.ContainerId; +import org.apache.hadoop.yarn.api.records.ContainerStatus; +import org.apache.hadoop.yarn.dtss.job.constants.TaskConstants; + +import java.text.MessageFormat; +import java.util.*; +import java.util.logging.Level; +import java.util.logging.Logger; + + +/** + * The simulated ApplicationMaster for a simplified MapReduce job. + */ +public final class SimulatedMRJobMaster extends SimulatedJobMaster { + private static final Logger LOG = Logger.getLogger(SimulatedMRJobMaster.class.getName()); + + enum Phase { + MAP, + MAP_DEPLOYED, + REDUCE, + REDUCE_DEPLOYED, + COMPLETED + } + + private static final long MAP_PHASE_SETUP_SECONDS = 10; + private static final long MR_AM_HEARTBEAT_SECONDS = 10; + private static final long MR_JOB_SPINUP_TIME_SECONDS = 60; + + private final Map containersReceived = new HashMap<>(); + private final Set runningContainers = new HashSet<>(); + private final List mapTasks = new ArrayList<>(); + private final List reduceTasks = new ArrayList<>(); + + private int numCompletedTasksInPhase = 0; + private boolean hasRequestedReduceContainers = false; + + private Phase phase = Phase.MAP; + + public SimulatedMRJobMaster( + final Injector injector, final SimulatedJob job) { + super(injector, job, MR_AM_HEARTBEAT_SECONDS, MR_JOB_SPINUP_TIME_SECONDS); + } + + /** + * Add tasks to the job and set up a call to initialize the job. + * For implementors of a different type of ApplicationMaster, + * remember to call super.init(). + */ + @Override + public void init() { + // Remember to call super.init() to set up alarms to invoke the start method + super.init(); + for (final SimulatedTask task : job.getContainers()) { + switch (task.getTaskType()) { + case TaskConstants.MAP_TASK_TYPE: + mapTasks.add(task); + break; + case TaskConstants.REDUCE_TASK_TYPE: + reduceTasks.add(task); + break; + default: + throw new NotImplementedException(MessageFormat.format( + "Container type {0} is not yet supported!", task.getTaskType())); + } + } + } + + /** + * Begin by requesting map tasks to start the job. + * For implementors of a different type of ApplicationMaster, + * remember to call super.init(). + */ + @Override + public void start() { + super.start(); + if (stopScheduled) { + return; + } + + clock.scheduleAlarm(MAP_PHASE_SETUP_SECONDS, alarm -> { + requestContainers(mapTasks); + }); + } + + /** + * Heartbeat from the RM. Check on the state of the job's tasks. + * Finish when all map and reduce phase tasks are completed. + * @param simulatedAllocateResponse The heartbeat response from the RM. + */ + @Override + void onHeartbeat(final SimulatedAllocateResponse simulatedAllocateResponse) { + switch (phase) { + case MAP: + waitForContainersAndDeployTasks( + simulatedAllocateResponse.getContainersAllocated(), + simulatedAllocateResponse.getContainersCompleted(), + mapTasks, + Phase.MAP_DEPLOYED + ); + break; + case MAP_DEPLOYED: + waitForContainersToComplete( + simulatedAllocateResponse.getContainersCompleted(), + mapTasks, + Phase.REDUCE + ); + break; + case REDUCE: + if (!hasRequestedReduceContainers) { + requestContainers(reduceTasks); + hasRequestedReduceContainers = true; + } + + waitForContainersAndDeployTasks( + simulatedAllocateResponse.getContainersAllocated(), + simulatedAllocateResponse.getContainersCompleted(), + reduceTasks, + Phase.REDUCE_DEPLOYED); + break; + case REDUCE_DEPLOYED: + waitForContainersToComplete( + simulatedAllocateResponse.getContainersCompleted(), + reduceTasks, + Phase.COMPLETED + ); + break; + case COMPLETED: + teardownAndScheduleStop(); + break; + default: + throw new UnsupportedOperationException("Phase not supported!"); + } + } + + @Override + public String getQueueName() { + return job.getQueueName(); + } + + @Override + JobSubmissionParameters getJobSubmissionParameters() { + return JobSubmissionParameters.Builder.newInstance() + .setIsUnmanaged(isUnmanaged) + .setUser(job.getUser()) + .setQueue(job.getQueueName()) + .setName(job.getTraceJobId()) + .build(); + } + + private void waitForContainersAndDeployTasks( + final List newContainersAllocated, + final List completedContainers, + final List tasks, + final Phase transition) { + for (final SimulatedContainer container : newContainersAllocated) { + if (containersReceived.containsKey(container.getId())) { + continue; + } + + runContainer(container, tasks.get(containersReceived.size())); + containersReceived.put(container.getId(), container); + runningContainers.add(container.getId()); + } + + for (final ContainerStatus completedContainer : completedContainers) { + if (!runningContainers.remove(completedContainer.getContainerId())) { + LOG.log(Level.WARNING, MessageFormat.format( + "Removed non-existent container {0}.", completedContainer.getContainerId())); + } else { + numCompletedTasksInPhase++; + } + } + + LOG.log(Level.FINE, containersReceived.size() + " allocated at " + clock.getInstant()); + if (containersReceived.size() < tasks.size()) { + // Do not change state + return; + } + + phase = transition; + LOG.log(Level.INFO, MessageFormat.format( + "Job {0} transitioned to {1} phase!", getAttemptId(), transition)); + } + + private void waitForContainersToComplete( + final List completedContainers, final List tasks, final Phase transition) { + for (final ContainerStatus status : completedContainers) { + if (!runningContainers.remove(status.getContainerId())) { + LOG.log(Level.WARNING, MessageFormat.format( + "Removed non-existent container {0}.", status.getContainerId())); + } else { + numCompletedTasksInPhase++; + } + } + + if (numCompletedTasksInPhase == tasks.size()) { + if (runningContainers.size() > 0) { + throw new RuntimeException("Expected to have no more running containers!"); + } + + LOG.log(Level.INFO, MessageFormat.format( + "All tasks for phase {0} completed for job {1} at {2}! Entering {3} phase...", + phase, getAttemptId(), clock.getInstant(), transition)); + + phase = transition; + numCompletedTasksInPhase = 0; + containersReceived.clear(); + LOG.log(Level.INFO, MessageFormat.format( + "Job {0} transitioned to {1} phase!", getAttemptId(), transition)); + } + } +} diff --git a/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/job/SimulatedTask.java b/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/job/SimulatedTask.java new file mode 100644 index 00000000000..f02358096c1 --- /dev/null +++ b/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/job/SimulatedTask.java @@ -0,0 +1,17 @@ +package org.apache.hadoop.yarn.dtss.job; + +import java.util.UUID; + +/** + * An abstract simulated task. + * Implementors should implement this abstraction for their own tasks spun off from their jobs. + */ +public abstract class SimulatedTask { + public abstract Long getStartTimeSeconds(); + + public abstract Long getDurationSeconds(); + + public abstract Integer getPriority(); + + public abstract String getTaskType(); +} diff --git a/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/job/constants/TaskConstants.java b/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/job/constants/TaskConstants.java new file mode 100644 index 00000000000..b0077cfae83 --- /dev/null +++ b/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/job/constants/TaskConstants.java @@ -0,0 +1,15 @@ +package org.apache.hadoop.yarn.dtss.job.constants; + +/** + * Constants for job tasks. + */ +public final class TaskConstants { + public static final String MAP_TASK_TYPE = "map"; + + public static final String REDUCE_TASK_TYPE = "reduce"; + + public static final String UNKNOWN_TASK_TYPE = "unknown"; + + private TaskConstants() { + } +} diff --git a/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/job/exceptions/SimulationJobFailedException.java b/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/job/exceptions/SimulationJobFailedException.java new file mode 100644 index 00000000000..bbef36b363d --- /dev/null +++ b/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/job/exceptions/SimulationJobFailedException.java @@ -0,0 +1,10 @@ +package org.apache.hadoop.yarn.dtss.job.exceptions; + +/** + * This exception should be thrown when a simulated job experiences a failure. + */ +public abstract class SimulationJobFailedException extends Exception { + SimulationJobFailedException(final String message) { + super(message); + } +} diff --git a/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/lifecycle/InitializedLifeCycle.java b/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/lifecycle/InitializedLifeCycle.java new file mode 100644 index 00000000000..5cfac6e64e1 --- /dev/null +++ b/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/lifecycle/InitializedLifeCycle.java @@ -0,0 +1,15 @@ +package org.apache.hadoop.yarn.dtss.lifecycle; + +/** + * A lifecycle object that has already been initialized + */ +public abstract class InitializedLifeCycle extends LifeCycle { + public InitializedLifeCycle() { + lifeCycleState = lifeCycleState.transition(LifeCycleState.INITED); + } + + @Override + public void init() { + // Do nothing + } +} diff --git a/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/lifecycle/LifeCycle.java b/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/lifecycle/LifeCycle.java new file mode 100644 index 00000000000..1ac75475422 --- /dev/null +++ b/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/lifecycle/LifeCycle.java @@ -0,0 +1,52 @@ +package org.apache.hadoop.yarn.dtss.lifecycle; + +/** + * Represents an object that has a start/stop lifecycle. + */ +public abstract class LifeCycle { + protected LifeCycleState lifeCycleState = LifeCycleState.NOT_INITED; + + /** + * Initializes the lifecycle object. + */ + public void init() { + lifeCycleState = lifeCycleState.transition(LifeCycleState.INITED); + } + + /** + * The entry point of the lifecycle object. + */ + public void start() { + lifeCycleState = lifeCycleState.transition(LifeCycleState.STARTED); + } + + /** + * Stop due to failure. + * @param e the Exception + */ + public void stop(final Exception e) { + lifeCycleState = lifeCycleState.transition(LifeCycleState.FAILED); + } + + /** + * The exit point of the lifecycle object. + */ + public void stop() { + lifeCycleState = lifeCycleState.transition(LifeCycleState.STOPPED); + } + + /** + * Directly transition to state. + * @param state the state + */ + public void transition(final LifeCycleState state) { + lifeCycleState = lifeCycleState.transition(state); + } + + /** + * @return the lifeCycleState of the lifecycle object + */ + public LifeCycleState getState() { + return lifeCycleState; + } +} diff --git a/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/lifecycle/LifeCycleState.java b/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/lifecycle/LifeCycleState.java new file mode 100644 index 00000000000..6c13e18257b --- /dev/null +++ b/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/lifecycle/LifeCycleState.java @@ -0,0 +1,72 @@ +package org.apache.hadoop.yarn.dtss.lifecycle; + + +import org.apache.hadoop.yarn.dtss.exceptions.StateException; + +import java.text.MessageFormat; + +/** + * Tracks the life cycle of an object. + * Used for objects implementing {@link LifeCycle}. + */ +public enum LifeCycleState { + NOT_INITED, + INITED, + STARTED, + STOPPED, + FAILED, + CANCELLED, + UNKNOWN; // Should not enter this state! + + public LifeCycleState transition(final LifeCycleState to) { + if (isValidTransition(to)) { + return to; + } + + throw new StateException( + MessageFormat.format("Invalid lifeCycleState transition from {0} to {1}.", this, to)); + } + + public boolean isValidTransition(final LifeCycleState to) { + if (this == to) { + return true; + } + + switch (this) { + case NOT_INITED: + switch (to) { + case STARTED: + return false; + default: + return true; + } + case INITED: + switch (to) { + case NOT_INITED: + return false; + default: + return true; + } + case STARTED: + switch (to) { + case NOT_INITED: + case INITED: + return false; + default: + return true; + } + case STOPPED: + case FAILED: + case CANCELLED: + return false; + default: + throw new StateException(String.format( + "Invalid lifecycle lifeCycleState %s.", this)); + } + } + + public boolean isDone() { + return this == STOPPED || this == FAILED || this == CANCELLED; + } +} + diff --git a/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/metrics/CapacitySchedulerWrapper.java b/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/metrics/CapacitySchedulerWrapper.java new file mode 100644 index 00000000000..c8bcb57ca31 --- /dev/null +++ b/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/metrics/CapacitySchedulerWrapper.java @@ -0,0 +1,112 @@ +package org.apache.hadoop.yarn.dtss.metrics; + +import com.codahale.metrics.Timer; +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; +import org.apache.hadoop.yarn.api.records.ContainerId; +import org.apache.hadoop.yarn.api.records.ResourceRequest; +import org.apache.hadoop.yarn.api.records.SchedulingRequest; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.AbstractYarnScheduler; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Allocation; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ContainerUpdates; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplication; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEvent; + +import java.util.List; + +/** + * Wrapper around the {@link CapacityScheduler} to enable event-driven simulation. + * Wires metrics for the scheduler. + */ +@InterfaceAudience.Private +public final class CapacitySchedulerWrapper extends CapacityScheduler implements SchedulerWrapper { + private MetricsManager metricsManager; + + public CapacitySchedulerWrapper() { + } + + @Override + public void setMetricsManager(final MetricsManager metricsManager) { + this.metricsManager = metricsManager; + } + + @Override public MetricsManager getMetricsManager() { + return metricsManager; + } + + @Override + public Allocation allocate( + ApplicationAttemptId applicationAttemptId, + List resourceRequests, + List schedulingRequests, + List containerIds, + List strings, + List strings2, + ContainerUpdates updateRequests) { + if (metricsManager.isMetricsOn()) { + final SchedulerApplication app = (SchedulerApplication) ((AbstractYarnScheduler) this).getSchedulerApplications() + .get(applicationAttemptId.getApplicationId()); + + metricsManager.registerQueueMetricsIfNew(app.getQueue()); + + Timer.Context allocateContext = null; + if (metricsManager.getSchedulerAllocateTimer() != null) { + allocateContext = metricsManager.getSchedulerAllocateTimer().time(); + } + + try { + return super.allocate( + applicationAttemptId, + resourceRequests, + schedulingRequests, + containerIds, + strings, + strings2, + updateRequests + ); + } finally { + if (allocateContext != null) { + allocateContext.stop(); + } + metricsManager.increaseSchedulerAllocationCounter(); + } + } else { + return super.allocate(applicationAttemptId, resourceRequests, schedulingRequests, + containerIds, strings, + strings2, updateRequests); + } + } + + @Override + public void handle(SchedulerEvent schedulerEvent) { + if (!metricsManager.isMetricsOn()) { + super.handle(schedulerEvent); + return; + } + + Timer.Context handlerTimer = null; + Timer.Context operationTimer = null; + + try { + if (metricsManager.getSchedulerHandleTimer() != null) { + handlerTimer = metricsManager.getSchedulerHandleTimer().time(); + } + + if (metricsManager.getSchedulerHandleTimer(schedulerEvent.getType()) != null) { + operationTimer = metricsManager.getSchedulerHandleTimer(schedulerEvent.getType()).time(); + } + + super.handle(schedulerEvent); + } finally { + if (handlerTimer != null) { + handlerTimer.stop(); + } + if (operationTimer != null) { + operationTimer.stop(); + } + + metricsManager.increaseSchedulerHandleCounter(schedulerEvent.getType()); + } + } +} diff --git a/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/metrics/FairSchedulerWrapper.java b/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/metrics/FairSchedulerWrapper.java new file mode 100644 index 00000000000..abcaad7ffb1 --- /dev/null +++ b/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/metrics/FairSchedulerWrapper.java @@ -0,0 +1,101 @@ +package org.apache.hadoop.yarn.dtss.metrics; + +import com.codahale.metrics.Timer; +import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; +import org.apache.hadoop.yarn.api.records.ContainerId; +import org.apache.hadoop.yarn.api.records.ResourceRequest; +import org.apache.hadoop.yarn.api.records.SchedulingRequest; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.AbstractYarnScheduler; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Allocation; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ContainerUpdates; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplication; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEvent; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FairScheduler; + +import java.util.List; + +/** + * Wrapper around the {@link FairScheduler} to enable discrete event-driven simulation. + * Wires metrics for scheduler. + */ +public final class FairSchedulerWrapper extends FairScheduler implements SchedulerWrapper { + private MetricsManager metricsManager; + + public FairSchedulerWrapper() { + } + + @Override + public void setMetricsManager(final MetricsManager metricsManager) { + this.metricsManager = metricsManager; + } + + @Override public MetricsManager getMetricsManager() { + return metricsManager; + } + + @Override + public Allocation allocate(ApplicationAttemptId attemptId, + List resourceRequests, + List schedulingRequests, List containerIds, + List blacklistAdditions, List blacklistRemovals, + ContainerUpdates updateRequests) { + if (metricsManager.isMetricsOn()) { + final SchedulerApplication app = (SchedulerApplication) ((AbstractYarnScheduler) this).getSchedulerApplications() + .get(attemptId.getApplicationId()); + + metricsManager.registerQueueMetricsIfNew(app.getQueue()); + + Timer.Context allocateContext = null; + if (metricsManager.getSchedulerAllocateTimer() != null) { + allocateContext = metricsManager.getSchedulerAllocateTimer().time(); + } + + try { + return super.allocate(attemptId, resourceRequests, + schedulingRequests, containerIds, + blacklistAdditions, blacklistRemovals, updateRequests); + } finally { + if (allocateContext != null) { + allocateContext.stop(); + } + metricsManager.increaseSchedulerAllocationCounter(); + } + } else { + return super.allocate(attemptId, resourceRequests, schedulingRequests, + containerIds, + blacklistAdditions, blacklistRemovals, updateRequests); + } + } + + @Override + public void handle(SchedulerEvent schedulerEvent) { + // metrics off + if (!metricsManager.isMetricsOn()) { + super.handle(schedulerEvent); + return; + } + + Timer.Context handlerTimer = null; + Timer.Context operationTimer = null; + + try { + if (metricsManager.getSchedulerHandleTimer() != null) { + handlerTimer = metricsManager.getSchedulerHandleTimer().time(); + } + + if (metricsManager.getSchedulerHandleTimer(schedulerEvent.getType()) != null) { + operationTimer = metricsManager.getSchedulerHandleTimer(schedulerEvent.getType()).time(); + } + + super.handle(schedulerEvent); + } finally { + if (handlerTimer != null) { + handlerTimer.stop(); + } + if (operationTimer != null) { + operationTimer.stop(); + } + metricsManager.increaseSchedulerHandleCounter(schedulerEvent.getType()); + } + } +} diff --git a/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/metrics/MetricsConfiguration.java b/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/metrics/MetricsConfiguration.java new file mode 100644 index 00000000000..09faae1b7a7 --- /dev/null +++ b/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/metrics/MetricsConfiguration.java @@ -0,0 +1,85 @@ +package org.apache.hadoop.yarn.dtss.metrics; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.yarn.dtss.config.parameters.IsMetricsOn; +import org.apache.hadoop.yarn.dtss.config.parameters.MetricsSet; +import org.apache.hadoop.yarn.dtss.config.parameters.RootMetricsOutputDirectory; +import org.apache.hadoop.yarn.dtss.config.parameters.runner.RunId; + +import javax.annotation.Nullable; +import javax.inject.Inject; +import javax.inject.Singleton; +import java.nio.file.Paths; +import java.util.Set; + +/** + * The configuration for metrics. + */ +@InterfaceAudience.Private +@Singleton +public final class MetricsConfiguration { + private final String runId; + private final String rootMetricsOutputDir; + private final String metricsOutputDir; + private final boolean isMetricsOn; + private final Set metricsSet; + + @Inject + private MetricsConfiguration( + @IsMetricsOn final boolean isMetricsOn, + @RunId final String runId, + @MetricsSet final Set metricsSet, + @Nullable @RootMetricsOutputDirectory final String rootMetricsOutputDir) { + this.runId = runId; + this.isMetricsOn = isMetricsOn; + this.metricsSet = metricsSet; + + if (isMetricsOn) { + if (rootMetricsOutputDir == null || rootMetricsOutputDir.trim().isEmpty()) { + throw new IllegalArgumentException("RootMetricsOutputDirectory must be configured if metrics is on!"); + } + + this.rootMetricsOutputDir = rootMetricsOutputDir; + this.metricsOutputDir = Paths.get(rootMetricsOutputDir, runId).toString(); + } else { + this.rootMetricsOutputDir = null; + this.metricsOutputDir = null; + } + } + + public String getMetricsDirectory() { + return metricsOutputDir; + } + + public String getMetricsFilePath(final String... pathNames) { + if (isMetricsOn) { + return Paths.get(metricsOutputDir, pathNames).toAbsolutePath().toString(); + } + + return null; + } + + public String getRunId() { + return runId; + } + + public String getRootMetricsOutputDir() { + return rootMetricsOutputDir; + } + + public boolean isMetricsOn() { + return isMetricsOn; + } + + public Set getMetricsSet() { + return metricsSet; + } + + public boolean metricsContains(final String metric) { + if (metricsSet == null || metricsSet.isEmpty()) { + return true; + } + + return metricsSet.contains(metric); + } +} diff --git a/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/metrics/MetricsCsvReporter.java b/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/metrics/MetricsCsvReporter.java new file mode 100644 index 00000000000..d13d6e6e239 --- /dev/null +++ b/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/metrics/MetricsCsvReporter.java @@ -0,0 +1,265 @@ +package org.apache.hadoop.yarn.dtss.metrics; + +import com.codahale.metrics.*; +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.yarn.dtss.time.Clock; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.*; +import java.nio.charset.Charset; +import java.util.*; +import java.util.concurrent.TimeUnit; + +/** + * Writes metrics in CSV when called on by the clock. + */ +@InterfaceAudience.Private +public final class MetricsCsvReporter implements Reporter { + private static final Logger LOG = LoggerFactory.getLogger(MetricsCsvReporter.class); + private static final Logger LOGGER = LoggerFactory.getLogger(CsvReporter.class); + private static final Charset UTF_8 = Charset.forName("UTF-8"); + private final File directory; + private final Locale locale; + private final CsvFileProvider csvFileProvider; + + private final MetricRegistry registry; + private final MetricFilter filter; + private final long durationFactor; + private final String durationUnit; + private final long rateFactor; + private final String rateUnit; + private final Clock clock; + + private UUID alarmId = null; + + private MetricsCsvReporter( + final Clock clock, + final MetricRegistry registry, + final MetricFilter filter, + final TimeUnit rateUnit, + final TimeUnit durationUnit, + final File directory, + final Locale locale, + final CsvFileProvider csvFileProvider) { + this.clock = clock; + this.registry = registry; + this.filter = filter; + this.rateFactor = rateUnit.toSeconds(1L); + this.rateUnit = this.calculateRateUnit(rateUnit); + this.durationFactor = durationUnit.toNanos(1L); + this.durationUnit = durationUnit.toString().toLowerCase(Locale.US); + this.directory = directory; + this.locale = locale; + this.csvFileProvider = csvFileProvider; + } + + public void start(long period, TimeUnit unit) { + this.start(period, period, unit); + } + + public synchronized void start(long initialDelay, long period, TimeUnit unit) { + final long initialDelaySec = TimeUnit.SECONDS.convert(initialDelay, unit); + final long periodSec = TimeUnit.SECONDS.convert(period, unit); + + alarmId = clock.schedulePeriodicAlarm(initialDelaySec, periodSec, alarm -> { + try { + MetricsCsvReporter.this.report(); + } catch (Throwable var2) { + MetricsCsvReporter.LOG.error( + "Exception thrown from {}#report. Exception was suppressed.", + MetricsCsvReporter.this.getClass().getSimpleName(), var2); + } + }); + } + + public void stop() { + clock.cancelAlarm(alarmId); + } + + public void close() { + this.stop(); + } + + public void report() { + synchronized(this) { + this.report( + this.registry.getGauges(this.filter), + this.registry.getCounters(this.filter), + this.registry.getHistograms(this.filter), + this.registry.getMeters(this.filter), + this.registry.getTimers(this.filter)); + } + } + + public void report( + final SortedMap gauges, + final SortedMap counters, + final SortedMap histograms, + final SortedMap meters, + final SortedMap timers) { + long timestamp = TimeUnit.MILLISECONDS.toSeconds(this.clock.getTime()); + Iterator var8 = gauges.entrySet().iterator(); + + Map.Entry entry; + while(var8.hasNext()) { + entry = (Map.Entry)var8.next(); + this.reportGauge(timestamp, (String)entry.getKey(), (Gauge)entry.getValue()); + } + + var8 = counters.entrySet().iterator(); + + while(var8.hasNext()) { + entry = (Map.Entry)var8.next(); + this.reportCounter(timestamp, (String)entry.getKey(), (Counter)entry.getValue()); + } + + var8 = histograms.entrySet().iterator(); + + while(var8.hasNext()) { + entry = (Map.Entry)var8.next(); + this.reportHistogram(timestamp, (String)entry.getKey(), (Histogram)entry.getValue()); + } + + var8 = meters.entrySet().iterator(); + + while(var8.hasNext()) { + entry = (Map.Entry)var8.next(); + this.reportMeter(timestamp, (String)entry.getKey(), (Meter)entry.getValue()); + } + + var8 = timers.entrySet().iterator(); + + while(var8.hasNext()) { + entry = (Map.Entry)var8.next(); + this.reportTimer(timestamp, (String)entry.getKey(), (com.codahale.metrics.Timer)entry.getValue()); + } + + } + + private void reportTimer(long timestamp, String name, com.codahale.metrics.Timer timer) { + Snapshot snapshot = timer.getSnapshot(); + this.report(timestamp, name, "count,max,mean,min,stddev,p50,p75,p95,p98,p99,p999,mean_rate,m1_rate,m5_rate,m15_rate,rate_unit,duration_unit", "%d,%f,%f,%f,%f,%f,%f,%f,%f,%f,%f,%f,%f,%f,%f,calls/%s,%s", timer.getCount(), this.convertDuration((double)snapshot.getMax()), this.convertDuration(snapshot.getMean()), this.convertDuration((double)snapshot.getMin()), this.convertDuration(snapshot.getStdDev()), this.convertDuration(snapshot.getMedian()), this.convertDuration(snapshot.get75thPercentile()), this.convertDuration(snapshot.get95thPercentile()), this.convertDuration(snapshot.get98thPercentile()), this.convertDuration(snapshot.get99thPercentile()), this.convertDuration(snapshot.get999thPercentile()), this.convertRate(timer.getMeanRate()), this.convertRate(timer.getOneMinuteRate()), this.convertRate(timer.getFiveMinuteRate()), this.convertRate(timer.getFifteenMinuteRate()), this.getRateUnit(), this.getDurationUnit()); + } + + private void reportMeter(long timestamp, String name, Meter meter) { + this.report(timestamp, name, "count,mean_rate,m1_rate,m5_rate,m15_rate,rate_unit", "%d,%f,%f,%f,%f,events/%s", meter.getCount(), this.convertRate(meter.getMeanRate()), this.convertRate(meter.getOneMinuteRate()), this.convertRate(meter.getFiveMinuteRate()), this.convertRate(meter.getFifteenMinuteRate()), this.getRateUnit()); + } + + private void reportHistogram(long timestamp, String name, Histogram histogram) { + Snapshot snapshot = histogram.getSnapshot(); + this.report(timestamp, name, "count,max,mean,min,stddev,p50,p75,p95,p98,p99,p999", "%d,%d,%f,%d,%f,%f,%f,%f,%f,%f,%f", histogram.getCount(), snapshot.getMax(), snapshot.getMean(), snapshot.getMin(), snapshot.getStdDev(), snapshot.getMedian(), snapshot.get75thPercentile(), snapshot.get95thPercentile(), snapshot.get98thPercentile(), snapshot.get99thPercentile(), snapshot.get999thPercentile()); + } + + private void reportCounter(long timestamp, String name, Counter counter) { + this.report(timestamp, name, "count", "%d", counter.getCount()); + } + + private void reportGauge(long timestamp, String name, Gauge gauge) { + this.report(timestamp, name, "value", "%s", gauge.getValue()); + } + + private void report(long timestamp, String name, String header, String line, Object... values) { + try { + File file = this.csvFileProvider.getFile(this.directory, name); + boolean fileAlreadyExists = file.exists(); + if (fileAlreadyExists || file.createNewFile()) { + PrintWriter out = new PrintWriter(new OutputStreamWriter(new FileOutputStream(file, true), UTF_8)); + + try { + if (!fileAlreadyExists) { + out.println("t," + header); + } + + out.printf(this.locale, String.format(this.locale, "%d,%s%n", timestamp, line), values); + } finally { + out.close(); + } + } + } catch (IOException var14) { + LOGGER.warn("Error writing to {}", name, var14); + } + } + + protected String getRateUnit() { + return this.rateUnit; + } + + protected String getDurationUnit() { + return this.durationUnit; + } + + protected double convertDuration(double duration) { + return duration / (double)this.durationFactor; + } + + protected double convertRate(double rate) { + return rate * (double)this.rateFactor; + } + + private String calculateRateUnit(TimeUnit unit) { + String s = unit.toString().toLowerCase(Locale.US); + return s.substring(0, s.length() - 1); + } + + public static Builder newBuilder(final MetricRegistry registry, final Clock clock) { + return new Builder(registry, clock); + } + + public static class Builder { + private final MetricRegistry registry; + private final Clock clock; + private Locale locale; + private TimeUnit rateUnit; + private TimeUnit durationUnit; + private MetricFilter filter; + private CsvFileProvider csvFileProvider; + + private Builder(MetricRegistry registry, Clock clock) { + this.registry = registry; + this.clock = clock; + this.locale = Locale.getDefault(); + this.rateUnit = TimeUnit.SECONDS; + this.durationUnit = TimeUnit.MILLISECONDS; + this.filter = MetricFilter.ALL; + this.csvFileProvider = new FixedNameCsvFileProvider(); + } + + public MetricsCsvReporter.Builder formatFor(Locale locale) { + this.locale = locale; + return this; + } + + public MetricsCsvReporter.Builder convertRatesTo(TimeUnit rateUnit) { + this.rateUnit = rateUnit; + return this; + } + + public MetricsCsvReporter.Builder convertDurationsTo(TimeUnit durationUnit) { + this.durationUnit = durationUnit; + return this; + } + + public MetricsCsvReporter.Builder filter(MetricFilter filter) { + this.filter = filter; + return this; + } + + public MetricsCsvReporter.Builder withCsvFileProvider(CsvFileProvider csvFileProvider) { + this.csvFileProvider = csvFileProvider; + return this; + } + + public MetricsCsvReporter build(File directory) { + return new MetricsCsvReporter( + clock, + registry, + filter, + rateUnit, + durationUnit, + directory, + locale, + csvFileProvider); + } + } +} diff --git a/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/metrics/MetricsManager.java b/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/metrics/MetricsManager.java new file mode 100644 index 00000000000..9bc139ba4ea --- /dev/null +++ b/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/metrics/MetricsManager.java @@ -0,0 +1,502 @@ +package org.apache.hadoop.yarn.dtss.metrics; + +import com.codahale.metrics.Timer; +import com.codahale.metrics.*; +import com.google.inject.Inject; +import com.google.inject.Singleton; +import org.apache.commons.lang.NullArgumentException; +import org.apache.hadoop.yarn.dtss.config.parameters.MetricsRecordIntervalSeconds; +import org.apache.hadoop.yarn.dtss.config.parameters.MetricsTimerWindowSize; +import org.apache.hadoop.yarn.dtss.exceptions.StateException; +import org.apache.hadoop.yarn.dtss.lifecycle.LifeCycle; +import org.apache.hadoop.yarn.dtss.metrics.job.JobHistoryManager; +import org.apache.hadoop.yarn.dtss.stats.ClusterStats; +import org.apache.hadoop.yarn.dtss.time.Clock; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Queue; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEventType; + +import java.io.File; +import java.util.*; +import java.util.concurrent.TimeUnit; +import java.util.logging.Level; +import java.util.logging.Logger; + +/** + * The manager of metrics, which keeps track of queue and overall cluster metrics. + * Metrics to record are registered here. + * Writes results at the end of simulation experiments. + * Based on the SchedulerMetrics class in SLS. + */ +@Singleton +public final class MetricsManager extends LifeCycle { + private static final Logger LOG = Logger.getLogger(MetricsManager.class.getName()); + + public static final String EOL = System.getProperty("line.separator"); + + private static final String METRIC_TYPE_VARIABLE = "variable"; + private static final String METRIC_TYPE_COUNTER = "counter"; + private static final String METRIC_TYPE_SAMPLER = "sampler"; + + public static final String CLUSTER_ALLOCATED_MEMORY = "cluster.allocated.memory"; + public static final String CLUSTER_ALLOCATED_VCORES = "cluster.allocated.vcores"; + public static final String CLUSTER_AVAILABLE_MEMORY = "cluster.available.memory"; + public static final String CLUSTER_AVAILABLE_VCORES = "cluster.available.vcores"; + + public static final String RUNNING_APPLICATIONS = "running.application"; + public static final String RUNNING_CONTAINERS = "running.container"; + public static final String PENDING_APPLICATIONS = "pending.application"; + + public static final String SCHEDULER_OPERATION_ALLOCATE = "scheduler.operation.allocate"; + public static final String SCHEDULER_OPERATION_HANDLE = "scheduler.operation.handle"; + + public enum QueueMetric { + PENDING_MEMORY("pending.memory"), + PENDING_VCORES("pending.vcores"), + ALLOCATED_MEMORY("allocated.memory"), + ALLOCATED_VCORES("allocated.vcores"), + AVAILABLE_MEMORY("available.memory"), + AVAILABLE_VCORES("available.vcores"); + + private String value; + + QueueMetric(final String value) { + this.value = value; + } + + public String getConfigValue() { + return "queue." + this.value; + } + } + + private static final int SAMPLING_SIZE = 60; + + private final Clock clock; + + private final MetricRegistry metricsRegistry; + private final int metricsTimerWindowSize; + private final int metricsRecordIntervalSeconds; + private final JobHistoryManager jobHistoryManager; + private final MetricsConfiguration metricsConfig; + private final ClusterStats clusterStats; + + private UUID histogramsAlarm = null; + + private UUID metricsAlarm = null; + + private Counter schedulerHandleCounter; + private Map schedulerHandleCounterMap; + + // Timers for scheduler allocate/handle operations + private Timer schedulerAllocateTimer; + private Counter schedulerAllocateCounter; + + private Timer schedulerHandleTimer; + private Map schedulerHandleTimerMap; + private List schedulerHistogramList; + private Map histogramTimerMap; + + private final Set trackedQueues = new HashSet<>(); + private ResourceScheduler scheduler = null; + + private transient List memoryUsedPercent = new LinkedList<>(); + private transient List vcoresUsedPercent = new LinkedList<>(); + + @Inject + private MetricsManager( + @MetricsTimerWindowSize final Integer metricsTimerWindowSize, + @MetricsRecordIntervalSeconds final Integer metricsRecordIntervalSeconds, + final ClusterStats clusterStats, + final JobHistoryManager jobHistoryManager, + final MetricsConfiguration metricsConfig, + final Clock clock) { + this.clock = clock; + this.metricsRegistry = new MetricRegistry(); + this.jobHistoryManager = jobHistoryManager; + this.clusterStats = clusterStats; + + this.metricsTimerWindowSize = metricsTimerWindowSize; + this.metricsRecordIntervalSeconds = metricsRecordIntervalSeconds; + this.metricsConfig = metricsConfig; + } + + public boolean isMetricsOn() { + return metricsConfig.isMetricsOn(); + } + + public MetricsConfiguration getMetricsConfig() { + return metricsConfig; + } + + private void validateSettings() { + if (!metricsConfig.isMetricsOn()) { + return; + } + + if (!(metricsTimerWindowSize > 0)) { + throw new IllegalArgumentException("metricsTimerWindowSize must be greater than 0"); + } + + if (!(metricsRecordIntervalSeconds > 0)) { + throw new IllegalArgumentException("metricsRecordIntervalSeconds must be greater than 0"); + } + + if (scheduler == null) { + throw new NullArgumentException("Scheduler must be non-null"); + } + + if (metricsConfig.getMetricsDirectory() == null) { + throw new NullArgumentException("Metrics output directory must be non-null"); + } + } + + @Override + public void init() { + super.init(); + if (scheduler == null) { + throw new NullArgumentException("Scheduler cannot be null!"); + } + + validateSettings(); + registerClusterResourceMetrics(); + registerContainerAppNumMetrics(); + registerSchedulerMetrics(); + clusterStats.init(); + } + + @Override + public void start() { + super.start(); + + LOG.log(Level.INFO, "Starting metrics manager..."); + + // .csv output + initMetricsCSVOutput(); + + // a thread to update histogram timer + if (schedulerHistogramList != null) { + histogramsAlarm = clock.schedulePeriodicAlarm(5, new HistogramsRunnable()); + } + + clusterStats.start(); + LOG.log(Level.INFO, "Metrics manager started!"); + } + + @Override + public void stop() { + super.stop(); + if (histogramsAlarm != null) { + clock.cancelAlarm(histogramsAlarm); + } + + if (metricsAlarm != null) { + clock.cancelAlarm(metricsAlarm); + } + + if (clusterStats != null) { + clusterStats.setClusterMemUtilization(getMemoryUsedPercent()); + clusterStats.setClusterCoreUtilization(getVcoresUsedPercent()); + clusterStats.stop(); + } + } + + public void writeResults() { + if (!lifeCycleState.isDone()) { + throw new StateException("Cannot write job and cluster results as simulation is not yet done!"); + } + + clusterStats.writeResultsToTsv(); + jobHistoryManager.getJobStats().writeResultsToTsv(); + + System.out.println("=== Cluster stats ==="); + System.out.println(clusterStats.toString()); + System.out.println(); + System.out.println("=== Job stats ==="); + System.out.println(jobHistoryManager.getJobStats().toString()); + System.out.println(); + } + + class HistogramsRunnable implements Runnable { + @Override + public void run() { + for (Histogram histogram : schedulerHistogramList) { + Timer timer = histogramTimerMap.get(histogram); + histogram.update((int) timer.getSnapshot().getMean()); + } + } + } + + private void registerClusterResourceMetrics() { + if (metricsConfig.metricsContains(CLUSTER_ALLOCATED_MEMORY)) { + metricsRegistry.register(METRIC_TYPE_VARIABLE + "." + CLUSTER_ALLOCATED_MEMORY, + (Gauge) () -> { + if (scheduler.getRootQueueMetrics() == null) { + return 0L; + } else { + memoryUsedPercent.add(0D); + + memoryUsedPercent.add( + (double) scheduler.getRootQueueMetrics().getAllocatedMB() / + ( + scheduler.getRootQueueMetrics().getAllocatedMB() + + scheduler.getRootQueueMetrics().getAvailableMB() + ) + ); + + return scheduler.getRootQueueMetrics().getAllocatedMB(); + } + } + ); + } + + if (metricsConfig.metricsContains(CLUSTER_ALLOCATED_VCORES)) { + metricsRegistry.register(METRIC_TYPE_VARIABLE + "." + CLUSTER_ALLOCATED_VCORES, + (Gauge) () -> { + if (scheduler.getRootQueueMetrics() == null) { + vcoresUsedPercent.add(0D); + return 0; + } else { + vcoresUsedPercent.add( + (double) scheduler.getRootQueueMetrics().getAllocatedVirtualCores() / + ( + scheduler.getRootQueueMetrics().getAllocatedVirtualCores() + + scheduler.getRootQueueMetrics().getAvailableVirtualCores() + ) + ); + + return scheduler.getRootQueueMetrics().getAllocatedVirtualCores(); + } + } + ); + } + + if (metricsConfig.metricsContains(CLUSTER_AVAILABLE_MEMORY)) { + metricsRegistry.register(METRIC_TYPE_VARIABLE + "." + CLUSTER_AVAILABLE_MEMORY, + (Gauge) () -> { + if (scheduler.getRootQueueMetrics() == null) { + return 0L; + } else { + return scheduler.getRootQueueMetrics().getAvailableMB(); + } + } + ); + } + + if (metricsConfig.metricsContains(CLUSTER_AVAILABLE_VCORES)) { + metricsRegistry.register(METRIC_TYPE_VARIABLE + "." + CLUSTER_AVAILABLE_VCORES, + (Gauge) () -> { + if (scheduler.getRootQueueMetrics() == null) { + return 0; + } else { + return scheduler.getRootQueueMetrics().getAvailableVirtualCores(); + } + } + ); + } + } + + private void registerContainerAppNumMetrics() { + if (metricsConfig.metricsContains(RUNNING_APPLICATIONS)) { + metricsRegistry.register(METRIC_TYPE_VARIABLE + "." + RUNNING_APPLICATIONS, + (Gauge) () -> { + if (scheduler.getRootQueueMetrics() == null) { + return 0; + } else { + return scheduler.getRootQueueMetrics().getAppsRunning(); + } + } + ); + } + + if (metricsConfig.metricsContains(PENDING_APPLICATIONS)) { + metricsRegistry.register(METRIC_TYPE_VARIABLE + "." + PENDING_APPLICATIONS, + (Gauge) () -> { + if (scheduler.getRootQueueMetrics() == null) { + return 0; + } else { + return scheduler.getRootQueueMetrics().getAppsPending(); + } + } + ); + } + + if (metricsConfig.metricsContains(RUNNING_CONTAINERS)) { + metricsRegistry.register(METRIC_TYPE_VARIABLE + "." + RUNNING_CONTAINERS, + (Gauge) () -> { + if (scheduler.getRootQueueMetrics() == null) { + return 0; + } else { + return scheduler.getRootQueueMetrics().getAllocatedContainers(); + } + } + ); + } + } + + private void registerSchedulerMetrics() { + // counters for scheduler operations + if (metricsConfig.metricsContains(SCHEDULER_OPERATION_ALLOCATE)) { + schedulerAllocateCounter = metricsRegistry.counter( + METRIC_TYPE_COUNTER + "." + SCHEDULER_OPERATION_ALLOCATE + ); + + schedulerAllocateTimer = new Timer(new SlidingWindowReservoir(metricsTimerWindowSize)); + + // histogram for scheduler operations (Samplers) + schedulerHistogramList = new ArrayList<>(); + histogramTimerMap = new HashMap<>(); + + // allocateTimecostHistogram + Histogram schedulerAllocateHistogram = new Histogram(new SlidingWindowReservoir(SAMPLING_SIZE)); + metricsRegistry.register( + METRIC_TYPE_SAMPLER + "." + SCHEDULER_OPERATION_ALLOCATE + ".timecost", + schedulerAllocateHistogram + ); + + schedulerHistogramList.add(schedulerAllocateHistogram); + histogramTimerMap.put(schedulerAllocateHistogram, schedulerAllocateTimer); + } + + if (metricsConfig.metricsContains(SCHEDULER_OPERATION_HANDLE)) { + schedulerHandleCounter = metricsRegistry.counter(METRIC_TYPE_COUNTER + "." + SCHEDULER_OPERATION_HANDLE); + schedulerHandleCounterMap = new HashMap<>(); + for (SchedulerEventType e : SchedulerEventType.values()) { + Counter counter = metricsRegistry.counter( + METRIC_TYPE_COUNTER + "." + SCHEDULER_OPERATION_HANDLE + "." + e); + schedulerHandleCounterMap.put(e, counter); + } + + schedulerHandleTimer = new Timer( + new SlidingWindowReservoir(metricsTimerWindowSize)); + + schedulerHandleTimerMap = new HashMap<>(); + for (SchedulerEventType e : SchedulerEventType.values()) { + Timer timer = new Timer(new SlidingWindowReservoir(metricsTimerWindowSize)); + schedulerHandleTimerMap.put(e, timer); + } + + // handleTimecostHistogram + Histogram schedulerHandleHistogram = new Histogram( + new SlidingWindowReservoir(SAMPLING_SIZE)); + metricsRegistry.register( + METRIC_TYPE_SAMPLER + "." + SCHEDULER_OPERATION_HANDLE + ".timecost", + schedulerHandleHistogram); + schedulerHistogramList.add(schedulerHandleHistogram); + histogramTimerMap.put(schedulerHandleHistogram, schedulerHandleTimer); + + for (SchedulerEventType e : SchedulerEventType.values()) { + Histogram histogram = new Histogram( + new SlidingWindowReservoir(SAMPLING_SIZE)); + metricsRegistry.register( + METRIC_TYPE_SAMPLER + "." + SCHEDULER_OPERATION_HANDLE + "." + e + ".timecost", histogram); + schedulerHistogramList.add(histogram); + histogramTimerMap.put(histogram, schedulerHandleTimerMap.get(e)); + } + } + } + + private void initMetricsCSVOutput() { + final String detailedMetricsDir = metricsConfig.getMetricsFilePath("detailed-metrics"); + if (detailedMetricsDir != null) { + final File dir = new File(detailedMetricsDir); + if(!dir.exists() && !dir.mkdirs()) { + LOG.log(Level.SEVERE,"Cannot create directory " + dir.getAbsoluteFile()); + throw new RuntimeException("Not able to create directory " + dir.getAbsoluteFile() + " for metrics"); + } + + final MetricsCsvReporter reporter = MetricsCsvReporter.newBuilder(metricsRegistry, clock) + .formatFor(Locale.US) + .convertRatesTo(TimeUnit.SECONDS) + .convertDurationsTo(TimeUnit.MILLISECONDS) + .build(new File(detailedMetricsDir)); + + reporter.start(metricsRecordIntervalSeconds, TimeUnit.SECONDS); + } + } + + public void setScheduler(final ResourceScheduler scheduler) { + this.scheduler = scheduler; + } + + public Timer getSchedulerHandleTimer() { + return schedulerHandleTimer; + } + + public Timer getSchedulerHandleTimer(SchedulerEventType schedulerEventType) { + if (schedulerHandleTimerMap == null) { + return null; + } + + return schedulerHandleTimerMap.get(schedulerEventType); + } + + public Timer getSchedulerAllocateTimer() { + return schedulerAllocateTimer; + } + + void increaseSchedulerAllocationCounter() { + if (schedulerAllocateCounter != null) { + schedulerAllocateCounter.inc(); + } + } + + public String getQueueMetricName(final String queue, final QueueMetric metric) { + return "variable.queue." + queue + "." + metric.value; + } + + public void increaseSchedulerHandleCounter(SchedulerEventType schedulerEventType) { + if (schedulerHandleCounter != null) { + schedulerHandleCounter.inc(); + schedulerHandleCounterMap.get(schedulerEventType).inc(); + } + } + + protected void registerQueueMetricsIfNew(final Queue queue) { + if (trackedQueues.contains(queue.getQueueName())) { + return; + } + + trackedQueues.add(queue.getQueueName()); + + for (final QueueMetric queueMetric : QueueMetric.values()) { + final String metricName = getQueueMetricName(queue.getQueueName(), queueMetric); + if (!metricsRegistry.getGauges().containsKey(metricName) && + metricsConfig.metricsContains(queueMetric.getConfigValue())) { + + final Gauge gauge; + switch (queueMetric) { + case ALLOCATED_MEMORY: + gauge = () -> queue.getMetrics().getAllocatedMB(); + break; + case ALLOCATED_VCORES: + gauge = () -> queue.getMetrics().getAllocatedVirtualCores(); + break; + case AVAILABLE_VCORES: + gauge = () -> queue.getMetrics().getAvailableVirtualCores(); + break; + case AVAILABLE_MEMORY: + gauge = () -> queue.getMetrics().getAvailableMB(); + break; + case PENDING_MEMORY: + gauge = () -> queue.getMetrics().getPendingMB(); + break; + case PENDING_VCORES: + gauge = () -> queue.getMetrics().getPendingVirtualCores(); + break; + default: + throw new UnsupportedOperationException("Unknown queue metric: " + queueMetric); + } + + metricsRegistry.register(metricName, gauge); + } + } + } + + public List getMemoryUsedPercent() { + return memoryUsedPercent; + } + + public List getVcoresUsedPercent() { + return vcoresUsedPercent; + } +} diff --git a/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/metrics/SchedulerWrapper.java b/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/metrics/SchedulerWrapper.java new file mode 100644 index 00000000000..39073873d47 --- /dev/null +++ b/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/metrics/SchedulerWrapper.java @@ -0,0 +1,18 @@ +package org.apache.hadoop.yarn.dtss.metrics; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; + +/** + * A wrapper around the scheduler to wire metrics through to the simulation environment. + * Please implement this in order to support new schedulers in the simulation environment. + * For an example, please see {@link CapacitySchedulerWrapper}. + */ +@InterfaceAudience.Public +@InterfaceStability.Unstable +public interface SchedulerWrapper { + + void setMetricsManager(MetricsManager metricsManager); + + MetricsManager getMetricsManager(); +} diff --git a/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/metrics/job/JobHistory.java b/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/metrics/job/JobHistory.java new file mode 100644 index 00000000000..c6ead6c82f6 --- /dev/null +++ b/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/metrics/job/JobHistory.java @@ -0,0 +1,101 @@ +package org.apache.hadoop.yarn.dtss.metrics.job; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.yarn.api.records.ApplicationId; +import org.apache.hadoop.yarn.dtss.job.ExperimentJobEndState; +import org.apache.hadoop.yarn.dtss.job.SimulatedJob; +import org.apache.hadoop.yarn.dtss.job.SimulatedJobMaster; +import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp; + +import java.time.Instant; + +/** + * Records the history of a job, including its state at the + * end of a simulation experiment, its application ID, and + * its submit, start, and end times. + */ +@InterfaceAudience.Private +public final class JobHistory { + private final SimulatedJobMaster simulatedJobMaster; + + private ApplicationId appId = null; + private Instant submitTime = null; + private Instant startTime = null; + private Instant endTime = null; + + private ExperimentJobEndState endState = ExperimentJobEndState.READ; + + public static JobHistory newInstance(final SimulatedJobMaster simulatedJobMaster) { + return new JobHistory(simulatedJobMaster); + } + + private JobHistory(final SimulatedJobMaster simulatedJob) { + this.simulatedJobMaster = simulatedJob; + } + + public SimulatedJob getSimulatedJob() { + return simulatedJobMaster.getSimulatedJob(); + } + + public void jobSubmitted(final Instant submitTime, final RMApp app) { + this.submitTime = submitTime; + this.appId = app.getApplicationId(); + } + + public void setStartTime(final Instant instant) { + this.startTime = instant; + } + + public void setEndTime(final Instant instant, final ExperimentJobEndState state) { + this.endState = state; + this.endTime = instant; + } + + public Long getSubmitTimeSeconds() { + if (submitTime == null) { + return null; + } + + return submitTime.toEpochMilli() / 1000; + } + + public Long getStartTimeSeconds() { + if (startTime == null) { + return null; + } + + return startTime.toEpochMilli() / 1000; + } + + public Long getEndTimeSeconds() { + if (endTime == null) { + return null; + } + + return endTime.toEpochMilli() / 1000; + } + + public ExperimentJobEndState getEndState() { + return endState; + } + + public String getQueue() { + return simulatedJobMaster.getQueueName(); + } + + public ApplicationId getApplicationId() { + return appId; + } + + public boolean isJobCompleted() { + return endState == ExperimentJobEndState.COMPLETED; + } + + public String getTraceJobId() { + return simulatedJobMaster.getTraceJobId(); + } + + public Integer getPriority() { + return simulatedJobMaster.getSimulatedJob().getPriority(); + } +} diff --git a/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/metrics/job/JobHistoryManager.java b/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/metrics/job/JobHistoryManager.java new file mode 100644 index 00000000000..576c3a5e747 --- /dev/null +++ b/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/metrics/job/JobHistoryManager.java @@ -0,0 +1,218 @@ +package org.apache.hadoop.yarn.dtss.metrics.job; + +import com.google.inject.Inject; +import com.google.inject.Singleton; +import org.apache.hadoop.yarn.dtss.job.ExperimentJobEndState; +import org.apache.hadoop.yarn.dtss.job.SimulatedJob; +import org.apache.hadoop.yarn.dtss.job.SimulatedJobMaster; +import org.apache.hadoop.yarn.dtss.job.SimulatedTask; +import org.apache.hadoop.yarn.dtss.lifecycle.LifeCycle; +import org.apache.hadoop.yarn.dtss.metrics.MetricsConfiguration; +import org.apache.hadoop.yarn.dtss.metrics.MetricsManager; +import org.apache.hadoop.yarn.dtss.stats.JobStats; +import org.apache.hadoop.yarn.dtss.time.Clock; +import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp; + +import java.io.BufferedWriter; +import java.io.FileOutputStream; +import java.io.IOException; +import java.io.OutputStreamWriter; +import java.nio.charset.StandardCharsets; +import java.util.HashMap; +import java.util.Map; +import java.util.Set; +import java.util.logging.Logger; + +/** + * A singleton object that collects all job execution history + * for a given trace in simulation. + * Called upon job start, complete, and failed. + */ +@Singleton +public final class JobHistoryManager extends LifeCycle { + private static final Logger LOG = Logger.getLogger(JobHistoryManager.class.getName()); + + private final Clock clock; + private final MetricsConfiguration metricsConfig; + private final JobStats jobStats; + + private BufferedWriter jobRuntimeLogBW; + + private final Map jobHistory = new HashMap<>(); + + @Inject + private JobHistoryManager( + final MetricsConfiguration metricsConfig, + final JobStats jobStats, + final Clock clock) { + this.clock = clock; + this.metricsConfig = metricsConfig; + this.jobStats = jobStats; + } + + @Override + public void init() { + super.init(); + jobStats.init(); + } + + @Override + public void start() { + super.start(); + if (metricsConfig.isMetricsOn()) { + try { + final String jobRuntimeCsvFilePath = metricsConfig.getMetricsFilePath("jobruntime.csv"); + assert jobRuntimeCsvFilePath != null; + + // application running information + jobRuntimeLogBW = + new BufferedWriter(new OutputStreamWriter(new FileOutputStream( + jobRuntimeCsvFilePath), StandardCharsets.UTF_8)); + jobRuntimeLogBW.write( + "TraceJobID," + + "real_submit_time_seconds," + + "real_start_time_seconds," + + "real_end_time_seconds," + + "AppAttemptID," + + "Queue," + + "EndState," + + "simulate_submit_time_seconds," + + "simulate_start_time_seconds," + + "simulate_end_time_seconds" + + MetricsManager.EOL); + + jobRuntimeLogBW.flush(); + } catch (IOException e) { + e.printStackTrace(); + throw new RuntimeException(e); + } + } + + jobStats.start(); + } + + /** + * Called when a job has been read from the trace and its application master created. + * @param job the mocked application master + */ + public void onJobRead(final SimulatedJobMaster job) { + jobHistory.put(job.getTraceJobId(), JobHistory.newInstance(job)); + } + + /** + * Called when a job has been submitted to the RM. + * @param job the submitted simulated job + * @param app the RM representation of the job + */ + public void onJobSubmitted(final SimulatedJob job, final RMApp app) { + jobHistory.get(job.getTraceJobId()).jobSubmitted(clock.getInstant(), app); + } + + /** + * Called when a job has started. + * @param job The started simulated job + */ + public void onJobStarted(final SimulatedJob job) { + jobHistory.get(job.getTraceJobId()).setStartTime(clock.getInstant()); + } + + /** + * Called when a job completes. Metrics are logged if metrics are turned on. + * @param job The completed simulated job. + */ + public void onJobCompleted(final SimulatedJob job) { + final JobHistory history = jobHistory.get(job.getTraceJobId()); + history.setEndTime(clock.getInstant(), ExperimentJobEndState.COMPLETED); + logJobMetrics(job); + } + + /** + * Called when a job fails. Metrics are logged if metrics are turned on. + * @param job The failed simulated job + */ + public void onJobFailed(final SimulatedJob job) { + final JobHistory history = jobHistory.get(job.getTraceJobId()); + history.setEndTime(null, ExperimentJobEndState.FAILED); + logJobMetrics(job); + } + + /** + * Called on jobs for all jobs that are read but not completed/failed by the end of experiment. + * @param job the simulated job + */ + public void onJobNotCompletedByEndOfExperiment(final SimulatedJob job) { + final JobHistory history = jobHistory.get(job.getTraceJobId()); + if (history.getSubmitTimeSeconds() == null) { + history.setEndTime(null, ExperimentJobEndState.NOT_SUBMITTED); + } else if (history.getStartTimeSeconds() == null) { + history.setEndTime(null, ExperimentJobEndState.SUBMITTED); + } else { + history.setEndTime(null, ExperimentJobEndState.STARTED); + } + + logJobMetrics(job); + } + + /** + * Write job metrics for a single finished job if metrics are enabled. + * @param job The simulated completed job. + */ + private void logJobMetrics(final SimulatedJob job) { + if (!metricsConfig.isMetricsOn()) { + return; + } + + try { + final JobHistory history = jobHistory.get(job.getTraceJobId()); + // Maintain job stats + jobStats.appendJobHistory(job, history); + + // write job runtime information + StringBuilder sb = new StringBuilder(); + sb.append(job.getTraceJobId()).append(",") + .append(job.getSubmitTimeSeconds()).append(",") + .append(emptyIfNull(job.getStartTimeSeconds())).append(",") + .append(emptyIfNull(job.getEndTimeSeconds())).append(",") + .append(emptyIfNull(history.getApplicationId())).append(",") + .append(emptyIfNull(history.getQueue())).append(",") + .append(emptyIfNull(history.getEndState())).append(",") + .append(emptyIfNull(history.getSubmitTimeSeconds())).append(",") + .append(emptyIfNull(history.getStartTimeSeconds())).append(",") + .append(emptyIfNull(history.getEndTimeSeconds())); + jobRuntimeLogBW.write(sb.toString() + MetricsManager.EOL); + jobRuntimeLogBW.flush(); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + + private static String emptyIfNull(final Object o) { + if (o == null) { + return ""; + } + + return o.toString(); + } + + public JobStats getJobStats() { + return jobStats; + } + + @Override + public void stop() { + super.stop(); + + if (jobRuntimeLogBW != null) { + try { + jobRuntimeLogBW.close(); + } catch (IOException e) { + e.printStackTrace(); + throw new RuntimeException(e); + } + } + + if (metricsConfig.isMetricsOn()) { + jobStats.stop(); + } + } +} diff --git a/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/nm/MockNM.java b/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/nm/MockNM.java new file mode 100644 index 00000000000..72f8de6c249 --- /dev/null +++ b/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/nm/MockNM.java @@ -0,0 +1,321 @@ +package org.apache.hadoop.yarn.dtss.nm; +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +import org.apache.hadoop.yarn.api.records.*; +import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.apache.hadoop.yarn.server.api.protocolrecords.*; +import org.apache.hadoop.yarn.server.api.records.AppCollectorData; +import org.apache.hadoop.yarn.server.api.records.MasterKey; +import org.apache.hadoop.yarn.server.api.records.NodeHealthStatus; +import org.apache.hadoop.yarn.server.api.records.NodeStatus; +import org.apache.hadoop.yarn.server.resourcemanager.ResourceTrackerService; +import org.apache.hadoop.yarn.server.utils.BuilderUtils; +import org.apache.hadoop.yarn.util.Records; +import org.apache.hadoop.yarn.util.YarnVersionInfo; +import org.apache.hadoop.yarn.util.resource.Resources; +import org.eclipse.jetty.util.log.Log; + +import java.util.*; +import java.util.concurrent.ConcurrentHashMap; +import java.util.logging.Logger; + +public class MockNM { + private static final Logger LOG = Logger.getLogger(MockNM.class.getName()); + + private int responseId; + private NodeId nodeId; + private Resource capability; + private ResourceTrackerService resourceTracker; + private int httpPort = 2; + private MasterKey currentContainerTokenMasterKey; + private MasterKey currentNMTokenMasterKey; + private String version; + private Map containerStats = new HashMap<>(); + private Map registeringCollectors + = new ConcurrentHashMap<>(); + private Set nodeLabels; + + public MockNM(String nodeIdStr, int memory, ResourceTrackerService resourceTracker) { + // scale vcores based on the requested memory + this(nodeIdStr, memory, + Math.max(1, (memory * YarnConfiguration.DEFAULT_NM_VCORES) / + YarnConfiguration.DEFAULT_NM_PMEM_MB), + resourceTracker); + } + + public MockNM(String nodeIdStr, int memory, int vcores, + ResourceTrackerService resourceTracker) { + this(nodeIdStr, memory, vcores, resourceTracker, + YarnVersionInfo.getVersion()); + } + + public MockNM(String nodeIdStr, int memory, int vcores, + ResourceTrackerService resourceTracker, String version) { + this(nodeIdStr, Resource.newInstance(memory, vcores), resourceTracker, + version); + } + + public MockNM(String nodeIdStr, Resource capability, + ResourceTrackerService resourceTracker) { + this(nodeIdStr, capability, resourceTracker, + YarnVersionInfo.getVersion()); + } + + public MockNM(String nodeIdStr, Resource capability, + ResourceTrackerService resourceTracker, String version) { + this.capability = capability; + this.resourceTracker = resourceTracker; + this.version = version; + String[] splits = nodeIdStr.split(":"); + nodeId = BuilderUtils.newNodeId(splits[0], Integer.parseInt(splits[1])); + } + + public MockNM(String nodeIdStr, Resource capability, + ResourceTrackerService resourceTracker, String version, Set + nodeLabels) { + this(nodeIdStr, capability, resourceTracker, version); + this.nodeLabels = nodeLabels; + } + + public NodeId getNodeId() { + return nodeId; + } + + public int getHttpPort() { + return httpPort; + } + + public void setHttpPort(int port) { + httpPort = port; + } + + public void setResourceTrackerService(ResourceTrackerService resourceTracker) { + this.resourceTracker = resourceTracker; + } + + public void containerStatus(ContainerStatus containerStatus) throws Exception { + Map> conts = + new HashMap>(); + conts.put(containerStatus.getContainerId().getApplicationAttemptId().getApplicationId(), + Arrays.asList(new ContainerStatus[] { containerStatus })); + nodeHeartbeat(conts, true); + } + + public void containerIncreaseStatus(Container container) throws Exception { + ContainerStatus containerStatus = BuilderUtils.newContainerStatus( + container.getId(), ContainerState.RUNNING, "Success", 0, + container.getResource()); + List increasedConts = Collections.singletonList(container); + nodeHeartbeat(Collections.singletonList(containerStatus), increasedConts, + true, responseId); + } + + public void addRegisteringCollector(ApplicationId appId, + AppCollectorData data) { + this.registeringCollectors.put(appId, data); + } + + public Map getRegisteringCollectors() { + return this.registeringCollectors; + } + + public void unRegisterNode() throws Exception { + UnRegisterNodeManagerRequest request = Records + .newRecord(UnRegisterNodeManagerRequest.class); + request.setNodeId(nodeId); + resourceTracker.unRegisterNodeManager(request); + } + + public RegisterNodeManagerResponse registerNode() throws Exception { + return registerNode(null, null); + } + + public RegisterNodeManagerResponse registerNode( + List runningApplications) throws Exception { + return registerNode(null, runningApplications); + } + + public RegisterNodeManagerResponse registerNode( + List containerReports, + List runningApplications) throws Exception { + RegisterNodeManagerRequest req = Records.newRecord( + RegisterNodeManagerRequest.class); + + req.setNodeId(nodeId); + req.setHttpPort(httpPort); + req.setResource(capability); + req.setContainerStatuses(containerReports); + req.setNMVersion(version); + req.setRunningApplications(runningApplications); + if ( nodeLabels != null && nodeLabels.size() > 0) { + req.setNodeLabels(nodeLabels); + } + + RegisterNodeManagerResponse registrationResponse = + resourceTracker.registerNodeManager(req); + this.currentContainerTokenMasterKey = + registrationResponse.getContainerTokenMasterKey(); + this.currentNMTokenMasterKey = registrationResponse.getNMTokenMasterKey(); + Resource newResource = registrationResponse.getResource(); + if (newResource != null) { + capability = Resources.clone(newResource); + } + containerStats.clear(); + if (containerReports != null) { + for (NMContainerStatus report : containerReports) { + if (report.getContainerState() != ContainerState.COMPLETE) { + containerStats.put(report.getContainerId(), + ContainerStatus.newInstance(report.getContainerId(), + report.getContainerState(), report.getDiagnostics(), + report.getContainerExitStatus())); + } + } + } + responseId = 0; + return registrationResponse; + } + + public NodeHeartbeatResponse nodeHeartbeat(boolean isHealthy) throws Exception { + return nodeHeartbeat(Collections.emptyList(), + Collections.emptyList(), isHealthy, responseId); + } + + public NodeHeartbeatResponse nodeHeartbeat(ApplicationAttemptId attemptId, + long containerId, ContainerState containerState) throws Exception { + ContainerStatus containerStatus = BuilderUtils.newContainerStatus( + BuilderUtils.newContainerId(attemptId, containerId), containerState, + "Success", 0, capability); + ArrayList containerStatusList = + new ArrayList(1); + containerStatusList.add(containerStatus); + Log.getLog().info("ContainerStatus: " + containerStatus); + return nodeHeartbeat(containerStatusList, + Collections.emptyList(), true, responseId); + } + + public NodeHeartbeatResponse nodeHeartbeat(Map> conts, boolean isHealthy) throws Exception { + return nodeHeartbeat(conts, isHealthy, responseId); + } + + public NodeHeartbeatResponse nodeHeartbeat(Map> conts, boolean isHealthy, int resId) throws Exception { + ArrayList updatedStats = new ArrayList(); + for (List stats : conts.values()) { + updatedStats.addAll(stats); + } + return nodeHeartbeat(updatedStats, Collections.emptyList(), + isHealthy, resId); + } + + public NodeHeartbeatResponse nodeHeartbeat( + List updatedStats, boolean isHealthy) throws Exception { + return nodeHeartbeat(updatedStats, Collections.emptyList(), + isHealthy, responseId); + } + + public NodeHeartbeatResponse nodeHeartbeat(List updatedStats, + List increasedConts, boolean isHealthy, int resId) + throws Exception { + NodeHeartbeatRequest req = Records.newRecord(NodeHeartbeatRequest.class); + NodeStatus status = Records.newRecord(NodeStatus.class); + status.setResponseId(resId); + status.setNodeId(nodeId); + ArrayList completedContainers = new ArrayList(); + for (ContainerStatus stat : updatedStats) { + if (stat.getState() == ContainerState.COMPLETE) { + completedContainers.add(stat.getContainerId()); + } + containerStats.put(stat.getContainerId(), stat); + } + status.setContainersStatuses( + new ArrayList<>(containerStats.values())); + for (ContainerId cid : completedContainers) { + containerStats.remove(cid); + } + status.setIncreasedContainers(increasedConts); + NodeHealthStatus healthStatus = Records.newRecord(NodeHealthStatus.class); + healthStatus.setHealthReport(""); + healthStatus.setIsNodeHealthy(isHealthy); + healthStatus.setLastHealthReportTime(1); + status.setNodeHealthStatus(healthStatus); + req.setNodeStatus(status); + req.setLastKnownContainerTokenMasterKey(this.currentContainerTokenMasterKey); + req.setLastKnownNMTokenMasterKey(this.currentNMTokenMasterKey); + + req.setRegisteringCollectors(this.registeringCollectors); + + NodeHeartbeatResponse heartbeatResponse = + resourceTracker.nodeHeartbeat(req); + responseId = heartbeatResponse.getResponseId(); + + MasterKey masterKeyFromRM = heartbeatResponse.getContainerTokenMasterKey(); + if (masterKeyFromRM != null + && masterKeyFromRM.getKeyId() != this.currentContainerTokenMasterKey + .getKeyId()) { + this.currentContainerTokenMasterKey = masterKeyFromRM; + } + + masterKeyFromRM = heartbeatResponse.getNMTokenMasterKey(); + if (masterKeyFromRM != null + && masterKeyFromRM.getKeyId() != this.currentNMTokenMasterKey + .getKeyId()) { + this.currentNMTokenMasterKey = masterKeyFromRM; + } + + Resource newResource = heartbeatResponse.getResource(); + if (newResource != null) { + capability = Resources.clone(newResource); + } + + return heartbeatResponse; + } + + public static NodeStatus createMockNodeStatus() { + NodeStatus mockNodeStatus = mock(NodeStatus.class); + NodeHealthStatus mockNodeHealthStatus = mock(NodeHealthStatus.class); + when(mockNodeStatus.getNodeHealthStatus()).thenReturn(mockNodeHealthStatus); + when(mockNodeHealthStatus.getIsNodeHealthy()).thenReturn(true); + return mockNodeStatus; + } + + public long getMemory() { + return capability.getMemorySize(); + } + + public int getvCores() { + return capability.getVirtualCores(); + } + + public Resource getCapability() { + return capability; + } + + public String getVersion() { + return version; + } + + public void setResponseId(int id) { + this.responseId = id; + } +} + diff --git a/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/random/RandomGenerator.java b/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/random/RandomGenerator.java new file mode 100644 index 00000000000..61c371d718a --- /dev/null +++ b/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/random/RandomGenerator.java @@ -0,0 +1,46 @@ +package org.apache.hadoop.yarn.dtss.random; + +import org.apache.commons.lang.RandomStringUtils; +import org.apache.hadoop.classification.InterfaceAudience; + +import java.util.Random; +import java.util.UUID; +import java.util.logging.Level; +import java.util.logging.Logger; + +/** + * A wrapper around the {@link Random} class. + */ +@InterfaceAudience.Private +public final class RandomGenerator { + private static final Logger LOG = Logger.getLogger(RandomGenerator.class.getName()); + + private final Random random; + + RandomGenerator(final long seed) { + random = new Random(seed); + LOG.log(Level.INFO, String.format("Initializing RandomGenerator object with seed %d.", seed)); + } + + public int randomInt(final int exclusiveMax) { + return random.nextInt(exclusiveMax); + } + + public int randomInt(final int inclusiveMin, final int exclusiveMax) { + final int diff = exclusiveMax - inclusiveMin; + return inclusiveMin + random.nextInt(diff); + } + + public double randomDouble(final double inclusiveMin, final double inclusiveMax) { + return inclusiveMin + (inclusiveMax - inclusiveMin) * random.nextDouble(); + } + + public UUID randomUUID() { + final byte[] randomBytes = RandomStringUtils + .random(25, 0, 0, false, false, null, random) + .getBytes(); + + return UUID.nameUUIDFromBytes(randomBytes); + } +} + diff --git a/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/random/RandomSeeder.java b/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/random/RandomSeeder.java new file mode 100644 index 00000000000..e940a29f54b --- /dev/null +++ b/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/random/RandomSeeder.java @@ -0,0 +1,37 @@ +package org.apache.hadoop.yarn.dtss.random; + +import com.google.inject.Inject; +import com.google.inject.Singleton; +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.yarn.dtss.config.parameters.RandomSeed; + +import javax.annotation.Nullable; +import java.util.Random; +import java.util.logging.Level; +import java.util.logging.Logger; + +/** + * Generates new {@link Random} objects with random seeds. + */ +@InterfaceAudience.Private +@Singleton +public final class RandomSeeder { + private static final Logger LOG = Logger.getLogger(RandomSeeder.class.getName()); + + private final Random random; + + @Inject + private RandomSeeder(@Nullable @RandomSeed final Long seed) { + if (seed != null) { + LOG.log(Level.INFO, String.format("Initializing RandomSeeder object with seed %d.", seed)); + this.random = new Random(seed); + } else { + LOG.log(Level.INFO, "Initializing RandomSeeder object without a seed."); + this.random = new Random(); + } + } + + public RandomGenerator newRandomGenerator() { + return new RandomGenerator(random.nextLong()); + } +} diff --git a/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/rm/AsyncDispatcher.java b/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/rm/AsyncDispatcher.java new file mode 100644 index 00000000000..abedf0e8798 --- /dev/null +++ b/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/rm/AsyncDispatcher.java @@ -0,0 +1,269 @@ +// +// Source code recreated from a .class file by IntelliJ IDEA +// (powered by Fernflower decompiler) +// + +package org.apache.hadoop.yarn.dtss.rm; + +import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting; +import java.lang.Thread.State; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.LinkedBlockingQueue; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.service.AbstractService; +import org.apache.hadoop.util.ShutdownHookManager; +import org.apache.hadoop.yarn.event.Dispatcher; +import org.apache.hadoop.yarn.event.Event; +import org.apache.hadoop.yarn.event.EventHandler; +import org.apache.hadoop.yarn.exceptions.YarnRuntimeException; + +/** + * NOTE: Based on classes that support MockRM in the tests package + * for resourcemanager. + */ +public class AsyncDispatcher extends AbstractService implements Dispatcher { + private static final Log LOG = LogFactory.getLog(AsyncDispatcher.class); + private final BlockingQueue eventQueue; + private volatile int lastEventQueueSizeLogged; + private volatile boolean stopped; + private volatile boolean drainEventsOnStop; + private volatile boolean drained; + private final Object waitForDrained; + private volatile boolean blockNewEvents; + private final EventHandler handlerInstance; + private Thread eventHandlingThread; + protected final Map, EventHandler> eventDispatchers; + private boolean exitOnDispatchException; + private String dispatcherThreadName; + + public AsyncDispatcher() { + this(new LinkedBlockingQueue()); + } + + public AsyncDispatcher(BlockingQueue eventQueue) { + super("Dispatcher"); + this.lastEventQueueSizeLogged = 0; + this.stopped = false; + this.drainEventsOnStop = false; + this.drained = true; + this.waitForDrained = new Object(); + this.blockNewEvents = false; + this.handlerInstance = new AsyncDispatcher.GenericEventHandler(); + this.exitOnDispatchException = true; + this.dispatcherThreadName = "AsyncDispatcher event handler"; + this.eventQueue = eventQueue; + this.eventDispatchers = new HashMap(); + } + + public AsyncDispatcher(String dispatcherName) { + this(); + this.dispatcherThreadName = dispatcherName; + } + + Runnable createThread() { + return new Runnable() { + public void run() { + while(!AsyncDispatcher.this.stopped && !Thread.currentThread().isInterrupted()) { + AsyncDispatcher.this.drained = AsyncDispatcher.this.eventQueue.isEmpty(); + if (AsyncDispatcher.this.blockNewEvents) { + synchronized(AsyncDispatcher.this.waitForDrained) { + if (AsyncDispatcher.this.drained) { + AsyncDispatcher.this.waitForDrained.notify(); + } + } + } + + Event event; + try { + event = (Event)AsyncDispatcher.this.eventQueue.take(); + } catch (InterruptedException var4) { + if (!AsyncDispatcher.this.stopped) { + AsyncDispatcher.LOG.warn("AsyncDispatcher thread interrupted", var4); + } + + return; + } + + if (event != null) { + AsyncDispatcher.this.dispatch(event); + } + } + } + }; + } + + @VisibleForTesting + public void disableExitOnDispatchException() { + this.exitOnDispatchException = false; + } + + protected void serviceStart() throws Exception { + super.serviceStart(); + this.eventHandlingThread = new Thread(this.createThread()); + this.eventHandlingThread.setName(this.dispatcherThreadName); + this.eventHandlingThread.start(); + } + + public void setDrainEventsOnStop() { + this.drainEventsOnStop = true; + } + + protected void serviceStop() throws Exception { + if (this.drainEventsOnStop) { + this.blockNewEvents = true; + LOG.info("AsyncDispatcher is draining to stop, ignoring any new events."); + long endTime = System.currentTimeMillis() + this.getConfig().getLong("yarn.dispatcher.drain-events.timeout", 300000L); + Object var3 = this.waitForDrained; + synchronized(this.waitForDrained) { + while(!this.isDrained() && this.eventHandlingThread != null && this.eventHandlingThread.isAlive() && System.currentTimeMillis() < endTime) { + this.waitForDrained.wait(100L); + LOG.info("Waiting for AsyncDispatcher to drain. Thread state is :" + this.eventHandlingThread.getState()); + } + } + } + + this.stopped = true; + if (this.eventHandlingThread != null) { + this.eventHandlingThread.interrupt(); + + try { + this.eventHandlingThread.join(); + } catch (InterruptedException var5) { + LOG.warn("Interrupted Exception while stopping", var5); + } + } + + super.serviceStop(); + } + + protected void dispatch(Event event) { + if (LOG.isDebugEnabled()) { + LOG.debug("Dispatching the event " + event.getClass().getName() + "." + event.toString()); + } + + Class type = event.getType().getDeclaringClass(); + + try { + EventHandler handler = (EventHandler)this.eventDispatchers.get(type); + if (handler == null) { + throw new Exception("No handler for registered for " + type); + } + + handler.handle(event); + } catch (Throwable var5) { + LOG.fatal("Error in dispatcher thread", var5); + if (this.exitOnDispatchException && !ShutdownHookManager.get().isShutdownInProgress() && !this.stopped) { + this.stopped = true; + Thread shutDownThread = new Thread(this.createShutDownThread()); + shutDownThread.setName("AsyncDispatcher ShutDown handler"); + shutDownThread.start(); + } + } + + } + + public void register(Class eventType, EventHandler handler) { + EventHandler registeredHandler = (EventHandler)this.eventDispatchers.get(eventType); + LOG.info("Registering " + eventType + " for " + handler.getClass()); + if (registeredHandler == null) { + this.eventDispatchers.put(eventType, handler); + } else { + AsyncDispatcher.MultiListenerHandler multiHandler; + if (!(registeredHandler instanceof AsyncDispatcher.MultiListenerHandler)) { + multiHandler = new AsyncDispatcher.MultiListenerHandler(); + multiHandler.addHandler(registeredHandler); + multiHandler.addHandler(handler); + this.eventDispatchers.put(eventType, multiHandler); + } else { + multiHandler = (AsyncDispatcher.MultiListenerHandler)registeredHandler; + multiHandler.addHandler(handler); + } + } + + } + + public EventHandler getEventHandler() { + return this.handlerInstance; + } + + Runnable createShutDownThread() { + return new Runnable() { + public void run() { + AsyncDispatcher.LOG.info("Exiting, bbye.."); + System.exit(-1); + } + }; + } + + @VisibleForTesting + protected boolean isEventThreadWaiting() { + return this.eventHandlingThread.getState() == State.WAITING; + } + + protected boolean isDrained() { + return this.drained; + } + + protected boolean isStopped() { + return this.stopped; + } + + static class MultiListenerHandler implements EventHandler { + List> listofHandlers = new ArrayList(); + + public MultiListenerHandler() { + } + + public void handle(Event event) { + Iterator var2 = this.listofHandlers.iterator(); + + while(var2.hasNext()) { + EventHandler handler = (EventHandler)var2.next(); + handler.handle(event); + } + + } + + void addHandler(EventHandler handler) { + this.listofHandlers.add(handler); + } + } + + class GenericEventHandler implements EventHandler { + GenericEventHandler() { + } + + public void handle(Event event) { + if (!AsyncDispatcher.this.blockNewEvents) { + AsyncDispatcher.this.drained = false; + int qSize = AsyncDispatcher.this.eventQueue.size(); + if (qSize != 0 && qSize % 1000 == 0 && AsyncDispatcher.this.lastEventQueueSizeLogged != qSize) { + AsyncDispatcher.this.lastEventQueueSizeLogged = qSize; + AsyncDispatcher.LOG.info("Size of event-queue is " + qSize); + } + + int remCapacity = AsyncDispatcher.this.eventQueue.remainingCapacity(); + if (remCapacity < 1000) { + AsyncDispatcher.LOG.warn("Very low remaining capacity in the event-queue: " + remCapacity); + } + + try { + AsyncDispatcher.this.eventQueue.put(event); + } catch (InterruptedException var5) { + if (!AsyncDispatcher.this.stopped) { + AsyncDispatcher.LOG.warn("AsyncDispatcher thread interrupted", var5); + } + + AsyncDispatcher.this.drained = AsyncDispatcher.this.eventQueue.isEmpty(); + throw new YarnRuntimeException(var5); + } + } + } + } +} diff --git a/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/rm/DrainDispatcher.java b/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/rm/DrainDispatcher.java new file mode 100644 index 00000000000..7db84b5e5d8 --- /dev/null +++ b/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/rm/DrainDispatcher.java @@ -0,0 +1,114 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS 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.yarn.dtss.rm; + +import org.apache.hadoop.yarn.event.Event; +import org.apache.hadoop.yarn.event.EventHandler; + +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.logging.Logger; + +/** + * NOTE: Based on classes that support MockRM in the tests package + * for resourcemanager. + */ +@SuppressWarnings("rawtypes") +public class DrainDispatcher extends AsyncDispatcher { + private static final Logger LOG = Logger.getLogger(DrainDispatcher.class.getName()); + + private volatile boolean drained = false; + private final BlockingQueue queue; + private final Object mutex; + + public DrainDispatcher() { + this(new LinkedBlockingQueue()); + } + + public DrainDispatcher(BlockingQueue eventQueue) { + super(eventQueue); + this.queue = eventQueue; + this.mutex = this; + // Disable system exit since this class is only for unit tests. + disableExitOnDispatchException(); + } + + /** + * Wait till event thread enters WAITING state (i.e. waiting for new events). + */ + public void waitForEventThreadToWait() { + while (!isEventThreadWaiting()) { + Thread.yield(); + } + } + + /** + * Busy loop waiting for all queued events to drain. + */ + public void await() { + while (!isDrained()) { + Thread.yield(); + } + } + + @Override + Runnable createThread() { + return new Runnable() { + @Override + public void run() { + while (!isStopped() && !Thread.currentThread().isInterrupted()) { + synchronized (mutex) { + // !drained if dispatch queued new events on this dispatcher + drained = queue.isEmpty(); + } + Event event; + try { + event = queue.take(); + } catch (InterruptedException ie) { + return; + } + if (event != null) { + dispatch(event); + } + } + } + }; + } + + @SuppressWarnings("unchecked") + @Override + public EventHandler getEventHandler() { + final EventHandler actual = super.getEventHandler(); + return new EventHandler() { + @Override + public void handle(Event event) { + synchronized (mutex) { + actual.handle(event); + drained = false; + } + } + }; + } + + @Override + protected boolean isDrained() { + synchronized (mutex) { + return drained; + } + } +} diff --git a/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/rm/MockMemoryRMStateStore.java b/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/rm/MockMemoryRMStateStore.java new file mode 100644 index 00000000000..631279d471a --- /dev/null +++ b/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/rm/MockMemoryRMStateStore.java @@ -0,0 +1,133 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.yarn.dtss.rm; + +import java.util.HashMap; +import java.util.Map; +import java.util.Map.Entry; + +import org.apache.hadoop.yarn.api.records.ApplicationId; +import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext; +import org.apache.hadoop.yarn.event.EventHandler; +import org.apache.hadoop.yarn.server.resourcemanager.recovery.MemoryRMStateStore; +import org.apache.hadoop.yarn.server.resourcemanager.recovery.records.ApplicationStateData; + +import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting; + +/** + * NOTE: Based on classes that support MockRM in the tests package + * for resourcemanager. + * Test helper for MemoryRMStateStore will make sure the event. + */ +public class MockMemoryRMStateStore extends MemoryRMStateStore { + + private Map appSubCtxtCopy = + new HashMap(); + + @SuppressWarnings("rawtypes") + @Override + protected EventHandler getRMStateStoreEventHandler() { + return rmStateStoreEventHandler; + } + + @Override + public synchronized RMState loadState() throws Exception { + + RMState cloneState = super.loadState(); + + for(Entry state : + cloneState.getApplicationState().entrySet()) { + ApplicationStateData oldStateData = state.getValue(); + oldStateData.setApplicationSubmissionContext( + this.appSubCtxtCopy.get(state.getKey())); + cloneState.getApplicationState().put(state.getKey(), oldStateData); + } + return cloneState; + } + + @Override + public synchronized void storeApplicationStateInternal( + ApplicationId appId, ApplicationStateData appState) + throws Exception { + // Clone Application Submission Context + this.cloneAppSubmissionContext(appState); + super.storeApplicationStateInternal(appId, appState); + } + + @Override + public synchronized void updateApplicationStateInternal( + ApplicationId appId, ApplicationStateData appState) + throws Exception { + // Clone Application Submission Context + this.cloneAppSubmissionContext(appState); + super.updateApplicationStateInternal(appId, appState); + } + + /** + * Clone Application Submission Context and Store in Map for + * later use. + * + * @param appState + */ + private void cloneAppSubmissionContext(ApplicationStateData appState) { + ApplicationSubmissionContext oldAppSubCtxt = + appState.getApplicationSubmissionContext(); + ApplicationSubmissionContext context = + ApplicationSubmissionContext.newInstance( + oldAppSubCtxt.getApplicationId(), + oldAppSubCtxt.getApplicationName(), + oldAppSubCtxt.getQueue(), + oldAppSubCtxt.getPriority(), + oldAppSubCtxt.getAMContainerSpec(), + oldAppSubCtxt.getUnmanagedAM(), + oldAppSubCtxt.getCancelTokensWhenComplete(), + oldAppSubCtxt.getMaxAppAttempts(), + oldAppSubCtxt.getResource() + ); + context.setAttemptFailuresValidityInterval( + oldAppSubCtxt.getAttemptFailuresValidityInterval()); + context.setKeepContainersAcrossApplicationAttempts( + oldAppSubCtxt.getKeepContainersAcrossApplicationAttempts()); + context.setAMContainerResourceRequests( + oldAppSubCtxt.getAMContainerResourceRequests()); + context.setLogAggregationContext(oldAppSubCtxt.getLogAggregationContext()); + context.setApplicationType(oldAppSubCtxt.getApplicationType()); + context.setNodeLabelExpression(oldAppSubCtxt.getNodeLabelExpression()); + this.appSubCtxtCopy.put(oldAppSubCtxt.getApplicationId(), context); + } + + /** + * Traverse each app state and replace cloned app sub context + * into the state. + * + * @param actualState + * @return actualState + */ + @VisibleForTesting + public RMState reloadStateWithClonedAppSubCtxt(RMState actualState) { + for(Entry state : + actualState.getApplicationState().entrySet()) { + ApplicationStateData oldStateData = state.getValue(); + oldStateData.setApplicationSubmissionContext( + this.appSubCtxtCopy.get(state.getKey())); + actualState.getApplicationState().put(state.getKey(), + oldStateData); + } + return actualState; + } +} \ No newline at end of file diff --git a/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/rm/MockRM.java b/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/rm/MockRM.java new file mode 100644 index 00000000000..7fee7dcc3a7 --- /dev/null +++ b/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/rm/MockRM.java @@ -0,0 +1,1454 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS 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.yarn.dtss.rm; + +import com.google.inject.Inject; +import com.google.inject.Singleton; +import org.apache.commons.lang.NullArgumentException; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.io.DataOutputBuffer; +import org.apache.hadoop.security.Credentials; +import org.apache.hadoop.security.UserGroupInformation; +import org.apache.hadoop.security.token.Token; +import org.apache.hadoop.yarn.api.ApplicationClientProtocol; +import org.apache.hadoop.yarn.api.protocolrecords.*; +import org.apache.hadoop.yarn.api.records.*; +import org.apache.hadoop.yarn.api.resource.PlacementConstraint; +import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.apache.hadoop.yarn.dtss.am.MockAM; +import org.apache.hadoop.yarn.dtss.metrics.MetricsManager; +import org.apache.hadoop.yarn.dtss.metrics.SchedulerWrapper; +import org.apache.hadoop.yarn.dtss.nm.MockNM; +import org.apache.hadoop.yarn.dtss.time.Clock; +import org.apache.hadoop.yarn.event.AsyncDispatcher; +import org.apache.hadoop.yarn.event.Dispatcher; +import org.apache.hadoop.yarn.event.EventHandler; +import org.apache.hadoop.yarn.exceptions.YarnException; +import org.apache.hadoop.yarn.security.AMRMTokenIdentifier; +import org.apache.hadoop.yarn.server.api.protocolrecords.NMContainerStatus; +import org.apache.hadoop.yarn.server.api.records.NodeStatus; +import org.apache.hadoop.yarn.server.resourcemanager.*; +import org.apache.hadoop.yarn.server.resourcemanager.amlauncher.AMLauncherEvent; +import org.apache.hadoop.yarn.server.resourcemanager.amlauncher.ApplicationMasterLauncher; +import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager; +import org.apache.hadoop.yarn.server.resourcemanager.recovery.MemoryRMStateStore; +import org.apache.hadoop.yarn.server.resourcemanager.recovery.NullRMStateStore; +import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore; +import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp; +import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppState; +import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.*; +import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer; +import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerState; +import org.apache.hadoop.yarn.server.resourcemanager.rmnode.*; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.*; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEvent; +import org.apache.hadoop.yarn.server.resourcemanager.security.ClientToAMTokenSecretManagerInRM; +import org.apache.hadoop.yarn.server.resourcemanager.security.NMTokenSecretManagerInRM; +import org.apache.hadoop.yarn.server.resourcemanager.security.RMContainerTokenSecretManager; +import org.apache.hadoop.yarn.util.Records; +import org.apache.hadoop.yarn.util.YarnVersionInfo; +import org.apache.log4j.Logger; + +import javax.annotation.Nullable; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.security.PrivilegedExceptionAction; +import java.util.*; +import java.util.logging.Level; + +/** + * NOTE: Based on classes that support MockRM in the tests package + * for resourcemanager. + */ +@SuppressWarnings("unchecked") +@Singleton +public class MockRM extends ResourceManager { + private static final java.util.logging.Logger JAVA_LOGGER = + java.util.logging.Logger.getLogger(MockRM.class.getName()); + static final Logger LOG = Logger.getLogger(MockRM.class); + static final String ENABLE_WEBAPP = "mockrm.webapp.enabled"; + private static final int SECOND = 1000; + private static final int TIMEOUT_MS_FOR_ATTEMPT = 40 * SECOND; + private static final int TIMEOUT_MS_FOR_APP_REMOVED = 40 * SECOND; + private static final int TIMEOUT_MS_FOR_CONTAINER_AND_NODE = 20 * SECOND; + private static final int WAIT_MS_PER_LOOP = 10; + + private final boolean useNullRMNodeLabelsManager; + + private boolean disableDrainEventsImplicitly; + private boolean serviceInitialized = false; + + private boolean useRealElector = false; + private Boolean shouldSubmitToNamedQueues = null; + + private Set configuredQueues; + private MetricsManager metricsManager; + + @Inject + public MockRM( + final Configuration conf, + final Clock clock, + final MetricsManager metricsManager) { + super(); + this.metricsManager = metricsManager; + this.useNullRMNodeLabelsManager = true; + this.useRealElector = false; + this.clock = clock; + init(conf instanceof YarnConfiguration ? conf : new YarnConfiguration(conf)); + Class storeClass = getRMContext().getStateStore().getClass(); + if (storeClass.equals(MemoryRMStateStore.class)) { + MockMemoryRMStateStore mockStateStore = new MockMemoryRMStateStore(); + mockStateStore.init(conf); + setRMStateStore(mockStateStore); + } else if (storeClass.equals(NullRMStateStore.class)) { + MockRMNullStateStore mockStateStore = new MockRMNullStateStore(); + mockStateStore.init(conf); + setRMStateStore(mockStateStore); + } + disableDrainEventsImplicitly = false; + } + + public class MockRMNullStateStore extends NullRMStateStore { + @SuppressWarnings("rawtypes") + @Override + protected EventHandler getRMStateStoreEventHandler() { + return rmStateStoreEventHandler; + } + } + + @Override + protected ResourceScheduler createScheduler() { + final ResourceScheduler sched = super.createScheduler(); + ((SchedulerWrapper)sched).setMetricsManager(metricsManager); + ((AbstractYarnScheduler)sched).setClock(clock); + return sched; + } + + @Nullable + public Set getConfiguredQueues() { + if (!serviceInitialized) { + return null; + } + + if (configuredQueues == null) { + if (scheduler instanceof CapacityScheduler) { + final CapacitySchedulerConfiguration conf = ((CapacityScheduler) scheduler).getConfiguration(); + configuredQueues = new HashSet<>(Arrays.asList(conf.getQueues("root"))); + JAVA_LOGGER.log(Level.INFO, "Got configured queues " + configuredQueues); + } else { + JAVA_LOGGER.log(Level.INFO, "Configured queues not supported!"); + configuredQueues = new HashSet<>(); + } + } + + return configuredQueues; + } + + @Nullable + public Boolean isOnlyDefaultQueue() { + final Set configuredQueues = getConfiguredQueues(); + if (configuredQueues == null) { + return null; + } + + return getConfiguredQueues().isEmpty() || + (configuredQueues.size() == 1 && configuredQueues.contains("default")); + } + + @Nullable + public Boolean getShouldSubmitToNamedQueues() { + if (!serviceInitialized || getConfiguredQueues() == null) { + return null; + } + + if (shouldSubmitToNamedQueues == null) { + final Boolean isOnlyDefaultQueue = isOnlyDefaultQueue(); + shouldSubmitToNamedQueues = isOnlyDefaultQueue == null ? null : !isOnlyDefaultQueue; + } + + return shouldSubmitToNamedQueues; + } + + // Get cluster metrics at the end of the run + public MetricsManager getMetricsManager() { + return ((SchedulerWrapper)scheduler).getMetricsManager(); + } + + @Override + protected RMNodeLabelsManager createNodeLabelManager() + throws InstantiationException, IllegalAccessException { + if (useNullRMNodeLabelsManager) { + RMNodeLabelsManager mgr = new NullRMNodeLabelsManager(); + mgr.init(getConfig()); + return mgr; + } else { + return super.createNodeLabelManager(); + } + } + + @Override + protected Dispatcher createDispatcher() { + return new DrainDispatcher(); + } + + @Override + protected EmbeddedElector createEmbeddedElector() throws IOException { + if (useRealElector) { + return super.createEmbeddedElector(); + } else { + return null; + } + } + + private void handleSchedulerEvent(final SchedulerEvent schedulerEvent) { + scheduler.handle(schedulerEvent); + } + + @Override + protected EventHandler createSchedulerEventDispatcher() { + return this::handleSchedulerEvent; + } + + public void drainEvents() { + Dispatcher rmDispatcher = rmContext.getDispatcher(); + if (rmDispatcher instanceof DrainDispatcher) { + ((DrainDispatcher) rmDispatcher).await(); + } else { + throw new UnsupportedOperationException("Not a Drain Dispatcher!"); + } + } + + private boolean waitForState(ApplicationId appId, EnumSet finalStates) + throws InterruptedException { + drainEventsImplicitly(); + RMApp app = getRMContext().getRMApps().get(appId); + final int timeoutMsecs = 80 * SECOND; + int timeWaiting = 0; + while (!finalStates.contains(app.getState())) { + if (timeWaiting >= timeoutMsecs) { + LOG.info("App State is : " + app.getState()); + return false; + } + + LOG.info("App : " + appId + " State is : " + app.getState() + + " Waiting for state : " + finalStates); + Thread.sleep(WAIT_MS_PER_LOOP); + timeWaiting += WAIT_MS_PER_LOOP; + } + + LOG.info("App State is : " + app.getState()); + return true; + } + + /** + * Wait until an application has reached a specified state. + * The timeout is 80 seconds. + * @param appId the id of an application + * @param finalState the application state waited + * @throws InterruptedException + * if interrupted while waiting for the state transition + */ + public boolean waitForState(ApplicationId appId, RMAppState finalState) + throws InterruptedException { + drainEventsImplicitly(); + RMApp app = getRMContext().getRMApps().get(appId); + final int timeoutMsecs = 80 * SECOND; + int timeWaiting = 0; + while (!finalState.equals(app.getState())) { + if (timeWaiting >= timeoutMsecs) { + LOG.info("App State is : " + app.getState()); + return false; + } + + LOG.info("App : " + appId + " State is : " + app.getState() + + " Waiting for state : " + finalState); + Thread.sleep(WAIT_MS_PER_LOOP); + timeWaiting += WAIT_MS_PER_LOOP; + } + + LOG.info("App State is : " + app.getState()); + return true; + } + + /** + * Wait until an attempt has reached a specified state. + * The timeout is 40 seconds. + * @param attemptId the id of an attempt + * @param finalState the attempt state waited + * @throws InterruptedException + * if interrupted while waiting for the state transition + */ + public boolean waitForState(ApplicationAttemptId attemptId, + RMAppAttemptState finalState) throws InterruptedException { + return waitForState(attemptId, finalState, TIMEOUT_MS_FOR_ATTEMPT); + } + + /** + * Wait until an attempt has reached a specified state. + * The timeout can be specified by the parameter. + * @param attemptId the id of an attempt + * @param finalState the attempt state waited + * @param timeoutMsecs the length of timeout in milliseconds + * @throws InterruptedException + * if interrupted while waiting for the state transition + */ + public boolean waitForState(ApplicationAttemptId attemptId, + RMAppAttemptState finalState, int timeoutMsecs) + throws InterruptedException { + drainEventsImplicitly(); + RMApp app = getRMContext().getRMApps().get(attemptId.getApplicationId()); + RMAppAttempt attempt = app.getRMAppAttempt(attemptId); + return MockRM.waitForState(attempt, finalState, timeoutMsecs); + } + + /** + * Wait until an attempt has reached a specified state. + * The timeout is 40 seconds. + * @param attempt an attempt + * @param finalState the attempt state waited + * @throws InterruptedException + * if interrupted while waiting for the state transition + */ + public static boolean waitForState(RMAppAttempt attempt, + RMAppAttemptState finalState) throws InterruptedException { + return waitForState(attempt, finalState, TIMEOUT_MS_FOR_ATTEMPT); + } + + /** + * Wait until an attempt has reached a specified state. + * The timeout can be specified by the parameter. + * @param attempt an attempt + * @param finalState the attempt state waited + * @param timeoutMsecs the length of timeout in milliseconds + * @throws InterruptedException + * if interrupted while waiting for the state transition + */ + public static boolean waitForState(RMAppAttempt attempt, + RMAppAttemptState finalState, + int timeoutMsecs) + throws InterruptedException { + int timeWaiting = 0; + while (finalState != attempt.getAppAttemptState()) { + if (timeWaiting >= timeoutMsecs) { + LOG.info("Attempt State is : " + attempt.getAppAttemptState()); + return false; + } + + LOG.info("AppAttempt : " + attempt.getAppAttemptId() + " State is : " + + attempt.getAppAttemptState() + " Waiting for state : " + finalState); + Thread.sleep(WAIT_MS_PER_LOOP); + timeWaiting += WAIT_MS_PER_LOOP; + } + + LOG.info("Attempt State is : " + attempt.getAppAttemptState()); + return true; + } + + public void waitForContainerToComplete(RMAppAttempt attempt, + NMContainerStatus completedContainer) throws InterruptedException { + drainEventsImplicitly(); + int timeWaiting = 0; + while (timeWaiting < TIMEOUT_MS_FOR_CONTAINER_AND_NODE) { + List containers = attempt.getJustFinishedContainers(); + LOG.info("Received completed containers " + containers); + for (ContainerStatus container : containers) { + if (container.getContainerId().equals( + completedContainer.getContainerId())) { + return; + } + } + Thread.sleep(WAIT_MS_PER_LOOP); + timeWaiting += WAIT_MS_PER_LOOP; + } + } + + public MockAM waitForNewAMToLaunchAndRegister(ApplicationId appId, int attemptSize, + MockNM nm) throws Exception { + RMApp app = getRMContext().getRMApps().get(appId); + int timeWaiting = 0; + while (app.getAppAttempts().size() != attemptSize) { + if (timeWaiting >= TIMEOUT_MS_FOR_ATTEMPT) { + break; + } + LOG.info("Application " + appId + + " is waiting for AM to restart. Current has " + + app.getAppAttempts().size() + " attempts."); + Thread.sleep(WAIT_MS_PER_LOOP); + timeWaiting += WAIT_MS_PER_LOOP; + } + return launchAndRegisterAM(app, this, nm); + } + + /** + * Wait until a container has reached a specified state. + * The timeout is 10 seconds. + * @param nm A mock nodemanager + * @param containerId the id of a container + * @param containerState the container state waited + * @return if reach the state before timeout; false otherwise. + * @throws Exception + * if interrupted while waiting for the state transition + * or an unexpected error while MockNM is hearbeating. + */ + public boolean waitForState(MockNM nm, ContainerId containerId, + RMContainerState containerState) throws Exception { + return waitForState(nm, containerId, containerState, + TIMEOUT_MS_FOR_CONTAINER_AND_NODE); + } + + /** + * Wait until a container has reached a specified state. + * The timeout is specified by the parameter. + * @param nm A mock nodemanager + * @param containerId the id of a container + * @param containerState the container state waited + * @param timeoutMsecs the length of timeout in milliseconds + * @return if reach the state before timeout; false otherwise. + * @throws Exception + * if interrupted while waiting for the state transition + * or an unexpected error while MockNM is hearbeating. + */ + public boolean waitForState(MockNM nm, ContainerId containerId, + RMContainerState containerState, int timeoutMsecs) throws Exception { + return waitForState(Arrays.asList(nm), containerId, containerState, + timeoutMsecs); + } + + /** + * Wait until a container has reached a specified state. + * The timeout is 10 seconds. + * @param nms array of mock nodemanagers + * @param containerId the id of a container + * @param containerState the container state waited + * @return if reach the state before timeout; false otherwise. + * @throws Exception + * if interrupted while waiting for the state transition + * or an unexpected error while MockNM is hearbeating. + */ + public boolean waitForState(Collection nms, ContainerId containerId, + RMContainerState containerState) throws Exception { + return waitForState(nms, containerId, containerState, + TIMEOUT_MS_FOR_CONTAINER_AND_NODE); + } + + /** + * Wait until a container has reached a specified state. + * The timeout is specified by the parameter. + * @param nms array of mock nodemanagers + * @param containerId the id of a container + * @param containerState the container state waited + * @param timeoutMsecs the length of timeout in milliseconds + * @return if reach the state before timeout; false otherwise. + * @throws Exception + * if interrupted while waiting for the state transition + * or an unexpected error while MockNM is hearbeating. + */ + public boolean waitForState(Collection nms, ContainerId containerId, + RMContainerState containerState, int timeoutMsecs) throws Exception { + drainEventsImplicitly(); + RMContainer container = scheduler.getRMContainer(containerId); + int timeWaiting = 0; + while (container == null) { + if (timeWaiting >= timeoutMsecs) { + return false; + } + + for (MockNM nm : nms) { + nm.nodeHeartbeat(true); + } + drainEventsImplicitly(); + container = scheduler.getRMContainer(containerId); + LOG.info("Waiting for container " + containerId + " to be " + + containerState + ", container is null right now."); + Thread.sleep(WAIT_MS_PER_LOOP); + timeWaiting += WAIT_MS_PER_LOOP; + } + + while (!containerState.equals(container.getState())) { + if (timeWaiting >= timeoutMsecs) { + return false; + } + + LOG.info("Container : " + containerId + " State is : " + + container.getState() + " Waiting for state : " + containerState); + for (MockNM nm : nms) { + nm.nodeHeartbeat(true); + } + drainEventsImplicitly(); + Thread.sleep(WAIT_MS_PER_LOOP); + timeWaiting += WAIT_MS_PER_LOOP; + } + + LOG.info("Container State is : " + container.getState()); + return true; + } + + // get new application id + public GetNewApplicationResponse getNewAppId() throws Exception { + ApplicationClientProtocol client = getClientRMService(); + return client.getNewApplication(Records + .newRecord(GetNewApplicationRequest.class)); + } + + public RMApp submitApp(int masterMemory) throws Exception { + return submitApp(masterMemory, false); + } + + public RMApp submitApp(int masterMemory, + String name, + String user, + boolean unmanaged, + String queue, + Priority priority) throws Exception { + Resource resource = Records.newRecord(Resource.class); + resource.setMemorySize(masterMemory); + return submitApp( + resource, + name, + user, + unmanaged, + queue, + priority + ); + } + + public RMApp submitApp(Resource resource, + String name, + String user, + boolean unmanaged, + String queue, + Priority priority) throws Exception { + priority = (priority == null) ? getDefaultPriority() : priority; + ResourceRequest amResourceRequest = ResourceRequest.newInstance( + priority, ResourceRequest.ANY, resource, 1); + return submitApp(Collections.singletonList(amResourceRequest), name, user, + null, unmanaged, queue, super.getConfig().getInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, + YarnConfiguration.DEFAULT_RM_AM_MAX_ATTEMPTS), null, null, true, + false, false, null, + 0, null, + true, priority, null, null, null, null); + } + + public RMApp submitApp(int masterMemory, Priority priority) throws Exception { + Resource resource = Resource.newInstance(masterMemory, 0); + return submitApp(resource, "", UserGroupInformation.getCurrentUser() + .getShortUserName(), null, false, null, + super.getConfig().getInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, + YarnConfiguration.DEFAULT_RM_AM_MAX_ATTEMPTS), null, null, true, + false, false, null, 0, null, true, priority); + } + + public RMApp submitApp(int masterMemory, Priority priority, + Credentials cred, ByteBuffer tokensConf) throws Exception { + Resource resource = Resource.newInstance(masterMemory, 0); + return submitApp(resource, "", UserGroupInformation.getCurrentUser() + .getShortUserName(), null, false, null, + super.getConfig().getInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, + YarnConfiguration.DEFAULT_RM_AM_MAX_ATTEMPTS), cred, null, true, + false, false, null, 0, null, true, priority, null, null, + tokensConf); + } + + public RMApp submitApp(int masterMemory, boolean unmanaged) + throws Exception { + return submitApp(masterMemory, "", UserGroupInformation.getCurrentUser() + .getShortUserName(), unmanaged); + } + + // client + public RMApp submitApp(int masterMemory, String name, String user) throws Exception { + return submitApp(masterMemory, name, user, false); + } + + public RMApp submitApp(int masterMemory, String name, String user, + boolean unmanaged) + throws Exception { + return submitApp(masterMemory, name, user, null, unmanaged, null, + super.getConfig().getInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, + YarnConfiguration.DEFAULT_RM_AM_MAX_ATTEMPTS), null); + } + + public RMApp submitApp(int masterMemory, String name, String user, + Map acls) throws Exception { + return submitApp(masterMemory, name, user, acls, false, null, + super.getConfig().getInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, + YarnConfiguration.DEFAULT_RM_AM_MAX_ATTEMPTS), null); + } + + public RMApp submitApp(int masterMemory, String name, String user, + Map acls, String queue) throws Exception { + return submitApp(masterMemory, name, user, acls, false, queue, + super.getConfig().getInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, + YarnConfiguration.DEFAULT_RM_AM_MAX_ATTEMPTS), null); + } + + public RMApp submitApp(int masterMemory, String name, String user, + Map acls, String queue, String amLabel) + throws Exception { + Resource resource = Records.newRecord(Resource.class); + resource.setMemorySize(masterMemory); + Priority priority = getDefaultPriority(); + return submitApp(resource, name, user, acls, false, queue, + super.getConfig().getInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, + YarnConfiguration.DEFAULT_RM_AM_MAX_ATTEMPTS), null, null, true, false, + false, null, 0, null, true, priority, amLabel, null, null); + } + + public RMApp submitApp(Resource resource, String name, String user, + Map acls, String queue) throws Exception { + return submitApp(resource, name, user, acls, false, queue, + super.getConfig().getInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, + YarnConfiguration.DEFAULT_RM_AM_MAX_ATTEMPTS), null, null, + true, false, false, null, 0, null, true, null); + } + + public RMApp submitApp(Resource resource, String name, String user, + Map acls, boolean unManaged, String queue) + throws Exception { + return submitApp(resource, name, user, acls, unManaged, queue, + super.getConfig().getInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, + YarnConfiguration.DEFAULT_RM_AM_MAX_ATTEMPTS), null, null, true, + false, false, null, 0, null, true, null); + } + + public RMApp submitApp(int masterMemory, String name, String user, + Map acls, String queue, + boolean waitForAccepted) throws Exception { + return submitApp(masterMemory, name, user, acls, false, queue, + super.getConfig().getInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, + YarnConfiguration.DEFAULT_RM_AM_MAX_ATTEMPTS), null, null, + waitForAccepted); + } + + public RMApp submitApp(int masterMemory, String name, String user, + Map acls, boolean unmanaged, String queue, + int maxAppAttempts, Credentials ts) throws Exception { + return submitApp(masterMemory, name, user, acls, unmanaged, queue, + maxAppAttempts, ts, null); + } + + public RMApp submitApp(int masterMemory, String name, String user, + Map acls, boolean unmanaged, String queue, + int maxAppAttempts, Credentials ts, String appType) throws Exception { + return submitApp(masterMemory, name, user, acls, unmanaged, queue, + maxAppAttempts, ts, appType, true); + } + + public RMApp submitApp(int masterMemory, String name, String user, + Map acls, boolean unmanaged, String queue, + int maxAppAttempts, Credentials ts, String appType, + boolean waitForAccepted) + throws Exception { + return submitApp(masterMemory, name, user, acls, unmanaged, queue, + maxAppAttempts, ts, appType, waitForAccepted, false); + } + + public RMApp submitApp(int masterMemory, String name, String user, + Map acls, boolean unmanaged, String queue, + int maxAppAttempts, Credentials ts, String appType, + boolean waitForAccepted, boolean keepContainers) throws Exception { + Resource resource = Records.newRecord(Resource.class); + resource.setMemorySize(masterMemory); + return submitApp(resource, name, user, acls, unmanaged, queue, + maxAppAttempts, ts, appType, waitForAccepted, keepContainers, + false, null, 0, null, true, getDefaultPriority()); + } + + public RMApp submitApp(int masterMemory, long attemptFailuresValidityInterval, + boolean keepContainers) throws Exception { + Resource resource = Records.newRecord(Resource.class); + resource.setMemorySize(masterMemory); + Priority priority = getDefaultPriority(); + return submitApp(resource, "", UserGroupInformation.getCurrentUser() + .getShortUserName(), null, false, null, + super.getConfig().getInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, + YarnConfiguration.DEFAULT_RM_AM_MAX_ATTEMPTS), null, null, true, keepContainers, + false, null, attemptFailuresValidityInterval, null, true, priority); + } + + public RMApp submitApp(int masterMemory, String name, String user, + Map acls, boolean unmanaged, String queue, + int maxAppAttempts, Credentials ts, String appType, + boolean waitForAccepted, boolean keepContainers, boolean isAppIdProvided, + ApplicationId applicationId) throws Exception { + Resource resource = Records.newRecord(Resource.class); + resource.setMemorySize(masterMemory); + Priority priority = getDefaultPriority(); + return submitApp(resource, name, user, acls, unmanaged, queue, + maxAppAttempts, ts, appType, waitForAccepted, keepContainers, + isAppIdProvided, applicationId, 0, null, true, priority); + } + + public RMApp submitApp(int masterMemory, + LogAggregationContext logAggregationContext) throws Exception { + Resource resource = Records.newRecord(Resource.class); + resource.setMemorySize(masterMemory); + Priority priority = getDefaultPriority(); + return submitApp(resource, "", UserGroupInformation.getCurrentUser() + .getShortUserName(), null, false, null, + super.getConfig().getInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, + YarnConfiguration.DEFAULT_RM_AM_MAX_ATTEMPTS), null, null, true, false, + false, null, 0, logAggregationContext, true, priority); + } + + public RMApp submitApp(Resource capability, String name, String user, + Map acls, boolean unmanaged, String queue, + int maxAppAttempts, Credentials ts, String appType, + boolean waitForAccepted, boolean keepContainers, boolean isAppIdProvided, + ApplicationId applicationId, long attemptFailuresValidityInterval, + LogAggregationContext logAggregationContext, + boolean cancelTokensWhenComplete, Priority priority) throws Exception { + return submitApp(capability, name, user, acls, unmanaged, queue, + maxAppAttempts, ts, appType, waitForAccepted, keepContainers, + isAppIdProvided, applicationId, attemptFailuresValidityInterval, + logAggregationContext, cancelTokensWhenComplete, priority, "", null, + null); + } + + public RMApp submitApp(Credentials cred, ByteBuffer tokensConf) + throws Exception { + return submitApp(Resource.newInstance(200, 1), "app1", "user", null, false, + null, super.getConfig().getInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, + YarnConfiguration.DEFAULT_RM_AM_MAX_ATTEMPTS), cred, null, true, + false, false, null, 0, null, true, getDefaultPriority(), null, null, + tokensConf); + } + + public RMApp submitApp(List amResourceRequests) + throws Exception { + return submitApp(amResourceRequests, "app1", "user", null, false, null, + super.getConfig().getInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, + YarnConfiguration.DEFAULT_RM_AM_MAX_ATTEMPTS), null, null, true, + false, false, null, 0, null, true, + amResourceRequests.get(0).getPriority(), + amResourceRequests.get(0).getNodeLabelExpression(), null, null, null); + } + + public RMApp submitApp(List amResourceRequests, + String appNodeLabel) throws Exception { + return submitApp(amResourceRequests, "app1", "user", null, false, null, + super.getConfig().getInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, + YarnConfiguration.DEFAULT_RM_AM_MAX_ATTEMPTS), null, null, true, + false, false, null, 0, null, true, + amResourceRequests.get(0).getPriority(), + amResourceRequests.get(0).getNodeLabelExpression(), null, null, + appNodeLabel); + } + + public RMApp submitApp(Resource capability, String name, String user, + Map acls, boolean unmanaged, String queue, + int maxAppAttempts, Credentials ts, String appType, + boolean waitForAccepted, boolean keepContainers, boolean isAppIdProvided, + ApplicationId applicationId, long attemptFailuresValidityInterval, + LogAggregationContext logAggregationContext, + boolean cancelTokensWhenComplete, Priority priority, String amLabel, + Map applicationTimeouts, + ByteBuffer tokensConf) + throws Exception { + priority = (priority == null) ? getDefaultPriority() : priority; + ResourceRequest amResourceRequest = ResourceRequest.newInstance( + priority, ResourceRequest.ANY, capability, 1); + if (amLabel != null && !amLabel.isEmpty()) { + amResourceRequest.setNodeLabelExpression(amLabel.trim()); + } + return submitApp(Collections.singletonList(amResourceRequest), name, user, + acls, unmanaged, queue, maxAppAttempts, ts, appType, waitForAccepted, + keepContainers, isAppIdProvided, applicationId, + attemptFailuresValidityInterval, logAggregationContext, + cancelTokensWhenComplete, priority, amLabel, applicationTimeouts, + tokensConf, null); + } + + public RMApp submitApp(List amResourceRequests, String name, + String user, Map acls, boolean unmanaged, + String queue, int maxAppAttempts, Credentials ts, String appType, + boolean waitForAccepted, boolean keepContainers, boolean isAppIdProvided, + ApplicationId applicationId, long attemptFailuresValidityInterval, + LogAggregationContext logAggregationContext, + boolean cancelTokensWhenComplete, Priority priority, String amLabel, + Map applicationTimeouts, + ByteBuffer tokensConf, String appNodeLabel) throws Exception { + ApplicationId appId = isAppIdProvided ? applicationId : null; + ApplicationClientProtocol client = getClientRMService(); + if (! isAppIdProvided) { + GetNewApplicationResponse resp = client.getNewApplication(Records + .newRecord(GetNewApplicationRequest.class)); + appId = resp.getApplicationId(); + } + SubmitApplicationRequest req = Records + .newRecord(SubmitApplicationRequest.class); + ApplicationSubmissionContext sub = Records + .newRecord(ApplicationSubmissionContext.class); + sub.setKeepContainersAcrossApplicationAttempts(keepContainers); + sub.setApplicationId(appId); + sub.setApplicationName(name); + sub.setMaxAppAttempts(maxAppAttempts); + if (applicationTimeouts != null && applicationTimeouts.size() > 0) { + sub.setApplicationTimeouts(applicationTimeouts); + } + if (unmanaged) { + sub.setUnmanagedAM(true); + } + if (queue != null) { + sub.setQueue(queue); + } + if (priority != null) { + sub.setPriority(priority); + } + if (appNodeLabel != null) { + sub.setNodeLabelExpression(appNodeLabel); + } + sub.setApplicationType(appType); + ContainerLaunchContext clc = Records + .newRecord(ContainerLaunchContext.class); + clc.setApplicationACLs(acls); + if (ts != null && UserGroupInformation.isSecurityEnabled()) { + DataOutputBuffer dob = new DataOutputBuffer(); + ts.writeTokenStorageToStream(dob); + ByteBuffer securityTokens = ByteBuffer.wrap(dob.getData(), 0, dob.getLength()); + clc.setTokens(securityTokens); + clc.setTokensConf(tokensConf); + } + sub.setAMContainerSpec(clc); + sub.setAttemptFailuresValidityInterval(attemptFailuresValidityInterval); + if (logAggregationContext != null) { + sub.setLogAggregationContext(logAggregationContext); + } + sub.setCancelTokensWhenComplete(cancelTokensWhenComplete); + if (amLabel != null && !amLabel.isEmpty()) { + for (ResourceRequest amResourceRequest : amResourceRequests) { + amResourceRequest.setNodeLabelExpression(amLabel.trim()); + } + } + sub.setAMContainerResourceRequests(amResourceRequests); + req.setApplicationSubmissionContext(sub); + UserGroupInformation fakeUser = + UserGroupInformation.createUserForTesting(user, new String[] {"someGroup"}); + PrivilegedExceptionAction action = + new PrivilegedExceptionAction() { + ApplicationClientProtocol client; + SubmitApplicationRequest req; + @Override + public SubmitApplicationResponse run() throws IOException, YarnException { + try { + return client.submitApplication(req); + } catch (YarnException | IOException e) { + e.printStackTrace(); + throw e; + } + } + PrivilegedExceptionAction setClientReq( + ApplicationClientProtocol client, SubmitApplicationRequest req) { + this.client = client; + this.req = req; + return this; + } + }.setClientReq(client, req); + fakeUser.doAs(action); + // make sure app is immediately available after submit + if (waitForAccepted) { + waitForState(appId, RMAppState.ACCEPTED); + } + RMApp rmApp = getRMContext().getRMApps().get(appId); + + // unmanaged AM won't go to RMAppAttemptState.SCHEDULED. + if (waitForAccepted && !unmanaged) { + waitForState(rmApp.getCurrentAppAttempt().getAppAttemptId(), + RMAppAttemptState.SCHEDULED); + } + + ((AbstractYarnScheduler)scheduler).update(); + + return rmApp; + } + + private Priority getDefaultPriority() { + return Priority.newInstance( + super.getConfig() + .getInt( + YarnConfiguration.NM_CONTAINER_EXECUTOR_SCHED_PRIORITY, + YarnConfiguration.DEFAULT_NM_CONTAINER_EXECUTOR_SCHED_PRIORITY)); + } + + public MockNM unRegisterNode(MockNM nm) throws Exception { + nm.unRegisterNode(); + drainEventsImplicitly(); + return nm; + } + + public MockNM registerNode(String nodeIdStr, int memory) throws Exception { + MockNM nm = new MockNM(nodeIdStr, memory, getResourceTrackerService()); + nm.registerNode(); + drainEventsImplicitly(); + return nm; + } + + public MockNM registerNode(String nodeIdStr, int memory, int vCores) + throws Exception { + MockNM nm = + new MockNM(nodeIdStr, memory, vCores, getResourceTrackerService()); + nm.registerNode(); + drainEventsImplicitly(); + return nm; + } + + public MockNM registerNode(String nodeIdStr, int memory, int vCores, + List runningApplications) throws Exception { + MockNM nm = + new MockNM(nodeIdStr, memory, vCores, getResourceTrackerService(), + YarnVersionInfo.getVersion()); + nm.registerNode(runningApplications); + drainEventsImplicitly(); + return nm; + } + + public MockNM registerNode(String nodeIdStr, Resource nodeCapability) + throws Exception { + MockNM nm = new MockNM(nodeIdStr, nodeCapability, + getResourceTrackerService()); + nm.registerNode(); + drainEventsImplicitly(); + return nm; + } + + public void sendNodeStarted(MockNM nm) throws Exception { + RMNodeImpl node = (RMNodeImpl) getRMContext().getRMNodes().get( + nm.getNodeId()); + NodeStatus mockNodeStatus = MockNM.createMockNodeStatus(); + node.handle(new RMNodeStartedEvent(nm.getNodeId(), null, null, + mockNodeStatus)); + drainEventsImplicitly(); + } + + public void sendNodeLost(MockNM nm) throws Exception { + RMNodeImpl node = (RMNodeImpl) getRMContext().getRMNodes().get( + nm.getNodeId()); + node.handle(new RMNodeEvent(nm.getNodeId(), RMNodeEventType.EXPIRE)); + drainEventsImplicitly(); + } + + private RMNode getRMNode(NodeId nodeId) { + RMNode node = getRMContext().getRMNodes().get(nodeId); + if (node == null) { + node = getRMContext().getInactiveRMNodes().get(nodeId); + } + return node; + } + + /** + * Wait until a node has reached a specified state. + * The timeout is 20 seconds. + * @param nodeId the id of a node + * @param finalState the node state waited + * @throws InterruptedException + * if interrupted while waiting for the state transition + */ + public boolean waitForState(NodeId nodeId, NodeState finalState) + throws InterruptedException { + drainEventsImplicitly(); + int timeWaiting = 0; + RMNode node = getRMNode(nodeId); + while (node == null) { + if (timeWaiting >= TIMEOUT_MS_FOR_CONTAINER_AND_NODE) { + LOG.info("Node " + nodeId + " State is : " + node.getState()); + return false; + } + node = getRMNode(nodeId); + Thread.sleep(WAIT_MS_PER_LOOP); + timeWaiting += WAIT_MS_PER_LOOP; + } + while (!finalState.equals(node.getState())) { + if (timeWaiting >= TIMEOUT_MS_FOR_CONTAINER_AND_NODE) { + LOG.info("Node " + nodeId + " State is : " + node.getState()); + return false; + } + + LOG.info("Node State is : " + node.getState() + + " Waiting for state : " + finalState); + Thread.sleep(WAIT_MS_PER_LOOP); + timeWaiting += WAIT_MS_PER_LOOP; + } + + LOG.info("Node " + nodeId + " State is : " + node.getState()); + return true; + } + + public void sendNodeGracefulDecommission( + MockNM nm, int timeout) throws Exception { + RMNodeImpl node = (RMNodeImpl) + getRMContext().getRMNodes().get(nm.getNodeId()); + node.handle(new RMNodeDecommissioningEvent(nm.getNodeId(), timeout)); + } + + public void sendNodeEvent(MockNM nm, RMNodeEventType event) throws Exception { + RMNodeImpl node = (RMNodeImpl) + getRMContext().getRMNodes().get(nm.getNodeId()); + node.handle(new RMNodeEvent(nm.getNodeId(), event)); + } + + public Integer getDecommissioningTimeout(NodeId nodeid) { + return this.getRMContext().getRMNodes() + .get(nodeid).getDecommissioningTimeout(); + } + + public KillApplicationResponse killApp(ApplicationId appId) throws Exception { + ApplicationClientProtocol client = getClientRMService(); + KillApplicationRequest req = KillApplicationRequest.newInstance(appId); + KillApplicationResponse response = client.forceKillApplication(req); + drainEventsImplicitly(); + return response; + } + + public FailApplicationAttemptResponse failApplicationAttempt( + ApplicationAttemptId attemptId) throws Exception { + ApplicationClientProtocol client = getClientRMService(); + FailApplicationAttemptRequest req = + FailApplicationAttemptRequest.newInstance(attemptId); + FailApplicationAttemptResponse response = + client.failApplicationAttempt(req); + drainEventsImplicitly(); + return response; + } + + /** + * recommend to use launchAM, or use sendAMLaunched like: + * 1, wait RMAppAttempt scheduled + * 2, send node heartbeat + * 3, sendAMLaunched + */ + public MockAM sendAMLaunched(ApplicationAttemptId appAttemptId) + throws Exception { + MockAM am = new MockAM(getRMContext(), masterService, appAttemptId); + ((AbstractYarnScheduler)scheduler).update(); + waitForState(appAttemptId, RMAppAttemptState.ALLOCATED); + //create and set AMRMToken + Token amrmToken = + this.rmContext.getAMRMTokenSecretManager().createAndGetAMRMToken( + appAttemptId); + ((RMAppAttemptImpl) this.rmContext.getRMApps() + .get(appAttemptId.getApplicationId()).getRMAppAttempt(appAttemptId)) + .setAMRMToken(amrmToken); + getRMContext() + .getDispatcher() + .getEventHandler() + .handle( + new RMAppAttemptEvent(appAttemptId, RMAppAttemptEventType.LAUNCHED)); + drainEventsImplicitly(); + return am; + } + + /** + * Launches the AM. + * @param am + */ + public void launchAM(final MockAM am) { + //create and set AMRMToken + Token amrmToken = + this.rmContext.getAMRMTokenSecretManager().createAndGetAMRMToken( + am.getApplicationAttemptId()); + ((RMAppAttemptImpl) this.rmContext.getRMApps() + .get(am.getApplicationAttemptId().getApplicationId()).getRMAppAttempt(am.getApplicationAttemptId())) + .setAMRMToken(amrmToken); + getRMContext() + .getDispatcher() + .getEventHandler() + .handle( + new RMAppAttemptEvent(am.getApplicationAttemptId(), RMAppAttemptEventType.LAUNCHED)); + drainEventsImplicitly(); + } + + public void sendAMLaunchFailed(ApplicationAttemptId appAttemptId) + throws Exception { + MockAM am = new MockAM(getRMContext(), masterService, appAttemptId); + waitForState(am.getApplicationAttemptId(), RMAppAttemptState.ALLOCATED); + getRMContext().getDispatcher().getEventHandler() + .handle(new RMAppAttemptEvent(appAttemptId, + RMAppAttemptEventType.LAUNCH_FAILED, "Failed")); + drainEventsImplicitly(); + } + + @Override + protected ClientRMService createClientRMService() { + return new ClientRMService(getRMContext(), scheduler, + rmAppManager, applicationACLsManager, queueACLsManager, + getRMContext().getRMDelegationTokenSecretManager(), clock) { + @Override + protected void serviceStart() { + // override to not start rpc handler + } + + @Override + protected void serviceStop() { + // don't do anything + } + }; + } + + @Override + protected ResourceTrackerService createResourceTrackerService() { + + RMContainerTokenSecretManager containerTokenSecretManager = + getRMContext().getContainerTokenSecretManager(); + containerTokenSecretManager.rollMasterKey(); + NMTokenSecretManagerInRM nmTokenSecretManager = + getRMContext().getNMTokenSecretManager(); + nmTokenSecretManager.rollMasterKey(); + return new ResourceTrackerService(getRMContext(), nodesListManager, + this.nmLivelinessMonitor, containerTokenSecretManager, + nmTokenSecretManager) { + + @Override + protected void serviceStart() { + // override to not start rpc handler + } + + @Override + protected void serviceStop() { + // don't do anything + } + }; + } + + @Override + protected ApplicationMasterService createApplicationMasterService() { + if (this.rmContext.getYarnConfiguration().getBoolean( + YarnConfiguration.OPPORTUNISTIC_CONTAINER_ALLOCATION_ENABLED, + YarnConfiguration.DEFAULT_OPPORTUNISTIC_CONTAINER_ALLOCATION_ENABLED)) { + return new OpportunisticContainerAllocatorAMService(getRMContext(), scheduler) { + @Override + protected void serviceStart() { + // override to not start rpc handler + } + + @Override + protected void serviceStop() { + // don't do anything + } + }; + } + return new ApplicationMasterService(getRMContext(), scheduler) { + @Override + protected void serviceStart() { + // override to not start rpc handler + } + + @Override + protected void serviceStop() { + // don't do anything + } + }; + } + + @Override + protected ApplicationMasterLauncher createAMLauncher() { + return new ApplicationMasterLauncher(getRMContext()) { + @Override + protected void serviceStart() { + // override to not start rpc handler + } + + @Override + public void handle(AMLauncherEvent appEvent) { + // don't do anything + } + + @Override + protected void serviceStop() { + // don't do anything + } + }; + } + + @Override + protected AdminService createAdminService() { + return new AdminService(this) { + @Override + protected void startServer() { + // override to not start rpc handler + } + + @Override + protected void stopServer() { + // don't do anything + } + }; + } + + public NodesListManager getNodesListManager() { + return this.nodesListManager; + } + + public ClientToAMTokenSecretManagerInRM getClientToAMTokenSecretManager() { + return this.getRMContext().getClientToAMTokenSecretManager(); + } + + public RMAppManager getRMAppManager() { + return this.rmAppManager; + } + + public AdminService getAdminService() { + return this.adminService; + } + + @Override + protected void startWepApp() { + if (getConfig().getBoolean(ENABLE_WEBAPP, false)) { + super.startWepApp(); + return; + } + + // Disable webapp + } + + public static void finishAMAndVerifyAppState(RMApp rmApp, MockRM rm, MockNM nm, + MockAM am) throws Exception { + FinishApplicationMasterRequest req = + FinishApplicationMasterRequest.newInstance( + FinalApplicationStatus.SUCCEEDED, "", ""); + am.unregisterAppAttempt(req,true); + rm.waitForState(am.getApplicationAttemptId(), RMAppAttemptState.FINISHING); + nm.nodeHeartbeat(am.getApplicationAttemptId(), 1, ContainerState.COMPLETE); + rm.drainEventsImplicitly(); + rm.waitForState(am.getApplicationAttemptId(), RMAppAttemptState.FINISHED); + rm.waitForState(rmApp.getApplicationId(), RMAppState.FINISHED); + } + + @SuppressWarnings("rawtypes") + private static void waitForSchedulerAppAttemptAdded( + ApplicationAttemptId attemptId, MockRM rm) throws InterruptedException { + int tick = 0; + rm.drainEventsImplicitly(); + // Wait for at most 5 sec + while (null == ((AbstractYarnScheduler) rm.getResourceScheduler()) + .getApplicationAttempt(attemptId) && tick < 50) { + Thread.sleep(100); + if (tick % 10 == 0) { + LOG.info("waiting for SchedulerApplicationAttempt=" + + attemptId + " added."); + } + tick++; + } + } + + public static MockAM launchAMWhenAsyncSchedulingEnabled(RMApp app, MockRM rm) + throws Exception { + int i = 0; + while (app.getCurrentAppAttempt() == null) { + if (i < 100) { + i++; + } + Thread.sleep(50); + } + + RMAppAttempt attempt = app.getCurrentAppAttempt(); + + rm.waitForState(attempt.getAppAttemptId(), + RMAppAttemptState.ALLOCATED); + MockAM am = rm.sendAMLaunched(attempt.getAppAttemptId()); + rm.waitForState(attempt.getAppAttemptId(), RMAppAttemptState.LAUNCHED); + + return am; + } + + /** + * NOTE: nm.nodeHeartbeat is explicitly invoked, + * don't invoke it before calling launchAM + */ + public static MockAM launchAM(RMApp app, MockRM rm, MockNM nm) + throws Exception { + rm.drainEventsImplicitly(); + RMAppAttempt attempt = waitForAttemptScheduled(app, rm); + LOG.info("Launch AM " + attempt.getAppAttemptId()); + nm.nodeHeartbeat(true); + ((AbstractYarnScheduler)rm.getResourceScheduler()).update(); + rm.drainEventsImplicitly(); + nm.nodeHeartbeat(true); + MockAM am = rm.sendAMLaunched(attempt.getAppAttemptId()); + rm.waitForState(attempt.getAppAttemptId(), RMAppAttemptState.LAUNCHED); + return am; + } + + public static MockAM launchUAM(RMApp app, MockRM rm, MockNM nm) + throws Exception { + rm.drainEventsImplicitly(); + // UAMs go directly to LAUNCHED state + rm.waitForState(app.getApplicationId(), RMAppState.ACCEPTED); + RMAppAttempt attempt = app.getCurrentAppAttempt(); + waitForSchedulerAppAttemptAdded(attempt.getAppAttemptId(), rm); + LOG.info("Launch AM " + attempt.getAppAttemptId()); + nm.nodeHeartbeat(true); + ((AbstractYarnScheduler)rm.getResourceScheduler()).update(); + rm.drainEventsImplicitly(); + nm.nodeHeartbeat(true); + MockAM am = new MockAM(rm.getRMContext(), rm.masterService, + attempt.getAppAttemptId()); + rm.waitForState(attempt.getAppAttemptId(), RMAppAttemptState.LAUNCHED); + return am; + } + + public static RMAppAttempt waitForAttemptScheduled(RMApp app, MockRM rm) + throws Exception { + rm.waitForState(app.getApplicationId(), RMAppState.ACCEPTED); + RMAppAttempt attempt = app.getCurrentAppAttempt(); + waitForSchedulerAppAttemptAdded(attempt.getAppAttemptId(), rm); + rm.waitForState(attempt.getAppAttemptId(), RMAppAttemptState.SCHEDULED); + return attempt; + } + + public static MockAM launchAndRegisterAM(RMApp app, MockRM rm, MockNM nm) + throws Exception { + MockAM am = launchAM(app, rm, nm); + am.registerAppAttempt(); + rm.waitForState(app.getApplicationId(), RMAppState.RUNNING); + return am; + } + + public static MockAM launchAndRegisterAM(RMApp app, MockRM rm, MockNM nm, + Map, PlacementConstraint> constraints) throws Exception { + MockAM am = launchAM(app, rm, nm); + for (Map.Entry, PlacementConstraint> e : + constraints.entrySet()) { + am.addPlacementConstraint(e.getKey(), e.getValue()); + } + am.registerAppAttempt(); + rm.waitForState(app.getApplicationId(), RMAppState.RUNNING); + return am; + } + + public ApplicationReport getApplicationReport(ApplicationId appId) + throws YarnException, IOException { + ApplicationClientProtocol client = getClientRMService(); + GetApplicationReportResponse response = + client.getApplicationReport(GetApplicationReportRequest + .newInstance(appId)); + return response.getApplicationReport(); + } + + public void updateReservationState(ReservationUpdateRequest request) + throws IOException, YarnException { + ApplicationClientProtocol client = getClientRMService(); + client.updateReservation(request); + drainEventsImplicitly(); + } + + // Explicitly reset queue metrics for testing. + @SuppressWarnings("static-access") + public void clearQueueMetrics(RMApp app) { + ((AbstractYarnScheduler) getResourceScheduler()) + .getSchedulerApplications().get(app.getApplicationId()).getQueue() + .getMetrics().clearQueueMetrics(); + } + + public RMActiveServices getRMActiveService() { + return activeServices; + } + + public void signalToContainer(ContainerId containerId, + SignalContainerCommand command) throws Exception { + ApplicationClientProtocol client = getClientRMService(); + SignalContainerRequest req = + SignalContainerRequest.newInstance(containerId, command); + client.signalToContainer(req); + drainEventsImplicitly(); + } + + /** + * Wait until an app removed from scheduler. + * The timeout is 40 seconds. + * @param appId the id of an app + * @throws InterruptedException + * if interrupted while waiting for app removed + */ + public void waitForAppRemovedFromScheduler(ApplicationId appId) + throws InterruptedException { + int timeWaiting = 0; + drainEventsImplicitly(); + + Map apps = + ((AbstractYarnScheduler) getResourceScheduler()) + .getSchedulerApplications(); + while (apps.containsKey(appId)) { + if (timeWaiting >= TIMEOUT_MS_FOR_APP_REMOVED) { + break; + } + LOG.info("wait for app removed, " + appId); + Thread.sleep(WAIT_MS_PER_LOOP); + timeWaiting += WAIT_MS_PER_LOOP; + } + LOG.info("app is removed from scheduler, " + appId); + } + + private void drainEventsImplicitly() { + if (!disableDrainEventsImplicitly) { + drainEvents(); + } + } + + public void disableDrainEventsImplicitly() { + disableDrainEventsImplicitly = true; + } + + public void enableDrainEventsImplicityly() { + disableDrainEventsImplicitly = false; + } + + public RMApp submitApp(int masterMemory, Priority priority, + Map applicationTimeouts) throws Exception { + Resource resource = Resource.newInstance(masterMemory, 0); + return submitApp( + resource, "", UserGroupInformation.getCurrentUser().getShortUserName(), + null, false, null, + super.getConfig().getInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, + YarnConfiguration.DEFAULT_RM_AM_MAX_ATTEMPTS), null, null, true, + false, false, null, 0, null, true, priority, null, applicationTimeouts, + null); + } + + @Override + protected void serviceInit(Configuration conf) throws Exception { + super.serviceInit(conf); + if (rmContext.getDispatcher() instanceof AsyncDispatcher) { + ((AsyncDispatcher) rmContext.getDispatcher()).disableExitOnDispatchException(); + } + + metricsManager.setScheduler(scheduler); + if (scheduler == null) { + throw new NullArgumentException("Scheduler cannot be null!"); + } + + if (metricsManager.isMetricsOn()) { + metricsManager.init(); + metricsManager.start(); + } + + serviceInitialized = true; + } + + public RMStateStore getRMStateStore() { + return getRMContext().getStateStore(); + } + + @Override + public void stop() { + super.stop(); + metricsManager.stop(); + } +} diff --git a/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/rm/NullRMNodeLabelsManager.java b/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/rm/NullRMNodeLabelsManager.java new file mode 100644 index 00000000000..16b8aebbbb4 --- /dev/null +++ b/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/rm/NullRMNodeLabelsManager.java @@ -0,0 +1,103 @@ +package org.apache.hadoop.yarn.dtss.rm; + +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.yarn.api.records.NodeId; +import org.apache.hadoop.yarn.api.records.NodeLabel; +import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.apache.hadoop.yarn.nodelabels.CommonNodeLabelsManager; +import org.apache.hadoop.yarn.nodelabels.NodeLabelsStore; +import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager; + +import java.io.IOException; +import java.util.Collection; +import java.util.List; +import java.util.Map; +import java.util.Set; + +/** + * NOTE: Based on classes that support MockRM in the tests package + * for resourcemanager. + */ +public class NullRMNodeLabelsManager extends RMNodeLabelsManager { + @Override + public void initNodeLabelStore(Configuration conf) { + this.store = new NodeLabelsStore() { + + @Override + public void recover() + throws IOException { + // do nothing + } + + @Override + public void init(Configuration conf, CommonNodeLabelsManager mgr) + throws Exception { + + } + + @Override + public void removeClusterNodeLabels(Collection labels) + throws IOException { + // do nothing + } + + @Override + public void updateNodeToLabelsMappings( + Map> nodeToLabels) throws IOException { + // do nothing + } + + @Override + public void storeNewClusterNodeLabels(List label) + throws IOException { + // do nothing + } + + @Override + public void close() throws IOException { + // do nothing + } + }; + } + + @Override + protected void initDispatcher(Configuration conf) { + super.dispatcher = null; + } + + @Override + protected void startDispatcher() { + // do nothing + } + + @Override + protected void stopDispatcher() { + // do nothing + } + + @Override + protected void serviceInit(Configuration conf) throws Exception { + // always enable node labels while using MemoryRMNodeLabelsManager + conf.setBoolean(YarnConfiguration.NODE_LABELS_ENABLED, true); + super.serviceInit(conf); + } +} + diff --git a/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/stats/ClusterStats.java b/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/stats/ClusterStats.java new file mode 100644 index 00000000000..9e9c66e12ed --- /dev/null +++ b/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/stats/ClusterStats.java @@ -0,0 +1,166 @@ +package org.apache.hadoop.yarn.dtss.stats; + +import org.apache.commons.csv.CSVFormat; +import org.apache.commons.csv.CSVPrinter; +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.apache.hadoop.yarn.dtss.cluster.ClusterTopologyBuilder; +import org.apache.hadoop.yarn.dtss.cluster.ClusterTopologyProperties; +import org.apache.hadoop.yarn.dtss.config.parameters.runner.SimulationConfigPath; +import org.apache.hadoop.yarn.dtss.exceptions.StateException; +import org.apache.hadoop.yarn.dtss.lifecycle.LifeCycle; +import org.apache.hadoop.yarn.dtss.metrics.MetricsConfiguration; +import org.apache.hadoop.yarn.dtss.time.SimulatedClock; + +import javax.inject.Inject; +import java.io.FileWriter; +import java.io.IOException; +import java.text.DecimalFormat; +import java.util.List; +import java.util.logging.Level; +import java.util.logging.Logger; + +/** + * Supports the writing of overall cluster metrics. + */ +@InterfaceAudience.Private +public final class ClusterStats extends LifeCycle { + private static final Logger LOG = Logger.getLogger(ClusterStats.class.getName()); + + private static long GB = 1024L; + private static DecimalFormat df2 = new DecimalFormat(".##"); + + private final String config; + private final String scheduler; + private final MetricsConfiguration metricsConfig; + private final ClusterTopologyBuilder ctpBuilder; + private final SimulatedClock clock; + + // Cluster stats + private int racks; + private int nodes; + // Per container stats + private long containerMemGB; + private int containerCores; + // Per node stats + private long nodeMemGB; + private int nodeCores; + + private double clusterMemUtilization; + private double clusterCoreUtilization; + + @Inject + private ClusterStats( + @SimulationConfigPath final String configPath, + final MetricsConfiguration metricsConfig, + final Configuration yarnConf, + final ClusterTopologyBuilder ctpBuilder, + final SimulatedClock clock) { + this.metricsConfig = metricsConfig; + final String schedulerClass = yarnConf.get(YarnConfiguration.RM_SCHEDULER); + this.config = configPath.substring(configPath.lastIndexOf('/') + 1); + this.scheduler = schedulerClass.substring( + schedulerClass.lastIndexOf('.') + 1, schedulerClass.lastIndexOf("Scheduler")); + this.ctpBuilder = ctpBuilder; + this.clock = clock; + } + + @Override + public void start() { + super.start(); + try { + final ClusterTopologyProperties ctp = ctpBuilder.getClusterTopology().getClusterTopologyProperties(); + this.racks = ctp.getClusterRacks(); + this.nodes = ctp.getClusterNodes(); + this.containerMemGB = ctp.getContainerMB() / GB; + this.containerCores = ctp.getContainerVCores(); + // TODO: Handle non-homogeneous cluster + // Assuming homogeneous cluster (same number of containers per node) + int containersPerNode = ctp.getRackNodeContainers().get(0).get(0); + this.nodeMemGB = containersPerNode * containerMemGB; + this.nodeCores = containersPerNode * containerCores; + } catch (final IOException ioe) { + throw new RuntimeException(ioe); + } + } + + public void setClusterMemUtilization(final List clusterMemTS) { + if (lifeCycleState.isDone()) { + throw new StateException("Unable to set cluster memory utilization, as the state is done!"); + } + + final StatUtils su = new StatUtils(clusterMemTS.stream().mapToDouble(Double::doubleValue).toArray()); + this.clusterMemUtilization = su.getMean(); + } + + public void setClusterCoreUtilization(final List clusterCoresTS) { + if (lifeCycleState.isDone()) { + throw new StateException("Unable to set cluster core utilization, as the state is done!"); + } + + final StatUtils su = new StatUtils(clusterCoresTS.stream().mapToDouble(Double::doubleValue).toArray()); + this.clusterCoreUtilization = su.getMean(); + } + + @Override + public String toString() { + return String.format("%" + 18 + "s", config) + ", " + + String.format("%" + 10 + "s", scheduler) + ", " + + String.format("%" + 6 + "s", racks) + ", " + + String.format("%" + 6 + "s", nodes) + ", " + + String.format("%" + 17 + "s", containerMemGB) + ", " + + String.format("%" + 15 + "s", containerCores) + ", " + + String.format("%" + 12 + "s", nodeMemGB) + ", " + + String.format("%" + 10 + "s", nodeCores) + ", " + + String.format("%" + 18 + "s", df2.format(clusterMemUtilization)) + ", " + + String.format("%" + 18 + "s", df2.format(clusterCoreUtilization)); + } + + public void writeResultsToTsv() { + if (!metricsConfig.isMetricsOn() || metricsConfig.getMetricsDirectory() == null || + metricsConfig.getMetricsDirectory().isEmpty()) { + LOG.log(Level.INFO, "Metrics is not turned on, not writing results to TSV..."); + return; + } + + final String clusterStatsFilePathStr = metricsConfig.getMetricsFilePath("clusterStatsFile.tsv"); + assert clusterStatsFilePathStr != null; + + LOG.log(Level.INFO, "Writing cluster stats to " + clusterStatsFilePathStr + "..."); + try (final CSVPrinter printer = new CSVPrinter(new FileWriter(clusterStatsFilePathStr), CSVFormat.TDF)) { + printer.printRecord( + "ConfigPath", + "SchedulerName", + "Racks", + "NumNodesPerRack", + "ContainerMemoryGB", + "ContainerCoresGB", + "NodeMemoryGB", + "NodeCores", + "ClusterMemUtilization", + "ClusterCoreUtilization", + "StartTimeEpochSeconds" + ); + + printer.printRecord( + config, + scheduler, + racks, + nodes, + containerMemGB, + containerCores, + nodeMemGB, + nodeCores, + clusterMemUtilization, + clusterCoreUtilization, + clock.getSimulationStartTime().getEpochSecond() + ); + } catch (final IOException e) { + e.printStackTrace(); + throw new RuntimeException(e); + } + + LOG.log(Level.INFO, "Done writing cluster stats!"); + } +} diff --git a/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/stats/JobStats.java b/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/stats/JobStats.java new file mode 100644 index 00000000000..1361f0b1827 --- /dev/null +++ b/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/stats/JobStats.java @@ -0,0 +1,146 @@ +package org.apache.hadoop.yarn.dtss.stats; + +import org.apache.commons.csv.CSVFormat; +import org.apache.commons.csv.CSVPrinter; +import org.apache.commons.csv.QuoteMode; +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.yarn.dtss.exceptions.StateException; +import org.apache.hadoop.yarn.dtss.job.SimulatedJob; +import org.apache.hadoop.yarn.dtss.lifecycle.LifeCycle; +import org.apache.hadoop.yarn.dtss.metrics.MetricsConfiguration; +import org.apache.hadoop.yarn.dtss.metrics.job.JobHistory; +import org.apache.hadoop.yarn.dtss.time.SimulatedClock; + +import javax.annotation.Nullable; +import javax.inject.Inject; +import java.io.FileWriter; +import java.io.IOException; +import java.text.DecimalFormat; +import java.util.Comparator; +import java.util.LinkedList; +import java.util.List; +import java.util.logging.Level; +import java.util.logging.Logger; +import java.util.stream.Collectors; + +/** + * Supports the writing of overall job metrics. + */ +@InterfaceAudience.Private +public final class JobStats extends LifeCycle { + private static final Logger LOG = Logger.getLogger(JobStats.class.getName()); + private static final DecimalFormat df2 = new DecimalFormat(".##"); + + private final MetricsConfiguration metricsConfig; + private final SimulatedClock clock; + + @Inject + private JobStats( + final MetricsConfiguration metricsConfig, + final SimulatedClock clock) { + this.metricsConfig = metricsConfig; + this.clock = clock; + } + + private static final class SimulatedJobWrapper { + private JobHistory jobHistory; + private SimulatedJob simulatedJob; + + private SimulatedJobWrapper(SimulatedJob sj, JobHistory jh) { + this.simulatedJob = sj; + this.jobHistory = jh; + } + } + + private final List jobHistoryList = new LinkedList<>(); + + public void appendJobHistory(final SimulatedJob job, final JobHistory jobHistory) { + if (this.lifeCycleState.isDone()) { + throw new StateException( + "Cannot append job history for job " + job.getTraceJobId() + + ", because JobStats is in state " + this.lifeCycleState); + } + + final SimulatedJobWrapper sjw = new SimulatedJobWrapper(job, jobHistory); + this.jobHistoryList.add(sjw); + } + + private List getSortedJobHistoryBySubmissionTime() { + return jobHistoryList.stream() + .sorted(Comparator.comparing(s -> { + return s.jobHistory.getSubmitTimeSeconds() == null ? Long.MAX_VALUE : s.jobHistory.getSubmitTimeSeconds(); + })) + .collect(Collectors.toList()); + } + + @Override + public String toString() { + final StringBuilder sb = new StringBuilder(); + final List submissionTimeSortedHistory = getSortedJobHistoryBySubmissionTime(); + + sb.append("ApplicationID, NumContainers, EndState, SubmitTimeTS, StartTimeTS, EndTimeTS\n"); + for(final SimulatedJobWrapper sjw : submissionTimeSortedHistory) { + sb.append(String.format("%" + 11 + "s", sjw.jobHistory.getApplicationId()) + ", "); + sb.append(String.format("%" + 11 + "s", sjw.jobHistory.getQueue() + ", ")); + sb.append(String.format("%" + 13 + "s", sjw.simulatedJob.getContainers().size() + ", ")); + sb.append(String.format("%" + 11 + "s", sjw.jobHistory.getEndState() + ", ")); + sb.append(String.format("%" + 13 + "s", sjw.jobHistory.getSubmitTimeSeconds() + ", ")); + sb.append(String.format("%" + 12 + "s", sjw.jobHistory.getStartTimeSeconds() + ", ")); + sb.append(String.format("%" + 10 + "s", sjw.jobHistory.getEndTimeSeconds())); + sb.append("\n"); + } + return sb.toString(); + } + + public void writeResultsToTsv() { + if (!metricsConfig.isMetricsOn() || + metricsConfig.getMetricsDirectory() == null || + metricsConfig.getMetricsDirectory().isEmpty()) { + LOG.log(Level.INFO, "Metrics is not turned on, not writing results to TSV..."); + return; + } + + final String jobStatsFilePathStr = metricsConfig.getMetricsFilePath("jobStatsFile.tsv"); + assert jobStatsFilePathStr != null; + + LOG.log(Level.INFO, "Writing job stats to " + jobStatsFilePathStr + "..."); + try (final CSVPrinter printer = new CSVPrinter(new FileWriter(jobStatsFilePathStr), + CSVFormat.TDF.withEscape('\\').withQuoteMode(QuoteMode.NONE)) + ) { + printer.printRecord( + "SubmitTimeSecondsOffset", + "TraceJobId", + "Queue", + "EndState", + "StartTimeSecondsOffset", + "EndTimeSecondsOffset", + "Priority" + ); + + for (final SimulatedJobWrapper sjw : getSortedJobHistoryBySubmissionTime()) { + printer.printRecord( + getTimeFromSimulationStart(sjw.jobHistory.getSubmitTimeSeconds()), + sjw.jobHistory.getTraceJobId(), + sjw.jobHistory.getQueue(), + sjw.jobHistory.getEndState(), + getTimeFromSimulationStart(sjw.jobHistory.getStartTimeSeconds()), + getTimeFromSimulationStart(sjw.jobHistory.getEndTimeSeconds()), + sjw.jobHistory.getPriority() + ); + } + } catch (final IOException e) { + e.printStackTrace(); + throw new RuntimeException(e); + } + + LOG.log(Level.INFO, "Done writing job stats!"); + } + + private String getTimeFromSimulationStart(@Nullable final Long seconds) { + if (seconds == null) { + return null; + } + + return Long.toString(seconds - clock.getSimulationStartTime().getEpochSecond()); + } +} diff --git a/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/stats/StatUtils.java b/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/stats/StatUtils.java new file mode 100644 index 00000000000..295663d9303 --- /dev/null +++ b/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/stats/StatUtils.java @@ -0,0 +1,46 @@ +package org.apache.hadoop.yarn.dtss.stats; + +import org.apache.hadoop.classification.InterfaceAudience; + +import java.util.Arrays; + +/** + * Supports computing statistics for cluster and job metrics. + */ +@InterfaceAudience.Private +public class StatUtils { + double[] data; + int size; + + public StatUtils(double[] data) { + this.data = data; + size = data.length; + } + + double getVariance() { + double mean = getMean(); + double temp = 0; + for (double a : data) + temp += (a - mean) * (a - mean); + return temp / size; + } + + double getStdDev() { + return Math.sqrt(getVariance()); + } + + double getMean() { + double sum = 0.0; + for (double a : data) + sum += a; + return sum / size; + } + + public double getMedian() { + Arrays.sort(data); + if (data.length % 2 == 0) { + return (data[(data.length / 2) - 1] + data[data.length / 2]) / 2.0; + } + return data[data.length / 2]; + } +} \ No newline at end of file diff --git a/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/time/Alarm.java b/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/time/Alarm.java new file mode 100644 index 00000000000..3dcfef6c694 --- /dev/null +++ b/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/time/Alarm.java @@ -0,0 +1,68 @@ +package org.apache.hadoop.yarn.dtss.time; + +import org.apache.hadoop.classification.InterfaceAudience; + +import java.time.Instant; +import java.util.UUID; + +/** + * Each {@link Alarm} object is tied to an event in the discrete event simulation. + * An {@link Alarm} is marked by a timestamp and a UUID, where the UUID + * allows unique identification of the event, while the timestamp allows + * the {@link Alarm}s to be ordered in a priority queue in {@link SimulatedClock}. + */ +@InterfaceAudience.Private +public abstract class Alarm implements Comparable { + private final Instant alarmTime; + private final UUID alarmId; + + public Alarm(final UUID alarmId, final Instant alarmTime) { + this.alarmTime = alarmTime; + this.alarmId = alarmId; + } + + public final UUID getAlarmId() { + return alarmId; + } + + public final Instant getInstant() { + return alarmTime; + } + + public abstract void handleAlarm(); + + @Override + public int compareTo(final Alarm that) { + if (!this.alarmTime.equals(that.alarmTime)) { + return this.alarmTime.compareTo(that.alarmTime); + } + + return this.alarmId.compareTo(that.alarmId); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + + if (o == null || getClass() != o.getClass()) { + return false; + } + + final Alarm alarm = (Alarm) o; + + if (alarmTime != null ? !alarmTime.equals(alarm.alarmTime) : alarm.alarmTime != null) { + return false; + } + + return alarmId != null ? alarmId.equals(alarm.alarmId) : alarm.alarmId == null; + } + + @Override + public int hashCode() { + int result = alarmTime != null ? alarmTime.hashCode() : 0; + result = 31 * result + (alarmId != null ? alarmId.hashCode() : 0); + return result; + } +} diff --git a/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/time/ClientAlarm.java b/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/time/ClientAlarm.java new file mode 100644 index 00000000000..ceee0134020 --- /dev/null +++ b/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/time/ClientAlarm.java @@ -0,0 +1,27 @@ +package org.apache.hadoop.yarn.dtss.time; + +import org.apache.hadoop.classification.InterfaceAudience; + +import java.time.Instant; +import java.util.UUID; +import java.util.function.Consumer; + +/** + * An {@link Alarm}, or event, scheduled on the {@link SimulatedClock}. These are not periodic. + */ +@InterfaceAudience.Private +public final class ClientAlarm extends Alarm { + private final Consumer handler; + + public ClientAlarm( + final UUID alarmId, final Instant alarmTime, final Consumer handler + ) { + super(alarmId, alarmTime); + this.handler = handler; + } + + @Override + public void handleAlarm() { + handler.accept(this); + } +} diff --git a/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/time/Clock.java b/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/time/Clock.java new file mode 100644 index 00000000000..b7bdb0d12e0 --- /dev/null +++ b/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/time/Clock.java @@ -0,0 +1,137 @@ +package org.apache.hadoop.yarn.dtss.time; + +import org.apache.hadoop.yarn.dtss.lifecycle.InitializedLifeCycle; +import org.apache.hadoop.yarn.dtss.lifecycle.LifeCycleState; + +import javax.annotation.Nullable; +import java.time.Instant; +import java.util.Optional; +import java.util.UUID; +import java.util.function.Consumer; + +/** + * The clock abstract class. + */ +public abstract class Clock extends InitializedLifeCycle implements org.apache.hadoop.yarn.util.EventClock { + private final MetricsClock metricsClock = new MetricsClock(); + + /** + * Schedules a periodic alarm on the clock, following an initial delay + * @param initialDelaySec initial delay in seconds + * @param taskPeriodicSec the task period in seconds + * @param handler the handler of the alarm event + * @return an ID of the scheduled task + */ + public abstract UUID schedulePeriodicAlarm( + long initialDelaySec, long taskPeriodicSec, Consumer handler); + + /** + * Schedules a periodic alarm on the clock. + * + * @param taskPeriodSec the task period in seconds + * @param handler the handler of the Alarm event + * @return an ID of the scheduled task + */ + public abstract UUID schedulePeriodicAlarm(long taskPeriodSec, Consumer handler); + + /** + * Schedules a periodic alarm on the clock with a runnable. + * + * @param taskPeriodSec the task period in seconds + * @param runnable the handler of the Alarm event + * @return an ID of the scheduled task + */ + public abstract UUID schedulePeriodicAlarm(long taskPeriodSec, Runnable runnable); + + /** + * Unschedules the periodic task provided the schedule ID. + * + * @param scheduleId the ID of the scheduled periodic task + */ + public abstract void unschedulePeriodicAlarm(UUID scheduleId); + + /** + * Schedules an alarm that fires {@code taskTimeOffsetSec} from now. + * + * @param taskTimeOffsetSec the offset from current time in seconds + * @param alarmHandler the handler for the alarm. + */ + public abstract UUID scheduleAlarm(long taskTimeOffsetSec, Consumer alarmHandler); + + /** + * Schedules an alarm that fires at or after {@code alarmTime}. + * + * @param alarmTime the alarm time + * @param alarmHandler the handler for the alarm + */ + public abstract UUID scheduleAbsoluteAlarm(Instant alarmTime, Consumer alarmHandler); + + /** + * Cancels the scheduled alarm. + * @param alarmId the alarm ID + */ + public abstract void cancelAlarm(UUID alarmId); + + /** + * Schedules an alarm that triggers the shutdown of the clock after + * {@code alarmTimeOffsetSec} seconds. + * + * @param alarmTimeOffsetSec the time to shut down the clock + */ + public abstract void scheduleShutdown(long alarmTimeOffsetSec); + + /** + * Schedules an alarm that triggers the shutdown of the clock at the specified time. + * + * @param alarmTime the time to shut down the clock + */ + public abstract void scheduleShutdown(Instant alarmTime); + + /** + * Polls the clock until the next alarm occurs. + * + * @return whether or not there are any more events in the clock + */ + public abstract LifeCycleState pollNextAlarm(); + + /** + * @return the current time + */ + public abstract Instant getInstant(); + + @Override + public long getTime() { + return currentTimeMillis(); + } + + /** + * @return the scheduled shutdown time, if any + */ + @Nullable + public abstract Instant getScheduledShutdown(); + + /** + * @return the current time in milliseconds + */ + public abstract long currentTimeMillis(); + + /** + * @return the stop time of the clock + */ + public abstract Optional getStopTime(); + + public com.codahale.metrics.Clock getMetricsClock() { + return metricsClock; + } + + private class MetricsClock extends com.codahale.metrics.Clock { + private MetricsClock() { + } + + @Override + public long getTick() { + return Clock.this.getInstant().toEpochMilli() * 1000; + } + } +} + diff --git a/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/time/PeriodicClientAlarm.java b/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/time/PeriodicClientAlarm.java new file mode 100644 index 00000000000..5255d2bc195 --- /dev/null +++ b/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/time/PeriodicClientAlarm.java @@ -0,0 +1,40 @@ +package org.apache.hadoop.yarn.dtss.time; + +import org.apache.hadoop.classification.InterfaceAudience; + +import java.time.Instant; +import java.util.UUID; +import java.util.function.Consumer; + +/** + * A periodic alarm that is fired periodically on the same handler by the {@link SimulatedClock}. + */ +@InterfaceAudience.Private +public final class PeriodicClientAlarm extends Alarm { + private final UUID periodicAlarmId; + private final long periodSec; + private final Consumer handler; + + public PeriodicClientAlarm(final UUID periodicAlarmId, + final long periodSec, + final Instant alarmTime, + final Consumer handler) { + super(periodicAlarmId, alarmTime); + this.periodicAlarmId = periodicAlarmId; + this.periodSec = periodSec; + this.handler = handler; + } + + @Override + public void handleAlarm() { + handler.accept(this); + } + + public UUID getPeriodicAlarmId() { + return periodicAlarmId; + } + + public long getPeriodSec() { + return periodSec; + } +} diff --git a/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/time/RuntimeStopAlarm.java b/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/time/RuntimeStopAlarm.java new file mode 100644 index 00000000000..f74e17517f4 --- /dev/null +++ b/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/time/RuntimeStopAlarm.java @@ -0,0 +1,22 @@ +package org.apache.hadoop.yarn.dtss.time; + +import org.apache.hadoop.classification.InterfaceAudience; + +import java.time.Instant; +import java.util.UUID; + +/** + * A special event that signals that the simulated environment should stop. + * Shuts down the experiment when scheduled. + */ +@InterfaceAudience.Private +public final class RuntimeStopAlarm extends Alarm { + public RuntimeStopAlarm(final UUID alarmId, final Instant alarmTime) { + super(alarmId, alarmTime); + } + + @Override + public void handleAlarm() { + // Do nothing + } +} diff --git a/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/time/SimulatedClock.java b/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/time/SimulatedClock.java new file mode 100644 index 00000000000..bbeeae87840 --- /dev/null +++ b/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/time/SimulatedClock.java @@ -0,0 +1,396 @@ +package org.apache.hadoop.yarn.dtss.time; + +import com.google.inject.Inject; +import com.google.inject.Singleton; +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.yarn.dtss.config.parameters.SimulationDurationMinutes; +import org.apache.hadoop.yarn.dtss.config.parameters.SimulationTraceStartTime; +import org.apache.hadoop.yarn.dtss.exceptions.OperationNotSupportedException; +import org.apache.hadoop.yarn.dtss.lifecycle.LifeCycleState; +import org.apache.hadoop.yarn.dtss.random.RandomGenerator; +import org.apache.hadoop.yarn.dtss.random.RandomSeeder; + +import javax.annotation.Nullable; +import java.time.Instant; +import java.util.*; +import java.util.concurrent.TimeUnit; +import java.util.function.Consumer; +import java.util.logging.Level; +import java.util.logging.Logger; + +/** + * A discrete, event-driven clock that polls events, or {@link Alarm}s, + * from a priority queue in discrete time order. + * Allows clients to schedule events that simulate events firing in real time. + * Shuts down when a {@link RuntimeStopAlarm} is encountered, + * or when there are no more events remaining in the clock. + */ +@Singleton +@InterfaceAudience.Private +public final class SimulatedClock extends Clock { + private static final Logger LOG = Logger.getLogger(SimulatedClock.class.getName()); + + private final Map> periodicTasks = new HashMap<>(); + private final PriorityQueue eventQueue = new PriorityQueue<>(); + private final RandomGenerator random; + private final Long simulationDurationMinutes; + + private final Map onClockStart = new HashMap<>(); + + private Instant scheduledShutdown = null; + private Instant simulationStartTime = null; + private Instant currTime; + + @Inject + private SimulatedClock( + final RandomSeeder randomSeeder, + @SimulationDurationMinutes final Long simulationDurationMinutes, + @Nullable @SimulationTraceStartTime final Long simulationTraceStartTime) { + this.random = randomSeeder.newRandomGenerator(); + this.simulationDurationMinutes = simulationDurationMinutes; + if (simulationTraceStartTime != null) { + setSimulationStartTime(Instant.ofEpochSecond(simulationTraceStartTime)); + } + } + + private UUID schedulePeriodicAlarm( + final Optional initialDelaySec, final long taskPeriodSec, final Consumer handler) { + final UUID periodicAlarmId = random.randomUUID(); + assert !periodicTasks.containsKey(periodicAlarmId); + periodicTasks.put(periodicAlarmId, handler); + + // If the clock has not yet started, queue up the task + if (lifeCycleState.compareTo(LifeCycleState.STARTED) < 0) { + onClockStart.put( + periodicAlarmId, + () -> addPeriodicAlarm(initialDelaySec, taskPeriodSec, handler, periodicAlarmId) + ); + return periodicAlarmId; + } + + addPeriodicAlarm(initialDelaySec, taskPeriodSec, handler, periodicAlarmId); + return periodicAlarmId; + } + + private void addPeriodicAlarm(final Optional initialDelaySec, + final long taskPeriodSec, + final Consumer handler, + final UUID periodicAlarmId) { + final Instant firstAlarmTime = initialDelaySec.isPresent() ? + currTime.plusSeconds(initialDelaySec.get()) : currTime.plusSeconds(taskPeriodSec); + + eventQueue.add(new PeriodicClientAlarm( + periodicAlarmId, taskPeriodSec, firstAlarmTime, handler) + ); + } + + /** + * Allows users to queue up events periodically with an initial delay. + * @param initialDelaySec initial delay in seconds + * @param taskPeriodSec the periodicity of the event + * @param handler the handler of the alarm event + * @return the UUID associated with the periodic events + */ + @Override + public UUID schedulePeriodicAlarm( + final long initialDelaySec, final long taskPeriodSec, final Consumer handler) { + return schedulePeriodicAlarm(Optional.of(initialDelaySec), taskPeriodSec, handler); + } + + /** + * Allows users to queue up events periodically without an initial delay. + * @param taskPeriodSec the periodicity of the event + * @param handler the handler of the alarm event + * @return the UUID associated with the periodic events + */ + @Override + public synchronized UUID schedulePeriodicAlarm( + final long taskPeriodSec, + final Consumer handler) { + return schedulePeriodicAlarm(Optional.empty(), taskPeriodSec, handler); + } + + /** + * Allows users to queue up events periodically without an initial delay. + * @param taskPeriodSec the periodicity of the event + * @param runnable the handler of the alarm event + * @return the UUID associated with the periodic events + */ + @Override + public UUID schedulePeriodicAlarm(final long taskPeriodSec, final Runnable runnable) { + return schedulePeriodicAlarm(taskPeriodSec, h -> runnable.run()); + } + + /** + * Cancels periodic tasks/events. + * @param scheduleId the ID of the scheduled periodic task + */ + @Override + public synchronized void unschedulePeriodicAlarm(final UUID scheduleId) { + if (scheduleId == null) { + return; + } + + onClockStart.remove(scheduleId); + periodicTasks.remove(scheduleId); + } + + private void addAlarm(final UUID alarmId, final Consumer alarmHandler, final long taskTimeOffsetSec) { + eventQueue.add(new ClientAlarm(alarmId, currTime.plusSeconds(taskTimeOffsetSec), alarmHandler)); + } + + /** + * Schedules an event on the clock in relative time. + * @param taskTimeOffsetSec the offset from current time in seconds + * @param alarmHandler the handler for the alarm. + * @return The identifier of the event. + */ + @Override + public synchronized UUID scheduleAlarm( + final long taskTimeOffsetSec, final Consumer alarmHandler) { + final UUID alarmId = random.randomUUID(); + if (lifeCycleState.compareTo(LifeCycleState.STARTED) < 0) { + onClockStart.put(alarmId, () -> addAlarm(alarmId, alarmHandler, taskTimeOffsetSec)); + return alarmId; + } + + addAlarm(alarmId, alarmHandler, taskTimeOffsetSec); + return alarmId; + } + + /** + * Schedules an event on the clock in absolute time. + * @param alarmTime the alarm time + * @param alarmHandler the handler for the alarm + * @return the identifier of the event. + */ + @Override + public synchronized UUID scheduleAbsoluteAlarm( + final Instant alarmTime, final Consumer alarmHandler) { + final UUID alarmId = random.randomUUID(); + eventQueue.add(new ClientAlarm(alarmId, alarmTime, alarmHandler)); + return alarmId; + } + + /** + * Cancels a future event + * @param alarmId the alarm ID + */ + @Override + public synchronized void cancelAlarm(final UUID alarmId) { + onClockStart.remove(alarmId); + eventQueue.removeIf(alarm -> alarm.getAlarmId().equals(alarmId)); + } + + /** + * Schedule a shutdown event on the clock. + * @param alarmTimeOffsetSec the time to shut down the clock + */ + @Override + public void scheduleShutdown(final long alarmTimeOffsetSec) { + scheduleShutdown(currTime.plusSeconds(alarmTimeOffsetSec)); + } + + @Override + @Nullable + public Instant getScheduledShutdown() { + return scheduledShutdown; + } + + /** + * Schedule a shutdown event on the clock. + * @param alarmTime the time to shut down the clock + */ + @Override + public void scheduleShutdown(final Instant alarmTime) { + LOG.log(Level.INFO, "Shutdown scheduled at " + alarmTime); + if (scheduledShutdown == null || alarmTime.compareTo(scheduledShutdown) < 0) { + scheduledShutdown = alarmTime; + } + + eventQueue.add(new RuntimeStopAlarm(random.randomUUID(), alarmTime)); + } + + @Override + public long currentTimeMillis() { + return currTime.toEpochMilli(); + } + + public Instant getSimulationStartTime() { + return simulationStartTime; + } + + public void setSimulationStartTime(final Instant instant) { + if (lifeCycleState.compareTo(LifeCycleState.STARTED) >= 0) { + throw new IllegalArgumentException( + "Clock has already started, should not be setting the simulation start time!"); + } + + assert instant != null; + LOG.log(Level.INFO, "Setting simulation start time epoch to " + instant.getEpochSecond()); + simulationStartTime = instant; + currTime = simulationStartTime; + } + + @Override + public Optional getStopTime() { + if (lifeCycleState.isDone()) { + return Optional.of(currTime); + } + + return Optional.empty(); + } + + /** + * Polls for the next event in the priority queue. + * @return The current state of the simulation + */ + @Override + public LifeCycleState pollNextAlarm() { + final Alarm alarm; + synchronized (this) { + if (lifeCycleState.isDone()) { + throw new OperationNotSupportedException( + String.format("Clock cannot handle more alarms after entering %s lifeCycleState!", lifeCycleState) + ); + } + + if (eventQueue.isEmpty()) { + LOG.log(Level.INFO, "Simulation clock has no more events! Shutting down..."); + scheduleShutdown(0); + } + + alarm = eventQueue.poll(); + } + + LifeCycleState newState = handleAlarm(alarm); + LifeCycleState setState = newState; + + // Trigger all other events that occur at the current time. + while (true) { + final boolean condition; + synchronized (this) { + condition = !eventQueue.isEmpty() && eventQueue.peek().getInstant().compareTo(currTime) == 0; + } + + if (!condition) { + break; + } + + newState = handleAlarm(eventQueue.poll()); + + if (newState.isDone()) { + setState = newState; + } + } + + synchronized (this) { + lifeCycleState = lifeCycleState.transition(setState); + return lifeCycleState; + } + } + + @Override + public Instant getInstant() { + return currTime; + } + + /** + * Check and handle the {@link Alarm} based on its type and the current state of the clock. + * If the type is a {@link RuntimeStopAlarm}, the simulation should end. + * If the type is periodic, a future alarm should also be scheduled. + * If the type is not periodic, it should be fire and forget. + * @param alarm The alarm polled + * @return The state of the clock + */ + private LifeCycleState handleAlarm(final Alarm alarm) { + if (alarm instanceof RuntimeStopAlarm) { + LOG.log(Level.INFO, () -> "Simulation clock received a RuntimeStopAlarm, setting lifeCycleState to STOPPED..."); + return LifeCycleState.STOPPED; + } else if (alarm instanceof PeriodicClientAlarm) { + synchronized (this) { + final PeriodicClientAlarm periodicClientAlarm = (PeriodicClientAlarm) alarm; + if (periodicTasks.containsKey(periodicClientAlarm.getPeriodicAlarmId())) { + eventQueue.add(new PeriodicClientAlarm( + periodicClientAlarm.getPeriodicAlarmId(), + periodicClientAlarm.getPeriodSec(), + periodicClientAlarm.getInstant().plusSeconds(periodicClientAlarm.getPeriodSec()), + periodicTasks.get(periodicClientAlarm.getPeriodicAlarmId()))); + } else { + // Ignore the periodic alarm if it is unscheduled already + return lifeCycleState; + } + } + } + + synchronized (this) { + if (alarm.getInstant().compareTo(currTime) < 0) { + LOG.log(Level.WARNING, "Alarm went backward in time! Alarm time: [" + alarm.getInstant() + "]," + + " current time: [" + currTime + "]. Ignoring the alarm!"); + return lifeCycleState; + } + + currTime = currTime.compareTo(alarm.getInstant()) > 0 ? currTime : alarm.getInstant(); + } + + alarm.handleAlarm(); + + return lifeCycleState; + } + + @Override + public synchronized void start() { + assert currTime != null; + assert simulationStartTime != null; + + LOG.log(Level.INFO, String.format("Simulation clock starting at %s!", currTime)); + super.start(); + if (simulationDurationMinutes != null) { + LOG.log(Level.INFO, "Simulation scheduled to run for " + simulationDurationMinutes + " minutes."); + scheduleShutdown(simulationDurationMinutes * 60); + } + + LOG.log(Level.INFO, "Running clock start events..."); + for (final Runnable clockStartEvents : onClockStart.values()) { + clockStartEvents.run(); + } + + onClockStart.clear(); + LOG.log(Level.INFO, "Done running clock start events!"); + } + + @Override + public synchronized void stop() { + LOG.log(Level.INFO, String.format("Simulation clock stopped at %s!", currTime)); + super.stop(); + } + + @Override + public void schedule( + final Runnable runnable, final long delay, final TimeUnit timeUnit) { + final long seconds = TimeUnit.SECONDS.convert(delay, timeUnit); + scheduleAlarm(seconds, alarm -> runnable.run()); + } + + @Override + public void scheduleWithFixedDelay( + final Runnable runnable, final long initialDelay, final long delay, final TimeUnit timeUnit) { + final long initialDelaySeconds = TimeUnit.SECONDS.convert(initialDelay, timeUnit); + final long delaySeconds = TimeUnit.SECONDS.convert(delay, timeUnit); + scheduleAlarm(initialDelaySeconds, alarm -> { + runnable.run(); + schedulePeriodicAlarm(delaySeconds, periodicAlarm -> runnable.run()); + }); + } + + @Override + public boolean isShutdown() { + return lifeCycleState.isDone(); + } + + @Override + public void shutdown() { + stop(); + } +} + diff --git a/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/trace/ListTraceReader.java b/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/trace/ListTraceReader.java new file mode 100644 index 00000000000..1ff43ae8030 --- /dev/null +++ b/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/trace/ListTraceReader.java @@ -0,0 +1,101 @@ +package org.apache.hadoop.yarn.dtss.trace; + +import org.apache.hadoop.thirdparty.com.google.common.collect.Lists; +import com.google.inject.Injector; +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.yarn.dtss.exceptions.EndOfExperimentNotificationException; +import org.apache.hadoop.yarn.dtss.job.SimulatedJob; +import org.apache.hadoop.yarn.dtss.job.SimulatedJobMaster; +import org.apache.hadoop.yarn.dtss.lifecycle.LifeCycleState; +import org.apache.hadoop.yarn.dtss.metrics.job.JobHistoryManager; +import org.apache.hadoop.yarn.dtss.time.Clock; + +import java.time.Instant; +import java.util.*; +import java.util.logging.Logger; + +/** + * A trace reader that reads in a trace into memory as a list. + */ +@InterfaceAudience.Public +@InterfaceStability.Unstable +public abstract class ListTraceReader extends TraceReader { + private static final Logger LOG = Logger.getLogger(ListTraceReader.class.getName()); + private List jobs; + private final Map jobMasters = new HashMap<>(); + + protected ListTraceReader(final Injector injector, + final Clock clock, + final JobHistoryManager jobHistoryManager) { + super(injector, clock, jobHistoryManager); + } + + /** + * Periodically checks the set of all created {@link SimulatedJobMaster}s + * and see if they are all done. If so, trace jobs are done. + * @return Whether or not the trace has completed + */ + @Override + public boolean areTraceJobsDone() { + if (getState().isDone()) { + return true; + } + + if (getState() != LifeCycleState.STARTED) { + return false; + } + + final List toRemove = new ArrayList<>(); + + for (final SimulatedJobMaster jobMaster : jobMasters.keySet()) { + if (jobMaster.getState().isDone()) { + toRemove.add(jobMaster); + } + } + + for (final SimulatedJobMaster jobMaster : toRemove) { + jobMasters.remove(jobMaster); + } + + return jobMasters.isEmpty(); + } + + @Override + public void init() { + jobs = Lists.newArrayList(parseJobs()); + super.init(); + } + + /** + * Parse the jobs + * @return An {@link Iterable} of {@link SimulatedJob}s + */ + protected abstract Iterable parseJobs(); + + @Override + public void start() { + // Mark jobs as read and add them to the set of job masters to check. + // Schedule jobs on the simulated clock based on their submission times. + for (final SimulatedJob job : jobs) { + final SimulatedJobMaster jobMaster = job.createJobMaster(injector); + jobHistoryManager.onJobRead(jobMaster); + jobMasters.put(jobMaster, null); + clock.scheduleAbsoluteAlarm(Instant.ofEpochSecond(job.getSubmitTimeSeconds()), alarm -> { + // Registers the job with the RM + jobMaster.init(); + }); + } + + super.start(); + } + + @Override + public void onStop() { + for (final SimulatedJobMaster jobMaster : jobMasters.keySet()) { + if (!jobMaster.getState().isDone()) { + jobMaster.stop(new EndOfExperimentNotificationException()); + } + } + } +} diff --git a/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/trace/TraceReader.java b/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/trace/TraceReader.java new file mode 100644 index 00000000000..4385f21245e --- /dev/null +++ b/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/trace/TraceReader.java @@ -0,0 +1,58 @@ +package org.apache.hadoop.yarn.dtss.trace; + + +import com.google.inject.Injector; +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.yarn.dtss.lifecycle.LifeCycle; +import org.apache.hadoop.yarn.dtss.metrics.job.JobHistoryManager; +import org.apache.hadoop.yarn.dtss.stats.JobStats; +import org.apache.hadoop.yarn.dtss.time.Clock; + +/** + * An abstraction for trace readers. + * Works with the simulation environment through the function areTraceJobsDone + * to determine whether or not a trace has completed. + */ +@InterfaceAudience.Public +@InterfaceStability.Unstable +public abstract class TraceReader extends LifeCycle { + protected final Clock clock; + protected final Injector injector; + protected final JobHistoryManager jobHistoryManager; + + protected TraceReader(final Injector injector, + final Clock clock, + final JobHistoryManager jobHistoryManager) { + this.injector = injector; + this.clock = clock; + this.jobHistoryManager = jobHistoryManager; + } + + @Override + public void init() { + super.init(); + jobHistoryManager.init(); + } + + @Override + public void start() { + super.start(); + jobHistoryManager.start(); + } + + @Override + public void stop() { + super.stop(); + jobHistoryManager.stop(); + } + + public abstract void onStop(); + + /** + * Works with the simulation environment through the function areTraceJobsDone + * to determine whether or not a trace has completed. + * @return whether or not trace jobs are all completed + */ + public abstract boolean areTraceJobsDone(); +} diff --git a/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/trace/sls/SLSJob.java b/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/trace/sls/SLSJob.java new file mode 100644 index 00000000000..bf2c39aa868 --- /dev/null +++ b/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/trace/sls/SLSJob.java @@ -0,0 +1,231 @@ +package org.apache.hadoop.yarn.dtss.trace.sls; + +import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting; +import com.google.gson.annotations.SerializedName; +import com.google.inject.Injector; +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.yarn.dtss.job.*; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; + +/** + * A simulated job implementation for an SLS job. + * Serialized to and from JSON. + */ +@InterfaceAudience.Public +@InterfaceStability.Unstable +public final class SLSJob extends SimulatedJob { + + public static class Constants { + public static final String JOB_TYPE = "am.type"; + + public static final String JOB_ID = "job.id"; + + public static final String JOB_USER = "job.user"; + + public static final String JOB_SUBMIT_MS = "job.submit.ms"; + + public static final String JOB_START_MS = "job.start.ms"; + + public static final String JOB_END_MS = "job.end.ms"; + + public static final String JOB_QUEUE_NAME = "job.queue.name"; + + public static final String JOB_PRIORITY = "job.priority"; + + public static final String JOB_TASKS = "job.tasks"; + + public static final String MAPREDUCE_JOB_TYPE = "mapreduce"; + + public static final String DEFAULT_JOB_TYPE = MAPREDUCE_JOB_TYPE; + + public static final String DEFAULT_JOB_USER = "default"; + + private Constants() { + } + } + + @VisibleForTesting + public static class Builder { + // Type of AM, optional, the default value is "mapreduce" + @SerializedName(Constants.JOB_TYPE) + private String jobType; + + // The job id used to track the job, optional. + // The default value, an zero-based integer increasing with number of jobs, + // is used if this is not specified or job.count > 1 + @SerializedName(Constants.JOB_ID) + private String jobId; + + // User, optional, the default value is "default" + @SerializedName(Constants.JOB_USER) + private String jobUser; + + @SerializedName(Constants.JOB_SUBMIT_MS) + private Long submitTimeMs; + + @SerializedName(Constants.JOB_START_MS) + private Long startTimeMs; + + @SerializedName(Constants.JOB_END_MS) + private Long endTimeMs; + + // the queue job will be submitted to + @SerializedName(Constants.JOB_QUEUE_NAME) + private String queueName; + + @SerializedName(Constants.JOB_PRIORITY) + private Integer priority = 20; + + @SerializedName(Constants.JOB_TASKS) + private List tasks; + + public SLSJob build() { + return new SLSJob(this); + } + + public Builder setStartTimeMs(final Long startTimeMs) { + this.startTimeMs = startTimeMs; + return this; + } + + public Builder setEndTimeMs(final Long endTimeMs) { + this.endTimeMs = endTimeMs; + return this; + } + + public Builder setPriority(final Integer priority) { + this.priority = priority; + return this; + } + + public Builder setJobId(final String jobId) { + this.jobId = jobId; + return this; + } + + public Builder setJobUser(final String jobUser) { + this.jobUser = jobUser; + return this; + } + + public Builder setJobType(final String jobType) { + this.jobType = jobType; + return this; + } + + public Builder setSubmitTimeMs(final Long submitTimeMs) { + this.submitTimeMs = submitTimeMs; + return this; + } + + public Builder setTasks(final List tasks) { + this.tasks = tasks; + return this; + } + + public Builder setQueueName(final String queueName) { + this.queueName = queueName; + return this; + } + + public static Builder newBuilder() { + return new Builder(); + } + } + + private final String jobType; + private final String jobId; + private final String jobUser; + private final Long submitTimeSeconds; + private final Long startTimeSeconds; + private final Long endTimeSeconds; + private final String queueName; + private final Integer priority; + private final List containers; + + private SLSJob(final Builder builder) { + this.jobType = builder.jobType == null ? Constants.DEFAULT_JOB_TYPE : builder.jobType; + this.jobId = builder.jobId; + this.jobUser = builder.jobUser == null ? Constants.DEFAULT_JOB_USER : builder.jobUser; + if (builder.submitTimeMs == null) { + assert builder.startTimeMs != null; + this.submitTimeSeconds = builder.startTimeMs / 1000; + } else { + this.submitTimeSeconds = builder.submitTimeMs / 1000; + } + + if (builder.startTimeMs != null) { + this.startTimeSeconds = builder.startTimeMs / 1000; + } else { + this.startTimeSeconds = null; + } + + if (builder.endTimeMs != null) { + this.endTimeSeconds = builder.endTimeMs / 1000; + } else { + this.endTimeSeconds = null; + } + + this.queueName = builder.queueName; + this.priority = builder.priority; + this.containers = builder.tasks == null ? Collections.emptyList() : new ArrayList<>(builder.tasks); + } + + @Override + public Long getSubmitTimeSeconds() { + return submitTimeSeconds; + } + + @Override + public Long getStartTimeSeconds() { + return startTimeSeconds; + } + + @Override + public Long getEndTimeSeconds() { + return endTimeSeconds; + } + + @Override + public List getContainers() { + return containers; + } + + @Override + public String getQueueName() { + return queueName; + } + + @Override + public Integer getPriority() { + return priority; + } + + @Override + public String getUser() { + return jobUser; + } + + @Override + public String getTraceJobId() { + return jobId; + } + + @Override + public String getJobName() { + return getTraceJobId(); + } + + @Override + public SimulatedJobMaster createJobMaster(final Injector injector) { + switch (jobType) { + case Constants.MAPREDUCE_JOB_TYPE: + default: + return new SimulatedMRJobMaster(injector, this); + } + } +} diff --git a/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/trace/sls/SLSTask.java b/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/trace/sls/SLSTask.java new file mode 100644 index 00000000000..9d1ca1836d7 --- /dev/null +++ b/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/trace/sls/SLSTask.java @@ -0,0 +1,138 @@ +package org.apache.hadoop.yarn.dtss.trace.sls; + +import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting; +import com.google.gson.annotations.SerializedName; +import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.yarn.dtss.job.SimulatedTask; +import org.apache.hadoop.yarn.dtss.job.constants.TaskConstants; + +/** + * A simulated task implementation for an SLS job. + * Serialized to and from JSON. + */ +public final class SLSTask extends SimulatedTask { + + public static final class Constants { + public static final String CONTAINER_COUNT = "count"; + + public static final String CONTAINER_HOST = "container.host"; + + public static final String CONTAINER_START_MS = "container.start.ms"; + + public static final String CONTAINER_END_MS = "container.end.ms"; + + public static final String CONTAINER_DURATION_MS = "duration.ms"; + + public static final String CONTAINER_PRIORITY = "container.priority"; + + public static final String CONTAINER_TYPE = "container.type"; + + public static final int DEFAULT_CONTAINER_COUNT = 1; + + public static final int DEFAULT_CONTAINER_PRIORITY = 20; + + public static final String UNKNOWN_CONTAINER_TYPE = "unknown"; + + public static final String MAP_CONTAINER_TYPE = "map"; + + public static final String REDUCE_CONTAINER_TYPE = "reduce"; + + public static final String DEFAULT_CONTAINER_TYPE = UNKNOWN_CONTAINER_TYPE; + + private Constants() { } + } + + // TODO: Support this + // number of tasks, optional, the default value is 1 + @SerializedName(Constants.CONTAINER_COUNT) + private Integer count; + + // TODO: Support this + // Host the container asks for + @SerializedName(Constants.CONTAINER_HOST) + private String host; + + // TODO: Support this for arbitrary jobs + // container start time, optional + @SerializedName(Constants.CONTAINER_START_MS) + private Long startMs; + + // TODO: Support this for arbitrary jobs + // container finish time, optional + @SerializedName(Constants.CONTAINER_END_MS) + private Long endMs; + + // duration of the container, optional if start and end time is specified + @SerializedName(Constants.CONTAINER_DURATION_MS) + private Long durationMs; + + // priority of the container, optional, the default value is 20 + @SerializedName(Constants.CONTAINER_PRIORITY) + private Integer priority; + + @SerializedName(Constants.CONTAINER_TYPE) + private String containerType; + + @InterfaceStability.Unstable + @VisibleForTesting + public static SLSTask newTask(final long durationMs) { + final SLSTask t = new SLSTask(); + t.durationMs = durationMs; + return t; + } + + public Integer getCount() { + if (count == null) { + return Constants.DEFAULT_CONTAINER_COUNT; + } + + return count; + } + + public String getHost() { + return host; + } + + @Override + public Long getStartTimeSeconds() { + return startMs; + } + + @Override + public Long getDurationSeconds() { + if (startMs != null && endMs != null) { + return (long) Math.ceil((endMs - startMs) / 1000.0); + } + + return (long) Math.ceil(durationMs / 1000.0); + } + + @Override + public Integer getPriority() { + if (priority == null) { + return Constants.DEFAULT_CONTAINER_PRIORITY; + } + + return priority; + } + + @Override + public String getTaskType() { + return translateContainerType(containerType); + } + + private static String translateContainerType(final String cType) { + if (cType == null) { + return TaskConstants.MAP_TASK_TYPE; + } + + switch (cType) { + case Constants.REDUCE_CONTAINER_TYPE: + return TaskConstants.REDUCE_TASK_TYPE; + case Constants.MAP_CONTAINER_TYPE: + return TaskConstants.MAP_TASK_TYPE; + default: + return TaskConstants.UNKNOWN_TASK_TYPE; + } + } +} diff --git a/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/trace/sls/SLSTraceReader.java b/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/trace/sls/SLSTraceReader.java new file mode 100644 index 00000000000..ff1073d9fef --- /dev/null +++ b/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/trace/sls/SLSTraceReader.java @@ -0,0 +1,67 @@ +package org.apache.hadoop.yarn.dtss.trace.sls; + +import com.google.gson.Gson; +import com.google.gson.GsonBuilder; +import com.google.gson.stream.JsonReader; +import com.google.gson.stream.JsonToken; +import com.google.inject.Inject; +import com.google.inject.Injector; +import com.google.inject.Singleton; +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.yarn.dtss.metrics.job.JobHistoryManager; +import org.apache.hadoop.yarn.dtss.trace.sls.config.SLSTraceFilePath; +import org.apache.hadoop.yarn.dtss.job.SimulatedJob; +import org.apache.hadoop.yarn.dtss.time.Clock; +import org.apache.hadoop.yarn.dtss.trace.ListTraceReader; + +import java.io.FileReader; +import java.util.ArrayList; +import java.util.List; +import java.util.logging.Level; +import java.util.logging.Logger; + +/** + * A trace reader that reads in an SLS trace into memory. + */ +@InterfaceAudience.Public +@InterfaceStability.Unstable +@Singleton +public final class SLSTraceReader extends ListTraceReader { + private static final Logger LOG = Logger.getLogger(SLSTraceReader.class.getName()); + + public static final String TRACE_TYPE = "SLS"; + + private final String traceFilePath; + private final Gson gson = new GsonBuilder().create(); + + @Inject + private SLSTraceReader( + @SLSTraceFilePath final String traceFilePath, + final Clock clock, + final Injector injector, + final JobHistoryManager jobHistoryManager) { + super(injector, clock, jobHistoryManager); + this.traceFilePath = traceFilePath; + } + + @Override + protected Iterable parseJobs() { + final List jobs = new ArrayList<>(); + try { + final JsonReader jsonReader = new JsonReader(new FileReader(traceFilePath)); + jsonReader.setLenient(true); + while (jsonReader.peek() != JsonToken.END_DOCUMENT) { + final SLSJob.Builder slsJobBuilder = gson.fromJson(jsonReader, SLSJob.Builder.class); + final SLSJob slsJob = slsJobBuilder.build(); + LOG.log(Level.INFO, slsJob.getTraceJobId()); + jobs.add(slsJob); + } + } catch (final Exception e) { + e.printStackTrace(); + throw new RuntimeException(e); + } + + return jobs; + } +} diff --git a/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/trace/sls/config/SLSTraceFilePath.java b/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/trace/sls/config/SLSTraceFilePath.java new file mode 100644 index 00000000000..bafcded2744 --- /dev/null +++ b/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/trace/sls/config/SLSTraceFilePath.java @@ -0,0 +1,19 @@ +package org.apache.hadoop.yarn.dtss.trace.sls.config; + + +import com.google.inject.BindingAnnotation; + +import java.lang.annotation.Retention; +import java.lang.annotation.Target; + +import static java.lang.annotation.ElementType.*; +import static java.lang.annotation.RetentionPolicy.RUNTIME; + +/** + * Annotation for the file path to the SLS trace. + */ +@BindingAnnotation +@Target({FIELD, PARAMETER, METHOD}) +@Retention(RUNTIME) +public @interface SLSTraceFilePath { +} diff --git a/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/trace/sls/config/SLSTraceModule.java b/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/trace/sls/config/SLSTraceModule.java new file mode 100644 index 00000000000..d848dfcf162 --- /dev/null +++ b/hadoop-tools/hadoop-dtss/src/main/java/org/apache/hadoop/yarn/dtss/trace/sls/config/SLSTraceModule.java @@ -0,0 +1,20 @@ +package org.apache.hadoop.yarn.dtss.trace.sls.config; + +import com.google.inject.AbstractModule; +import org.apache.hadoop.yarn.dtss.trace.TraceReader; +import org.apache.hadoop.yarn.dtss.trace.sls.SLSTraceReader; + +/** + * Configurations associated with using an SLS trace. + */ +public final class SLSTraceModule extends AbstractModule { + public static final String TRACE_TYPE = SLSTraceReader.TRACE_TYPE; + + private final String traceFilePath = null; + + @Override + protected void configure() { + bind(TraceReader.class).to(SLSTraceReader.class); + bind(String.class).annotatedWith(SLSTraceFilePath.class).toInstance(traceFilePath); + } +} diff --git a/hadoop-tools/hadoop-dtss/src/main/sample/config/cluster/topology-sample-large.json b/hadoop-tools/hadoop-dtss/src/main/sample/config/cluster/topology-sample-large.json new file mode 100644 index 00000000000..018459baafd --- /dev/null +++ b/hadoop-tools/hadoop-dtss/src/main/sample/config/cluster/topology-sample-large.json @@ -0,0 +1,5 @@ +{ + "containerVCores": 1, + "containerMB": 1024, + "rackNodeContainers":[[1000, 1000, 1000, 1000, 1000, 1000, 1000, 1000, 1000, 1000, 1000, 1000]] +} \ No newline at end of file diff --git a/hadoop-tools/hadoop-dtss/src/main/sample/config/cluster/topology-sample.json b/hadoop-tools/hadoop-dtss/src/main/sample/config/cluster/topology-sample.json new file mode 100644 index 00000000000..2b65a3f4688 --- /dev/null +++ b/hadoop-tools/hadoop-dtss/src/main/sample/config/cluster/topology-sample.json @@ -0,0 +1,5 @@ +{ + "containerVCores": 1, + "containerMB": 1024, + "rackNodeContainers":[[100, 100, 100, 100, 100]] +} \ No newline at end of file diff --git a/hadoop-tools/hadoop-dtss/src/main/sample/config/queue-settings/default.json b/hadoop-tools/hadoop-dtss/src/main/sample/config/queue-settings/default.json new file mode 100644 index 00000000000..db0d046831e --- /dev/null +++ b/hadoop-tools/hadoop-dtss/src/main/sample/config/queue-settings/default.json @@ -0,0 +1,9 @@ +{ + "user-limit-factor": 1, + "state": "RUNNING", + "acl_submit_applications": "*", + "acl_administer_queue": "*", + "acl_application_max_priority": "*", + "maximum-application-lifetime": -1, + "default-application-lifetime": -1 +} \ No newline at end of file diff --git a/hadoop-tools/hadoop-dtss/src/main/sample/config/queue-settings/fifo-ordering-policy.json b/hadoop-tools/hadoop-dtss/src/main/sample/config/queue-settings/fifo-ordering-policy.json new file mode 100644 index 00000000000..22f64901227 --- /dev/null +++ b/hadoop-tools/hadoop-dtss/src/main/sample/config/queue-settings/fifo-ordering-policy.json @@ -0,0 +1,3 @@ +{ + "ordering-policy": "fifo" +} \ No newline at end of file diff --git a/hadoop-tools/hadoop-dtss/src/main/sample/config/sim-sample.json b/hadoop-tools/hadoop-dtss/src/main/sample/config/sim-sample.json new file mode 100644 index 00000000000..dcf493240d0 --- /dev/null +++ b/hadoop-tools/hadoop-dtss/src/main/sample/config/sim-sample.json @@ -0,0 +1,12 @@ +{ + "clusterTopologyFilePath": "", + "traceConfig": { + "traceType": "SLS", + "traceFilePath": "" + }, + "isMetricsOn": true, + "utilityEquation": "dl", + "metricsOutputDirectory": "", + "randomSeed": 5, + "simulationDurationMinutes": 10 +} diff --git a/hadoop-tools/hadoop-dtss/src/main/sample/config/yarn/capacity-scheduler.xml b/hadoop-tools/hadoop-dtss/src/main/sample/config/yarn/capacity-scheduler.xml new file mode 100644 index 00000000000..61be96ae6d4 --- /dev/null +++ b/hadoop-tools/hadoop-dtss/src/main/sample/config/yarn/capacity-scheduler.xml @@ -0,0 +1,60 @@ + + + + + + + + yarn.scheduler.capacity.root.queues + sls_queue_1,sls_queue_2,sls_queue_3 + The queues at the this level (root is the root queue). + + + + + yarn.scheduler.capacity.root.sls_queue_1.capacity + 25 + + + + yarn.scheduler.capacity.root.sls_queue_1.maximum-capacity + 100 + + + + yarn.scheduler.capacity.root.sls_queue_2.capacity + 25 + + + + yarn.scheduler.capacity.root.sls_queue_2.maximum-capacity + 100 + + + + yarn.scheduler.capacity.root.sls_queue_3.capacity + 50 + + + + yarn.scheduler.capacity.root.sls_queue_3.maximum-capacity + 100 + + diff --git a/hadoop-tools/hadoop-dtss/src/main/sample/config/yarn/core-site.xml b/hadoop-tools/hadoop-dtss/src/main/sample/config/yarn/core-site.xml new file mode 100644 index 00000000000..f0d3085ef85 --- /dev/null +++ b/hadoop-tools/hadoop-dtss/src/main/sample/config/yarn/core-site.xml @@ -0,0 +1,25 @@ + + + + + + + + + hadoop.security.token.service.use_ip + false + + + diff --git a/hadoop-tools/hadoop-dtss/src/main/sample/config/yarn/fair-scheduler.xml b/hadoop-tools/hadoop-dtss/src/main/sample/config/yarn/fair-scheduler.xml new file mode 100644 index 00000000000..41b1618952f --- /dev/null +++ b/hadoop-tools/hadoop-dtss/src/main/sample/config/yarn/fair-scheduler.xml @@ -0,0 +1,42 @@ + + + + + + + + 1024 mb, 1 vcores + fair + 0.25 + 2 + + + 1024 mb, 1 vcores + fair + 0.25 + 2 + + + 1024 mb, 1 vcores + 0.5 + fair + 2 + + diff --git a/hadoop-tools/hadoop-dtss/src/main/sample/config/yarn/yarn-site.xml b/hadoop-tools/hadoop-dtss/src/main/sample/config/yarn/yarn-site.xml new file mode 100644 index 00000000000..74f5ee5a22e --- /dev/null +++ b/hadoop-tools/hadoop-dtss/src/main/sample/config/yarn/yarn-site.xml @@ -0,0 +1,39 @@ + + + + + + + yarn.resourcemanager.scheduler.class + + org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler + + + + + yarn.resourcemanager.reservation-system.enable + false + + + + + Defines maximum application priority in a cluster. + If an application is submitted with a priority higher than this value, it will be + reset to this maximum value. + + yarn.cluster.max-application-priority + 2147483647 + + \ No newline at end of file diff --git a/hadoop-tools/hadoop-dtss/src/main/sample/traces/sls/sls-sample.json b/hadoop-tools/hadoop-dtss/src/main/sample/traces/sls/sls-sample.json new file mode 100644 index 00000000000..e29e3c0338e --- /dev/null +++ b/hadoop-tools/hadoop-dtss/src/main/sample/traces/sls/sls-sample.json @@ -0,0 +1,90 @@ +{ + "am.type" : "mapreduce", // type of AM, optional, the default value is "mapreduce" + "job.submit.ms": 0, + "job.start.ms" : 0, // job start time + "job.end.ms" : 95375, // job finish time, optional, the default value is 0 + "job.queue.name" : "sls_queue_1", // the queue job will be submitted to + "job.id" : "job_1", // the job id used to track the job, optional. The default value, an zero-based integer increasing with number of jobs, is used if this is not specified or job.count > 1 + "job.user" : "default", // user, optional, the default value is "default" + "job.count" : 1, // number of jobs, optional, the default value is 1 + "job.tasks" : [ { + "count": 1, // number of tasks, optional, the default value is 1 + "container.host" : "/default-rack/node1", // host the container asks for + "container.start.ms" : 6664, // container start time, optional + "container.end.ms" : 23707, // container finish time, optional + "duration.ms": 50000, // duration of the container, optional if start and end time is specified + "container.priority" : 20, // priority of the container, optional, the default value is 20 + "container.type" : "map" // type of the container, could be "map" or "reduce", optional, the default value is "map" + }, { + "container.host" : "/default-rack/node3", + "container.start.ms" : 6665, + "container.end.ms" : 21593, + "container.priority" : 20, + "container.type" : "reduce" + }, { + "container.host" : "/default-rack/node2", + "container.start.ms" : 68770, + "container.end.ms" : 86613, + "container.priority" : 20, + "container.type" : "reduce" + } ] +} +{ + "am.type" : "mapreduce", + "job.start.ms" : 105204, // If all dependency is polling, then this is optional + "job.end.ms" : 197256, + "job.queue.name" : "sls_queue_2", + "job.id" : "job_2", + "job.user" : "default", + "job.tasks" : [ { + "container.host" : "/default-rack/node1", + "container.start.ms" : 111822, + "container.end.ms" : 133985, + "container.priority" : 20, + "container.type" : "map" + }, { + "container.host" : "/default-rack/node2", + "container.start.ms" : 111788, + "container.end.ms" : 131377, + "container.priority" : 20, + "container.type" : "map" + } ] +} +{ + "am.type" : "mapreduce", + "job.start.ms" : 60432, + "job.end.ms" : 147226, + "job.queue.name" : "sls_queue_2", + "job.id" : "job_3", + "job.user" : "default", + "job.tasks" : [ { + "container.host" : "/default-rack/node1", + "duration.ms": 50000, + "container.priority" : 20, + "container.type" : "map" + }, { + "container.host" : "/default-rack/node2", + "duration.ms": 50000, + "container.priority" : 20, + "container.type" : "map" + } ] +} +{ + "am.type" : "mapreduce", + "job.start.ms" : 85219, + "job.end.ms" : 147226, + "job.queue.name" : "sls_queue_2", + "job.id" : "job_4", + "job.user" : "default", + "job.tasks" : [ { + "container.host" : "/default-rack/node1", + "duration.ms": 50000, + "container.priority" : 20, + "container.type" : "map" + }, { + "container.host" : "/default-rack/node2", + "duration.ms": 50000, + "container.priority" : 20, + "container.type" : "map" + } ] +} \ No newline at end of file diff --git a/hadoop-tools/hadoop-dtss/src/test/java/org/apache/hadoop/yarn/dtss/BaseDTSSRunnerTest.java b/hadoop-tools/hadoop-dtss/src/test/java/org/apache/hadoop/yarn/dtss/BaseDTSSRunnerTest.java new file mode 100644 index 00000000000..eebfe4d9296 --- /dev/null +++ b/hadoop-tools/hadoop-dtss/src/test/java/org/apache/hadoop/yarn/dtss/BaseDTSSRunnerTest.java @@ -0,0 +1,292 @@ +package org.apache.hadoop.yarn.dtss; + +import net.jcip.annotations.NotThreadSafe; +import org.apache.commons.csv.CSVRecord; +import org.apache.commons.math3.util.Precision; +import org.apache.hadoop.yarn.dtss.cluster.ClusterTopology; +import org.apache.hadoop.yarn.dtss.cluster.ClusterTopologyProperties; +import org.apache.hadoop.yarn.dtss.config.SimulatorModule; +import org.apache.hadoop.yarn.dtss.job.SimulatedTask; +import org.apache.hadoop.yarn.dtss.metrics.MetricsManager; +import org.apache.hadoop.yarn.dtss.trace.TestTraceModule; +import org.apache.hadoop.yarn.dtss.trace.results.*; +import org.apache.hadoop.yarn.dtss.trace.sls.SLSJob; +import org.junit.Assert; + +import java.io.File; +import java.io.IOException; +import java.util.*; + +@NotThreadSafe +@SuppressWarnings("VisibilityModifier") +public abstract class BaseDTSSRunnerTest { + private static final int RANDOM_SEED = 5; + protected static final int DEFAULT_RUNTIME_MINUTES = 30; + protected static final int DEFAULT_NUM_RACKS = 2; + protected static final int DEFAULT_NUM_MACHINES_ON_RACK = 2; + protected static final int DEFAULT_NUM_CONTAINERS_IN_MACHINE = 10; + protected static final int DEFAULT_CONTAINER_MB = 1024; + protected static final int DEFAULT_VCORES = 1; + + private static final String[] metricsList = new String[] { + MetricsManager.CLUSTER_ALLOCATED_VCORES, + MetricsManager.CLUSTER_AVAILABLE_VCORES, + MetricsManager.RUNNING_APPLICATIONS, + MetricsManager.RUNNING_CONTAINERS, + "queue.allocated.vcores", + "queue.available.vcores", + }; + + private static final MetricInvariantCheck GEQ_ZERO_CHECKER = new MetricInvariantCheck<>( + (t, v) -> Precision.compareTo(v, 0D, 1) >= 0, + "GreaterOrEqualToZero" + ); + + protected SimulatorModule simulatorModule; + protected Runner.RunnerModule runnerModule; + protected String runId; + protected File tempDir; + + public void setup( + final int simulationRuntimeMinutes, + final ClusterTopologyProperties clusterTopologyProperties, + final TestTraceModule testTraceModule + ) throws IOException { + setup(simulationRuntimeMinutes, clusterTopologyProperties, testTraceModule, false); + } + + public void setup( + final int simulationRuntimeMinutes, + final ClusterTopologyProperties clusterTopologyProperties, + final TestTraceModule testTraceModule, + final boolean isUnmanaged + ) throws IOException { + final File dtssDir = new File("target", "dtssTestOutput"); + runId = UUID.randomUUID().toString(); + tempDir = new File(dtssDir, runId); + if (!tempDir.mkdirs()) { + throw new RuntimeException("Failed mkdirs for directory " + tempDir.getAbsolutePath()); + } + + final File topologyPath = new File(tempDir, "topology.json"); + ClusterTopology.writeClusterTopologyFile(clusterTopologyProperties, topologyPath.getAbsolutePath()); + simulatorModule = SimulatorModule.newModule( + RANDOM_SEED, + simulationRuntimeMinutes, + topologyPath.getAbsolutePath(), + dtssDir.getAbsolutePath(), + testTraceModule, + isUnmanaged, + Arrays.asList(metricsList) + ); + + runnerModule = Runner.RunnerModule.newModule("EMPTY", runId); + } + + protected ClusterTopologyProperties createClusterTopologyProperties( + final int numRacks, + final int numMachinesOnRack, + final int numContainersInMachine, + final int containerMB, + final int containerVCores + ) { + final List> racksArray = new ArrayList<>(); + for (int r = 0; r < numRacks; r++) { + final List machinesArray = new ArrayList<>(); + for (int m = 0; m < numMachinesOnRack; m++) { + machinesArray.add(numContainersInMachine); + } + + racksArray.add(machinesArray); + } + + return new ClusterTopologyProperties( + racksArray, containerMB, containerVCores + ); + } + + protected ClusterTopologyProperties createDefaultClusterTopologyProperties() { + return createClusterTopologyProperties( + DEFAULT_NUM_RACKS, + DEFAULT_NUM_MACHINES_ON_RACK, + DEFAULT_NUM_CONTAINERS_IN_MACHINE, + DEFAULT_CONTAINER_MB, + DEFAULT_VCORES + ); + } + + private static void addDefaultQueueMetricInvariants(final QueueMetric metric, + final List> checks) { + // All valid cluster metrics should be >= 0 + checks.add(GEQ_ZERO_CHECKER); + } + + private static void checkMetricInvariants( + final DetailedMetricReader reader, final List> checks) { + while (reader.hasNextRecord()) { + final CSVRecord record = reader.readRecord(); + final long time = Long.parseLong(record.get(0)); + final double vCores = Double.parseDouble(record.get(1)); + for (final MetricInvariantCheck invCheck : checks) { + Assert.assertTrue(invCheck.getErrorMessage(time, vCores), invCheck.evaluate(time, vCores)); + } + } + } + protected void checkQueueMetricInvariants(final String queueName, final QueueMetric metric) throws IOException { + checkQueueMetricInvariants(queueName, metric, new ArrayList<>()); + } + + protected void checkQueueMetricInvariants(final String queueName, + final QueueMetric metric, + final List> invariantChecks) + throws IOException { + final List> allChecks = new ArrayList<>(); + addDefaultQueueMetricInvariants(metric, allChecks); + allChecks.addAll(invariantChecks); + + final DetailedMetricReader reader = new DetailedMetricReader( + getMetricsFile(QueueMetric.getQueueMetricFileName(queueName, metric)).getAbsolutePath(), + ',', + true + ); + + checkMetricInvariants(reader, allChecks); + } + + private static void addDefaultClusterMetricInvariants(final ClusterMetric metric, + final List> checks) { + // All valid cluster metrics should be >= 0 + checks.add(GEQ_ZERO_CHECKER); + } + + protected void checkClusterMetricInvariants(final ClusterMetric metric) throws IOException { + checkClusterMetricInvariants(metric, new ArrayList<>()); + } + + protected void checkClusterMetricInvariants( + final ClusterMetric metric, + final List> invariantChecks) throws IOException { + final List> allChecks = new ArrayList<>(); + addDefaultClusterMetricInvariants(metric, allChecks); + allChecks.addAll(invariantChecks); + + final DetailedMetricReader reader = new DetailedMetricReader( + getMetricsFile(ClusterMetric.getClusterMetricFileName(metric)).getAbsolutePath(), + ',', + true + ); + + checkMetricInvariants(reader, allChecks); + } + + protected File getMetricsDir() { + return new File(tempDir.getAbsolutePath(), "detailed-metrics"); + } + + protected File getMetricsFile(final String fileName) { + final File metricsFile = new File(getMetricsDir(), fileName); + Assert.assertTrue(metricsFile.exists()); + return metricsFile; + } + + protected List readJobsFromJobsStatsFile() throws IOException { + final List entries = new ArrayList<>(); + final File jobStatsFile = getJobStatsFile(); + final JobStatsFileReader reader = new JobStatsFileReader(jobStatsFile.getAbsolutePath()); + while (reader.hasNextRecord()) { + entries.add(reader.readRecord()); + } + + return entries; + } + + protected File getJobStatsFile() { + final File jobStatsFile = new File(tempDir.getAbsolutePath(), "jobStatsFile.tsv"); + Assert.assertTrue(jobStatsFile.exists()); + return jobStatsFile; + } + + protected void checkJobInvariants( + final List experimentJobs, + final Map referenceJobs) { + for (final JobStatsFileReader.JobStatsEntry job : experimentJobs) { + Assert.assertTrue(referenceJobs.containsKey(job.getTraceJobId())); + final SLSJob referenceJob = referenceJobs.get(job.getTraceJobId()); + Assert.assertEquals(referenceJob.getQueueName(), job.getQueueName()); + switch(job.getExperimentJobEndState()) { + case READ: + case NOT_SUBMITTED: + Assert.assertNull(job.getSubmitTimeSecondsOffset()); + Assert.assertNull(job.getStartTimeSecondsOffset()); + Assert.assertNull(job.getEndTimeSecondsOffset()); + break; + case SUBMITTED: + validateSubmissionTime(job, referenceJob); + Assert.assertNull(job.getStartTimeSecondsOffset()); + Assert.assertNull(job.getEndTimeSecondsOffset()); + break; + case STARTED: + validateSubmissionTime(job, referenceJob); + validateStartTime(job, referenceJob); + Assert.assertNull(job.getEndTimeSecondsOffset()); + break; + case COMPLETED: + validateSubmissionTime(job, referenceJob); + validateStartTime(job, referenceJob); + validateCompletionTime(job, referenceJob); + break; + default: + throw new RuntimeException("Unexpected state for experiments!"); + } + } + } + + private static void validateSubmissionTime( + final JobStatsFileReader.JobStatsEntry job, final SLSJob referenceJob + ) { + Assert.assertNotNull(job.getSubmitTimeSecondsOffset()); + Assert.assertTrue(job.getSubmitTimeSecondsOffset() >= 0); + Assert.assertTrue( + job.getSubmitTimeSecondsOffset() >= referenceJob.getSubmitTimeSeconds() + ); + } + + private static void validateStartTime( + final JobStatsFileReader.JobStatsEntry job, final SLSJob referenceJob + ) { + Assert.assertNotNull(job.getStartTimeSecondsOffset()); + Assert.assertTrue(job.getStartTimeSecondsOffset() >= job.getSubmitTimeSecondsOffset()); + Assert.assertTrue( + job.getStartTimeSecondsOffset() >= referenceJob.getSubmitTimeSeconds() + ); + } + + private static void validateCompletionTime( + final JobStatsFileReader.JobStatsEntry job, final SLSJob referenceJob + ) { + Assert.assertNotNull(job.getEndTimeSecondsOffset()); + Assert.assertTrue(job.getEndTimeSecondsOffset() >= job.getSubmitTimeSecondsOffset()); + Assert.assertTrue(job.getEndTimeSecondsOffset() >= job.getStartTimeSecondsOffset()); + Assert.assertTrue( + job.getEndTimeSecondsOffset() >= referenceJob.getSubmitTimeSeconds() + ); + + long earliestEndTimePossible = referenceJob.getSubmitTimeSeconds(); + for (final SimulatedTask task : referenceJob.getContainers()) { + earliestEndTimePossible = Math.max( + referenceJob.getSubmitTimeSeconds() + task.getDurationSeconds(), earliestEndTimePossible + ); + } + + Assert.assertTrue(job.getEndTimeSecondsOffset() >= earliestEndTimePossible); + } + + protected static Map constructJobMap(final List jobList) { + final Map jobMap = new HashMap<>(); + for (final SLSJob job : jobList) { + jobMap.put(job.getTraceJobId(), job); + } + + return jobMap; + } +} diff --git a/hadoop-tools/hadoop-dtss/src/test/java/org/apache/hadoop/yarn/dtss/TestDTSSRunner.java b/hadoop-tools/hadoop-dtss/src/test/java/org/apache/hadoop/yarn/dtss/TestDTSSRunner.java new file mode 100644 index 00000000000..df76016a0b7 --- /dev/null +++ b/hadoop-tools/hadoop-dtss/src/test/java/org/apache/hadoop/yarn/dtss/TestDTSSRunner.java @@ -0,0 +1,363 @@ +package org.apache.hadoop.yarn.dtss; + +import net.jcip.annotations.NotThreadSafe; +import org.apache.commons.math3.util.Precision; +import org.apache.hadoop.yarn.dtss.cluster.ClusterTopologyProperties; +import org.apache.hadoop.yarn.dtss.job.ExperimentJobEndState; +import org.apache.hadoop.yarn.dtss.trace.TestTraceModule; +import org.apache.hadoop.yarn.dtss.trace.TestTraceProvider; +import org.apache.hadoop.yarn.dtss.trace.results.ClusterMetric; +import org.apache.hadoop.yarn.dtss.trace.results.JobStatsFileReader; +import org.apache.hadoop.yarn.dtss.trace.results.MetricInvariantCheck; +import org.apache.hadoop.yarn.dtss.trace.results.QueueMetric; +import org.apache.hadoop.yarn.dtss.trace.sls.SLSJob; +import org.apache.hadoop.yarn.dtss.trace.sls.SLSTask; +import org.junit.Assert; +import org.junit.Test; + +import java.io.IOException; +import java.util.*; +import java.util.concurrent.atomic.AtomicBoolean; + +/** + * End-to-end tests for simple DTSSRunner. + */ +@NotThreadSafe +public class TestDTSSRunner extends BaseDTSSRunnerTest { + private static final String QUEUE_1_NAME = "sls_queue_1"; + private static final String QUEUE_2_NAME = "sls_queue_2"; + private static final String QUEUE_3_NAME = "sls_queue_3"; + private static final String[] QUEUES = new String[]{QUEUE_1_NAME, QUEUE_2_NAME, QUEUE_3_NAME}; + + @Test + public void testNoResourcesAllocated() throws IOException { + final ClusterTopologyProperties properties = createDefaultClusterTopologyProperties(); + setup(DEFAULT_RUNTIME_MINUTES, properties, new TestTraceModule(new TestTraceProvider(new ArrayList<>()))); + + Runner.runSimulator(simulatorModule, runnerModule); + checkClusterMetricInvariants( + ClusterMetric.CLUSTER_ALLOCATED_VCORES, + Collections.singletonList( + new MetricInvariantCheck<>( + (t, v) -> Precision.equals(v, 0D), + "NoAllocation" + ) + ) + ); + + final List jobs = readJobsFromJobsStatsFile(); + Assert.assertEquals(0, jobs.size()); + } + + @Test + public void testRunOneJobUnmanagedAM() throws IOException { + final List jobList = Collections.singletonList( + SLSJob.Builder.newBuilder() + .setTasks(Collections.singletonList(SLSTask.newTask(60 * 1000))) + .setSubmitTimeMs(0L) + .setPriority(30) + .setJobId("Job_1") + .setQueueName(QUEUE_1_NAME) + .build() + ); + + final ClusterTopologyProperties properties = createDefaultClusterTopologyProperties(); + setup(DEFAULT_RUNTIME_MINUTES, properties, new TestTraceModule(new TestTraceProvider(jobList)), true); + + Runner.runSimulator(simulatorModule, runnerModule); + + final AtomicBoolean clusterSeenOneAllocated = new AtomicBoolean(false); + checkClusterMetricInvariants( + ClusterMetric.CLUSTER_ALLOCATED_VCORES, + Collections.singletonList( + new MetricInvariantCheck<>( + (t, v) -> { + if (Precision.equals(v, 1D)) { + clusterSeenOneAllocated.set(true); + } + + return Precision.compareTo(v, 1D, 1) <= 0; + }, + "ClusterSingleAllocation" + ) + ) + ); + + Assert.assertTrue("Must have seen one container allocated.", clusterSeenOneAllocated.get()); + + final AtomicBoolean queueSeenOneAllocated = new AtomicBoolean(false); + checkQueueMetricInvariants( + QUEUE_1_NAME, + QueueMetric.QUEUE_ALLOCATED_VCORES, + Collections.singletonList( + new MetricInvariantCheck<>( + (t, v) -> { + if (Precision.equals(v, 1D)) { + queueSeenOneAllocated.set(true); + } + + return Precision.compareTo(v, 1D, 1) <= 0; + }, + "QueueSingleAllocation" + ) + ) + ); + + Assert.assertTrue("Must have seen one container allocated.", queueSeenOneAllocated.get()); + final List jobs = readJobsFromJobsStatsFile(); + Assert.assertEquals(1, jobs.size()); + final JobStatsFileReader.JobStatsEntry oneJob = jobs.get(0); + Assert.assertEquals(ExperimentJobEndState.COMPLETED, oneJob.getExperimentJobEndState()); + Assert.assertTrue(oneJob.getEndTimeSecondsOffset() >= oneJob.getStartTimeSecondsOffset()); + Assert.assertTrue(oneJob.getStartTimeSecondsOffset() >= oneJob.getSubmitTimeSecondsOffset()); + Assert.assertEquals(QUEUE_1_NAME, oneJob.getQueueName()); + + final Map jobMap = constructJobMap(jobList); + checkJobInvariants(jobs, jobMap); + } + + @Test + public void testLargeJobCannotFinish() throws IOException { + // SLS Queue 1 should have at most 10 containers, + // based on the setting of capacity-scheduler.xml + final List largeJobTaskList = new ArrayList<>(); + final long taskRuntimeSeconds = 300; + final long taskRuntimeMs = taskRuntimeSeconds * 1000; + + for (int i = 0; i < 100; i++) { + largeJobTaskList.add(SLSTask.newTask(taskRuntimeMs)); + } + + final List jobList = Collections.singletonList( + SLSJob.Builder.newBuilder() + .setTasks(largeJobTaskList) + .setSubmitTimeMs(0L) + .setPriority(30) + .setJobId("Job_1") + .setQueueName(QUEUE_1_NAME) + .build() + ); + + final ClusterTopologyProperties properties = createDefaultClusterTopologyProperties(); + setup(DEFAULT_RUNTIME_MINUTES, properties, new TestTraceModule(new TestTraceProvider(jobList))); + + Runner.runSimulator(simulatorModule, runnerModule); + + checkClusterMetricInvariants(ClusterMetric.CLUSTER_ALLOCATED_VCORES); + + final List jobs = readJobsFromJobsStatsFile(); + Assert.assertEquals(1, jobs.size()); + final JobStatsFileReader.JobStatsEntry oneJob = jobs.get(0); + Assert.assertTrue(oneJob.getExperimentJobEndState().equals(ExperimentJobEndState.SUBMITTED) || + oneJob.getExperimentJobEndState().equals(ExperimentJobEndState.STARTED)); + + final Map jobMap = constructJobMap(jobList); + checkJobInvariants(jobs, jobMap); + } + + @Test + public void testLargeJobGetsDelayed() throws IOException { + // SLS Queue 1 should have at most 10 containers, + // based on the setting of capacity-scheduler.xml + final List largeJobTaskList = new ArrayList<>(); + final long taskRuntimeSeconds = 300; + final long taskRuntimeMs = taskRuntimeSeconds * 1000; + + for (int i = 0; i < 30; i++) { + largeJobTaskList.add(SLSTask.newTask(taskRuntimeMs)); + } + + final List jobList = Collections.singletonList( + SLSJob.Builder.newBuilder() + .setTasks(largeJobTaskList) + .setSubmitTimeMs(0L) + .setPriority(30) + .setJobId("Job_1") + .setQueueName(QUEUE_1_NAME) + .build() + ); + + final ClusterTopologyProperties properties = createDefaultClusterTopologyProperties(); + setup(DEFAULT_RUNTIME_MINUTES, properties, new TestTraceModule(new TestTraceProvider(jobList))); + + Runner.runSimulator(simulatorModule, runnerModule); + + checkClusterMetricInvariants(ClusterMetric.CLUSTER_ALLOCATED_VCORES); + + final List jobs = readJobsFromJobsStatsFile(); + Assert.assertEquals(1, jobs.size()); + final JobStatsFileReader.JobStatsEntry oneJob = jobs.get(0); + Assert.assertEquals(ExperimentJobEndState.COMPLETED, oneJob.getExperimentJobEndState()); + Assert.assertTrue( + oneJob.getEndTimeSecondsOffset() - oneJob.getStartTimeSecondsOffset() > 2 * taskRuntimeSeconds + ); + + final Map jobMap = constructJobMap(jobList); + checkJobInvariants(jobs, jobMap); + } + + @Test + public void testMultiJobResourceContention() throws IOException { + // SLS Queue 1 should have at most 10 containers, + // based on the setting of capacity-scheduler.xml + final List jobList = new ArrayList<>(); + final int numJobs = 3; + final int numTasksPerJob = 15; + final long taskRuntimeSeconds = 60 * 10; + final long taskRuntimeMs = taskRuntimeSeconds * 1000; + + for (int jobId = 0; jobId < numJobs; jobId++) { + final SLSJob.Builder builder = SLSJob.Builder.newBuilder(); + final List taskList = new ArrayList<>(); + for (int taskId = 0; taskId < numTasksPerJob; taskId++) { + taskList.add(SLSTask.newTask(taskRuntimeMs)); + } + + jobList.add(builder + .setTasks(taskList) + .setSubmitTimeMs(0L) + .setPriority(30) + .setJobId("Job_" + jobId) + .setQueueName(QUEUE_1_NAME) + .build() + ); + } + + final ClusterTopologyProperties properties = createDefaultClusterTopologyProperties(); + setup(DEFAULT_RUNTIME_MINUTES, properties, new TestTraceModule(new TestTraceProvider(jobList))); + + Runner.runSimulator(simulatorModule, runnerModule); + + checkClusterMetricInvariants(ClusterMetric.CLUSTER_ALLOCATED_VCORES); + + final List jobs = readJobsFromJobsStatsFile(); + Assert.assertEquals(numJobs, jobs.size()); + boolean foundIncomplete = false; + for (final JobStatsFileReader.JobStatsEntry jobStatsEntry : jobs) { + if (jobStatsEntry.getExperimentJobEndState().equals(ExperimentJobEndState.SUBMITTED) || + jobStatsEntry.getExperimentJobEndState().equals(ExperimentJobEndState.STARTED)) { + foundIncomplete = true; + } + } + + Assert.assertTrue(foundIncomplete); + final AtomicBoolean queueSeenAllocation = new AtomicBoolean(false); + checkQueueMetricInvariants( + QUEUE_1_NAME, + QueueMetric.QUEUE_ALLOCATED_VCORES, + Collections.singletonList( + new MetricInvariantCheck<>( + (t, v) -> { + if (Precision.compareTo(v, 0D, 1) > 0) { + queueSeenAllocation.set(true); + } + + return true; + }, + "QueueAllocationCheck" + ) + ) + ); + + Assert.assertTrue(queueSeenAllocation.get()); + + final Map jobMap = constructJobMap(jobList); + checkJobInvariants(jobs, jobMap); + } + + @Test + public void testDTSSStressUnmanaged() throws IOException { + final List jobList = new ArrayList<>(); + + // 24 hours worth of simulation + final int experimentRuntimeMinutes = 60 * 24; + + // Job arrives with 5 minute mean Poisson + final int arrivalTimeMeanMinutes = 5; + + final int numBufferJobs = 20; + final Random random = new Random(5); + + final int numJobs = experimentRuntimeMinutes / arrivalTimeMeanMinutes + numBufferJobs; + long currMs = 0; + + for (int jobId = 0; jobId < numJobs; jobId++) { + final SLSJob.Builder builder = SLSJob.Builder.newBuilder(); + final List taskList = new ArrayList<>(); + // Create between 1 and 60 tasks + final int numJobTasks = random.nextInt(60) + 1; + // Average time for each task of the job between 1 and 10 minutes + final int taskAverageRuntimeMinutes = 1 + random.nextInt(10); + final double deviation = taskAverageRuntimeMinutes / 5.0; + for (int taskId = 0; taskId < numJobTasks; taskId++) { + final double gaussian = random.nextGaussian(); + final double taskRuntimeMinutes = Math.max( + taskAverageRuntimeMinutes + gaussian * deviation, 1.0 / 60 + ); + + final int taskRuntimeSeconds = (int) (taskRuntimeMinutes * 60); + final long taskRuntimeMs = taskRuntimeSeconds * 1000; + taskList.add(SLSTask.newTask(taskRuntimeMs)); + } + + final int queueIdxToSubmit = random.nextInt(QUEUES.length); + currMs += getPoissonRandom(random, arrivalTimeMeanMinutes) * 60 * 1000; + + jobList.add(builder + .setTasks(taskList) + .setSubmitTimeMs(currMs) + .setPriority(30) + .setJobId("Job_" + jobId) + .setQueueName(QUEUES[queueIdxToSubmit]) + .build() + ); + } + + final ClusterTopologyProperties properties = createDefaultClusterTopologyProperties(); + + setup(experimentRuntimeMinutes, properties, new TestTraceModule(new TestTraceProvider(jobList)), true); + + Runner.runSimulator(simulatorModule, runnerModule); + + checkClusterMetricInvariants(ClusterMetric.CLUSTER_ALLOCATED_VCORES); + + final List jobs = readJobsFromJobsStatsFile(); + Assert.assertEquals(numJobs, jobs.size()); + + for (final String queueName : QUEUES) { + final AtomicBoolean queueSeenAllocation = new AtomicBoolean(false); + checkQueueMetricInvariants( + queueName, + QueueMetric.QUEUE_ALLOCATED_VCORES, + Collections.singletonList( + new MetricInvariantCheck<>( + (t, v) -> { + if (Precision.compareTo(v, 0D, 1) > 0) { + queueSeenAllocation.set(true); + } + + return true; + }, + "QueueAllocationCheck" + ) + ) + ); + + Assert.assertTrue(queueSeenAllocation.get()); + } + + final Map jobMap = constructJobMap(jobList); + checkJobInvariants(jobs, jobMap); + } + + private static int getPoissonRandom(final Random r, final double mean) { + double L = Math.exp(-mean); + int k = 0; + double p = 1.0; + do { + p = p * r.nextDouble(); + k++; + } while (p > L); + return k - 1; + } +} diff --git a/hadoop-tools/hadoop-dtss/src/test/java/org/apache/hadoop/yarn/dtss/trace/SLSJobArrivalFunction.java b/hadoop-tools/hadoop-dtss/src/test/java/org/apache/hadoop/yarn/dtss/trace/SLSJobArrivalFunction.java new file mode 100644 index 00000000000..0df8fe5dca9 --- /dev/null +++ b/hadoop-tools/hadoop-dtss/src/test/java/org/apache/hadoop/yarn/dtss/trace/SLSJobArrivalFunction.java @@ -0,0 +1,17 @@ +package org.apache.hadoop.yarn.dtss.trace; + +import org.apache.hadoop.yarn.dtss.trace.sls.SLSJob; + +public abstract class SLSJobArrivalFunction { + protected long startTimeUnixTS; + protected long endTimeUnixTS; + + public SLSJobArrivalFunction( + final long startTimeUnixTS, final long endTimeUnixTS + ) { + this.startTimeUnixTS = startTimeUnixTS; + this.endTimeUnixTS = endTimeUnixTS; + } + + public abstract SLSJob getNextJob(); +} diff --git a/hadoop-tools/hadoop-dtss/src/test/java/org/apache/hadoop/yarn/dtss/trace/TestTraceModule.java b/hadoop-tools/hadoop-dtss/src/test/java/org/apache/hadoop/yarn/dtss/trace/TestTraceModule.java new file mode 100644 index 00000000000..76d16d6e126 --- /dev/null +++ b/hadoop-tools/hadoop-dtss/src/test/java/org/apache/hadoop/yarn/dtss/trace/TestTraceModule.java @@ -0,0 +1,16 @@ +package org.apache.hadoop.yarn.dtss.trace; + +import com.google.inject.AbstractModule; + +public final class TestTraceModule extends AbstractModule { + private final TestTraceProvider testTraceProvider; + public TestTraceModule(final TestTraceProvider testTraceProvider) { + this.testTraceProvider = testTraceProvider; + } + + @Override + protected void configure() { + bind(TraceReader.class).to(TestTraceReader.class); + bind(TestTraceProvider.class).toInstance(testTraceProvider); + } +} diff --git a/hadoop-tools/hadoop-dtss/src/test/java/org/apache/hadoop/yarn/dtss/trace/TestTraceProvider.java b/hadoop-tools/hadoop-dtss/src/test/java/org/apache/hadoop/yarn/dtss/trace/TestTraceProvider.java new file mode 100644 index 00000000000..2f586cc41d5 --- /dev/null +++ b/hadoop-tools/hadoop-dtss/src/test/java/org/apache/hadoop/yarn/dtss/trace/TestTraceProvider.java @@ -0,0 +1,33 @@ +package org.apache.hadoop.yarn.dtss.trace; + +import org.apache.hadoop.yarn.dtss.trace.sls.SLSJob; + +import java.util.ArrayDeque; +import java.util.List; +import java.util.Queue; + +public final class TestTraceProvider { + private final Queue jobsList = new ArrayDeque<>(); + + public TestTraceProvider( + final SLSJobArrivalFunction arrivalFunction + ) { + SLSJob slsJob = arrivalFunction.getNextJob(); + while (slsJob != null) { + jobsList.add(slsJob); + slsJob = arrivalFunction.getNextJob(); + } + } + + public TestTraceProvider(final List jobsList) { + this.jobsList.addAll(jobsList); + } + + public boolean hasNextJob() { + return jobsList.size() > 0; + } + + public SLSJob getNextJob() { + return jobsList.poll(); + } +} diff --git a/hadoop-tools/hadoop-dtss/src/test/java/org/apache/hadoop/yarn/dtss/trace/TestTraceReader.java b/hadoop-tools/hadoop-dtss/src/test/java/org/apache/hadoop/yarn/dtss/trace/TestTraceReader.java new file mode 100644 index 00000000000..01ac795cc79 --- /dev/null +++ b/hadoop-tools/hadoop-dtss/src/test/java/org/apache/hadoop/yarn/dtss/trace/TestTraceReader.java @@ -0,0 +1,84 @@ +package org.apache.hadoop.yarn.dtss.trace; + +import com.google.inject.Inject; +import com.google.inject.Injector; +import com.google.inject.Singleton; +import org.apache.hadoop.yarn.dtss.exceptions.EndOfExperimentNotificationException; +import org.apache.hadoop.yarn.dtss.job.SimulatedJob; +import org.apache.hadoop.yarn.dtss.job.SimulatedJobMaster; +import org.apache.hadoop.yarn.dtss.lifecycle.LifeCycleState; +import org.apache.hadoop.yarn.dtss.metrics.job.JobHistoryManager; +import org.apache.hadoop.yarn.dtss.time.Clock; + +import java.time.Instant; +import java.util.*; + +@Singleton +public final class TestTraceReader extends TraceReader { + private final TestTraceProvider testTraceProvider; + private final Map jobMasters = new HashMap<>(); + private long currSubmitTimeSeconds = 0L; + + @Inject + protected TestTraceReader(final Injector injector, + final Clock clock, + final JobHistoryManager jobHistoryManager, + final TestTraceProvider testTraceProvider) { + super(injector, clock, jobHistoryManager); + this.testTraceProvider = testTraceProvider; + } + + @Override + public void start() { + while (testTraceProvider.hasNextJob()) { + final SimulatedJob job = testTraceProvider.getNextJob(); + assert job.getSubmitTimeSeconds() != null; + // Time must move forward + assert currSubmitTimeSeconds <= job.getSubmitTimeSeconds(); + currSubmitTimeSeconds = job.getSubmitTimeSeconds(); + final SimulatedJobMaster jobMaster = job.createJobMaster(injector); + jobHistoryManager.onJobRead(jobMaster); + jobMasters.put(jobMaster, null); + clock.scheduleAbsoluteAlarm(Instant.ofEpochSecond(job.getSubmitTimeSeconds()), alarm -> { + // Registers the job with the RM + jobMaster.init(); + }); + } + + super.start(); + } + + @Override + public void onStop() { + for (final SimulatedJobMaster jobMaster : jobMasters.keySet()) { + if (!jobMaster.getState().isDone()) { + jobMaster.stop(new EndOfExperimentNotificationException()); + } + } + } + + @Override + public boolean areTraceJobsDone() { + if (getState().isDone()) { + return true; + } + + if (getState() != LifeCycleState.STARTED) { + return false; + } + + final List toRemove = new ArrayList<>(); + + for (final SimulatedJobMaster jobMaster : jobMasters.keySet()) { + if (jobMaster.getState().isDone()) { + toRemove.add(jobMaster); + } + } + + for (final SimulatedJobMaster jobMaster : toRemove) { + jobMasters.remove(jobMaster); + } + + return jobMasters.isEmpty(); + } +} diff --git a/hadoop-tools/hadoop-dtss/src/test/java/org/apache/hadoop/yarn/dtss/trace/results/ClusterMetric.java b/hadoop-tools/hadoop-dtss/src/test/java/org/apache/hadoop/yarn/dtss/trace/results/ClusterMetric.java new file mode 100644 index 00000000000..022a08cea85 --- /dev/null +++ b/hadoop-tools/hadoop-dtss/src/test/java/org/apache/hadoop/yarn/dtss/trace/results/ClusterMetric.java @@ -0,0 +1,29 @@ +package org.apache.hadoop.yarn.dtss.trace.results; + +public enum ClusterMetric { + CLUSTER_ALLOCATED_VCORES, + CLUSTER_AVAILABLE_VCORES, + CLUSTER_ALLOCATED_MEMORY, + CLUSTER_AVAILABLE_MEMORY, + CLUSTER_RUNNING_APPS, + CLUSTER_RUNNING_CONTAINERS; + + public static String getClusterMetricFileName(final ClusterMetric metric) { + switch (metric) { + case CLUSTER_ALLOCATED_VCORES: + return "variable.cluster.allocated.vcores.csv"; + case CLUSTER_AVAILABLE_VCORES: + return "variable.cluster.available.vcores.csv"; + case CLUSTER_ALLOCATED_MEMORY: + return "variable.cluster.allocated.memory.csv"; + case CLUSTER_AVAILABLE_MEMORY: + return "variable.cluster.available.memory.csv"; + case CLUSTER_RUNNING_APPS: + return "variable.running.application.csv"; + case CLUSTER_RUNNING_CONTAINERS: + return "variable.running.container.csv"; + default: + throw new IllegalArgumentException("Invalid metric!"); + } + } +} diff --git a/hadoop-tools/hadoop-dtss/src/test/java/org/apache/hadoop/yarn/dtss/trace/results/DetailedMetricReader.java b/hadoop-tools/hadoop-dtss/src/test/java/org/apache/hadoop/yarn/dtss/trace/results/DetailedMetricReader.java new file mode 100644 index 00000000000..35b9ac09fdd --- /dev/null +++ b/hadoop-tools/hadoop-dtss/src/test/java/org/apache/hadoop/yarn/dtss/trace/results/DetailedMetricReader.java @@ -0,0 +1,51 @@ +package org.apache.hadoop.yarn.dtss.trace.results; + +import org.apache.commons.csv.CSVFormat; +import org.apache.commons.csv.CSVParser; +import org.apache.commons.csv.CSVRecord; + +import java.io.*; +import java.util.Iterator; + +public final class DetailedMetricReader implements Closeable { + private final CSVParser parser; + private final Iterator iterator; + + public DetailedMetricReader( + final String fpath, final char delimiter, final boolean hasHeader + ) throws IOException { + final InputStreamReader inputStreamReader = new InputStreamReader(new FileInputStream(fpath)); + switch (delimiter) { + case '\t': + parser = CSVFormat.TDF.parse(inputStreamReader); + break; + case ',': + parser = CSVFormat.DEFAULT.parse(inputStreamReader); + break; + default: + throw new IllegalArgumentException("Invalid delimiter for CSV reader."); + } + + iterator = parser.iterator(); + + if (hasHeader) { + // Skip the header + iterator.next(); + } + } + + public boolean hasNextRecord() { + return iterator.hasNext(); + } + + public CSVRecord readRecord() { + return iterator.next(); + } + + @Override + public void close() throws IOException { + if (parser != null && !parser.isClosed()) { + parser.close(); + } + } +} diff --git a/hadoop-tools/hadoop-dtss/src/test/java/org/apache/hadoop/yarn/dtss/trace/results/JobStatsFileReader.java b/hadoop-tools/hadoop-dtss/src/test/java/org/apache/hadoop/yarn/dtss/trace/results/JobStatsFileReader.java new file mode 100644 index 00000000000..4aa5236084a --- /dev/null +++ b/hadoop-tools/hadoop-dtss/src/test/java/org/apache/hadoop/yarn/dtss/trace/results/JobStatsFileReader.java @@ -0,0 +1,101 @@ +package org.apache.hadoop.yarn.dtss.trace.results; + +import org.apache.hadoop.thirdparty.com.google.common.base.Strings; +import org.apache.commons.csv.CSVFormat; +import org.apache.commons.csv.CSVParser; +import org.apache.commons.csv.CSVRecord; +import org.apache.hadoop.yarn.dtss.job.ExperimentJobEndState; +import org.junit.Assert; + +import java.io.FileInputStream; +import java.io.IOException; +import java.io.InputStreamReader; +import java.util.Iterator; + +public final class JobStatsFileReader { + private static final String[] HEADER = { + "SubmitTimeSecondsOffset", + "TraceJobId", + "Queue", + "EndState", + "StartTimeSecondsOffset", + "EndTimeSecondsOffset", + "Priority" + }; + + private final CSVParser parser; + private final Iterator iterator; + + public JobStatsFileReader(final String fpath) throws IOException { + final InputStreamReader inputStreamReader = new InputStreamReader(new FileInputStream(fpath)); + parser = CSVFormat.TDF.withHeader().parse(inputStreamReader); + + iterator = parser.iterator(); + } + + public boolean hasNextRecord() { + return iterator.hasNext(); + } + + public JobStatsEntry readRecord() { + return new JobStatsEntry(iterator.next()); + } + + public static final class JobStatsEntry { + private final Long submitTimeSecondsOffset; + private final String traceJobId; + private final String queueName; + private final ExperimentJobEndState endState; + private final Long startTimeSecondsOffset; + private final Long endTimeSecondsOffset; + private final Integer priority; + + JobStatsEntry(final CSVRecord record) { + Assert.assertEquals(record.size(), HEADER.length); + + final String submitTimeSecondsOffsetStr = record.get(HEADER[0]); + submitTimeSecondsOffset = Strings.isNullOrEmpty(submitTimeSecondsOffsetStr) ? + null : Long.parseLong(submitTimeSecondsOffsetStr); + this.traceJobId = record.get(HEADER[1]); + this.queueName = record.get(HEADER[2]); + this.endState = ExperimentJobEndState.valueOf(record.get(HEADER[3])); + final String startTimeSecondsOffsetStr = record.get(HEADER[4]); + this.startTimeSecondsOffset = Strings.isNullOrEmpty(startTimeSecondsOffsetStr) ? + null : Long.parseLong(startTimeSecondsOffsetStr); + final String endTimeSecondsOffsetStr = record.get(HEADER[5]); + this.endTimeSecondsOffset = Strings.isNullOrEmpty(endTimeSecondsOffsetStr) ? + null : Long.parseLong(endTimeSecondsOffsetStr); + final String priorityStr = record.get(HEADER[6]); + this.priority = Strings.isNullOrEmpty(priorityStr) ? + null : Integer.parseInt(priorityStr); + } + + public Long getSubmitTimeSecondsOffset() { + return submitTimeSecondsOffset; + } + + public String getTraceJobId() { + return traceJobId; + } + + public String getQueueName() { + return queueName; + } + + public Long getStartTimeSecondsOffset() { + return startTimeSecondsOffset; + } + + public Long getEndTimeSecondsOffset() { + return endTimeSecondsOffset; + } + + public Integer getPriority() { + return priority; + } + + public ExperimentJobEndState getExperimentJobEndState() { + return endState; + } + } +} diff --git a/hadoop-tools/hadoop-dtss/src/test/java/org/apache/hadoop/yarn/dtss/trace/results/MetricInvariantCheck.java b/hadoop-tools/hadoop-dtss/src/test/java/org/apache/hadoop/yarn/dtss/trace/results/MetricInvariantCheck.java new file mode 100644 index 00000000000..e3121b51f6f --- /dev/null +++ b/hadoop-tools/hadoop-dtss/src/test/java/org/apache/hadoop/yarn/dtss/trace/results/MetricInvariantCheck.java @@ -0,0 +1,38 @@ +package org.apache.hadoop.yarn.dtss.trace.results; + +import java.util.function.BiFunction; + +public final class MetricInvariantCheck { + private final BiFunction checker; + private final BiFunction errorMessage; + private final String checkName; + + public MetricInvariantCheck(final BiFunction checker, + final String checkName) { + this(checker, checkName, null); + } + + public MetricInvariantCheck(final BiFunction checker, + final String checkName, + final BiFunction errorMessage) { + this.checker = checker; + this.checkName = checkName; + this.errorMessage = errorMessage; + } + + public String getCheckName() { + return checkName; + } + + public boolean evaluate(final T time, final V value) { + return checker.apply(time, value); + } + + public String getErrorMessage(final T time, final V value) { + if (errorMessage != null) { + return "Invariant with name " + checkName + " failed with message:\n" + errorMessage.apply(time, value); + } + + return "Invariant with name " + checkName + " failed at time " + time + " with value " + value; + } +} diff --git a/hadoop-tools/hadoop-dtss/src/test/java/org/apache/hadoop/yarn/dtss/trace/results/QueueMetric.java b/hadoop-tools/hadoop-dtss/src/test/java/org/apache/hadoop/yarn/dtss/trace/results/QueueMetric.java new file mode 100644 index 00000000000..0f65cab6507 --- /dev/null +++ b/hadoop-tools/hadoop-dtss/src/test/java/org/apache/hadoop/yarn/dtss/trace/results/QueueMetric.java @@ -0,0 +1,23 @@ +package org.apache.hadoop.yarn.dtss.trace.results; + +public enum QueueMetric { + QUEUE_ALLOCATED_VCORES, + QUEUE_AVAILABLE_VCORES, + QUEUE_ALLOCATED_MEMORY, + QUEUE_AVAILABLE_MEMORY; + + public static String getQueueMetricFileName(final String queueName, final QueueMetric queueMetric) { + switch (queueMetric) { + case QUEUE_ALLOCATED_VCORES: + return "variable.queue." + queueName + ".allocated.vcores.csv"; + case QUEUE_AVAILABLE_VCORES: + return "variable.queue." + queueName + ".available.vcores.csv"; + case QUEUE_ALLOCATED_MEMORY: + return "variable.queue." + queueName + ".allocated.memory.csv"; + case QUEUE_AVAILABLE_MEMORY: + return "variable.queue." + queueName + ".available.memory.csv"; + default: + throw new IllegalArgumentException("Invalid queue metric!"); + } + } +} diff --git a/hadoop-tools/hadoop-dtss/src/test/resources/capacity-scheduler.xml b/hadoop-tools/hadoop-dtss/src/test/resources/capacity-scheduler.xml new file mode 100644 index 00000000000..b663a74adb9 --- /dev/null +++ b/hadoop-tools/hadoop-dtss/src/test/resources/capacity-scheduler.xml @@ -0,0 +1,60 @@ + + + + + + + + yarn.scheduler.capacity.root.queues + sls_queue_1,sls_queue_2,sls_queue_3 + The queues at the this level (root is the root queue). + + + + + yarn.scheduler.capacity.root.sls_queue_1.capacity + 25 + + + + yarn.scheduler.capacity.root.sls_queue_1.maximum-capacity + 25 + + + + yarn.scheduler.capacity.root.sls_queue_2.capacity + 25 + + + + yarn.scheduler.capacity.root.sls_queue_2.maximum-capacity + 25 + + + + yarn.scheduler.capacity.root.sls_queue_3.capacity + 50 + + + + yarn.scheduler.capacity.root.sls_queue_3.maximum-capacity + 50 + + diff --git a/hadoop-tools/hadoop-dtss/src/test/resources/core-site.xml b/hadoop-tools/hadoop-dtss/src/test/resources/core-site.xml new file mode 100644 index 00000000000..f0d3085ef85 --- /dev/null +++ b/hadoop-tools/hadoop-dtss/src/test/resources/core-site.xml @@ -0,0 +1,25 @@ + + + + + + + + + hadoop.security.token.service.use_ip + false + + + diff --git a/hadoop-tools/hadoop-dtss/src/test/resources/yarn-site.xml b/hadoop-tools/hadoop-dtss/src/test/resources/yarn-site.xml new file mode 100644 index 00000000000..ee464ff481c --- /dev/null +++ b/hadoop-tools/hadoop-dtss/src/test/resources/yarn-site.xml @@ -0,0 +1,37 @@ + + + + + + + yarn.resourcemanager.scheduler.class + org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler + + + + yarn.resourcemanager.reservation-system.enable + false + + + + + Defines maximum application priority in a cluster. + If an application is submitted with a priority higher than this value, it will be + reset to this maximum value. + + yarn.cluster.max-application-priority + 2147483647 + + \ No newline at end of file diff --git a/hadoop-tools/hadoop-tools-dist/pom.xml b/hadoop-tools/hadoop-tools-dist/pom.xml index cc811fca694..d5ec9934480 100644 --- a/hadoop-tools/hadoop-tools-dist/pom.xml +++ b/hadoop-tools/hadoop-tools-dist/pom.xml @@ -121,6 +121,12 @@ hadoop-sls compile + + org.apache.hadoop + hadoop-dtss + compile + ${project.version} + org.apache.hadoop hadoop-resourceestimator diff --git a/hadoop-tools/pom.xml b/hadoop-tools/pom.xml index f026bc261e0..e22bc857e04 100644 --- a/hadoop-tools/pom.xml +++ b/hadoop-tools/pom.xml @@ -51,6 +51,7 @@ hadoop-azure-datalake hadoop-aliyun hadoop-fs2img + hadoop-dtss diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/EventClock.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/EventClock.java new file mode 100644 index 00000000000..e2ace974336 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/EventClock.java @@ -0,0 +1,13 @@ +package org.apache.hadoop.yarn.util; + +import java.util.concurrent.TimeUnit; + +public interface EventClock extends Clock { + void schedule(Runnable runnable, long delay, TimeUnit timeUnit); + + void scheduleWithFixedDelay(Runnable runnable, long initialDelay, long delay, TimeUnit timeUnit); + + boolean isShutdown(); + + void shutdown(); +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/ScheduledThreadPoolEventUTCClock.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/ScheduledThreadPoolEventUTCClock.java new file mode 100644 index 00000000000..e9ef9ad6896 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/ScheduledThreadPoolEventUTCClock.java @@ -0,0 +1,39 @@ +package org.apache.hadoop.yarn.util; + +import java.util.concurrent.ScheduledThreadPoolExecutor; +import java.util.concurrent.TimeUnit; + +public final class ScheduledThreadPoolEventUTCClock implements EventClock { + private final ScheduledThreadPoolExecutor executor; + private final Clock clock; + + public ScheduledThreadPoolEventUTCClock(final int poolSize) { + this.executor = new ScheduledThreadPoolExecutor(poolSize); + this.clock = new UTCClock(); + } + + @Override + public void schedule(Runnable runnable, long delay, TimeUnit timeUnit) { + executor.schedule(runnable, delay, timeUnit); + } + + @Override + public void scheduleWithFixedDelay(Runnable runnable, long initialDelay, long delay, TimeUnit timeUnit) { + executor.scheduleWithFixedDelay(runnable, initialDelay, delay, timeUnit); + } + + @Override + public boolean isShutdown() { + return executor.isShutdown(); + } + + @Override + public void shutdown() { + executor.shutdown(); + } + + @Override + public long getTime() { + return clock.getTime(); + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/pom.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/pom.xml index 485da711ccf..1430e7e5311 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/pom.xml +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/pom.xml @@ -32,6 +32,10 @@ + + net.objecthunter + exp4j + javax.servlet javax.servlet-api diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ClientRMService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ClientRMService.java index 085e9e4df0b..ce349e9994b 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ClientRMService.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ClientRMService.java @@ -693,8 +693,7 @@ public SubmitApplicationResponse submitApplication( try { // call RMAppManager to submit application directly - rmAppManager.submitApplication(submissionContext, - System.currentTimeMillis(), user); + rmAppManager.submitApplication(submissionContext, clock.getTime(), user); LOG.info("Application with id " + applicationId.getId() + " submitted by user " + user); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMAppManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMAppManager.java index 25ff384c742..d31f16d1e04 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMAppManager.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMAppManager.java @@ -77,6 +77,8 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FairScheduler; import org.apache.hadoop.yarn.server.security.ApplicationACLsManager; import org.apache.hadoop.yarn.server.utils.BuilderUtils; +import org.apache.hadoop.yarn.util.Clock; +import org.apache.hadoop.yarn.util.SystemClock; import org.apache.hadoop.yarn.util.Times; import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting; @@ -101,6 +103,7 @@ private final ApplicationMasterService masterService; private final YarnScheduler scheduler; private final ApplicationACLsManager applicationACLsManager; + private final Clock clock; private Configuration conf; private YarnAuthorizationProvider authorizer; private boolean timelineServiceV2Enabled; @@ -111,7 +114,15 @@ public RMAppManager(RMContext context, YarnScheduler scheduler, ApplicationMasterService masterService, - ApplicationACLsManager applicationACLsManager, Configuration conf) { + ApplicationACLsManager applicationACLsManager, + Configuration conf) { + this(context, scheduler, masterService, applicationACLsManager, SystemClock.getInstance(), conf); + } + + public RMAppManager(RMContext context, + YarnScheduler scheduler, ApplicationMasterService masterService, + ApplicationACLsManager applicationACLsManager, + Clock clock, Configuration conf) { this.rmContext = context; this.scheduler = scheduler; this.masterService = masterService; @@ -127,6 +138,7 @@ public RMAppManager(RMContext context, if (this.maxCompletedAppsInStateStore > this.maxCompletedAppsInMemory) { this.maxCompletedAppsInStateStore = this.maxCompletedAppsInMemory; } + this.clock = clock; this.authorizer = YarnAuthorizationProvider.getInstance(conf); this.timelineServiceV2Enabled = YarnConfiguration. timelineServiceV2Enabled(conf); @@ -782,8 +794,7 @@ public void updateApplicationPriority(UserGroupInformation callerUGI, } // Update the changed application state to timeline server - rmContext.getSystemMetricsPublisher().appUpdated(app, - System.currentTimeMillis()); + rmContext.getSystemMetricsPublisher().appUpdated(app, clock.getTime()); } /** diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.java index 6d2a9fed08b..eca4256d3e2 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.java @@ -128,6 +128,8 @@ import org.apache.hadoop.yarn.server.webproxy.ProxyUriUtils; import org.apache.hadoop.yarn.server.webproxy.WebAppProxy; import org.apache.hadoop.yarn.server.webproxy.WebAppProxyServlet; +import org.apache.hadoop.yarn.util.EventClock; +import org.apache.hadoop.yarn.util.ScheduledThreadPoolEventUTCClock; import org.apache.hadoop.yarn.webapp.WebApp; import org.apache.hadoop.yarn.webapp.WebApps; import org.apache.hadoop.yarn.webapp.WebApps.Builder; @@ -221,6 +223,7 @@ private final String zkRootNodePassword = Long.toString(new SecureRandom().nextLong()); private boolean recoveryEnabled; + protected EventClock clock = null; @VisibleForTesting protected String webAppAddress; @@ -268,6 +271,10 @@ protected ResourceProfilesManager createResourceProfileManager() { @Override protected void serviceInit(Configuration conf) throws Exception { + if (clock == null) { + clock = new ScheduledThreadPoolEventUTCClock(1); + } + this.conf = conf; UserGroupInformation.setConfiguration(conf); this.rmContext = new RMContextImpl(); @@ -500,8 +507,10 @@ protected ReservationSystem createReservationSystem() { try { Class reservationClazz = Class.forName(reservationClassName); if (ReservationSystem.class.isAssignableFrom(reservationClazz)) { - return (ReservationSystem) ReflectionUtils.newInstance( + final ReservationSystem reservationSystem = (ReservationSystem) ReflectionUtils.newInstance( reservationClazz, this.conf); + reservationSystem.setClock(clock); + return reservationSystem; } else { throw new YarnRuntimeException("Class: " + reservationClassName + " not instance of " + ReservationSystem.class.getCanonicalName()); @@ -781,7 +790,9 @@ protected void serviceInit(Configuration configuration) throws Exception { // Initialize the scheduler scheduler = createScheduler(); scheduler.setRMContext(rmContext); + scheduler.setClock(clock); addIfService(scheduler); + rmContext.setScheduler(scheduler); schedulerDispatcher = createSchedulerEventDispatcher(); @@ -1450,7 +1461,8 @@ protected ResourceTrackerService createResourceTrackerService() { protected ClientRMService createClientRMService() { return new ClientRMService(this.rmContext, scheduler, this.rmAppManager, this.applicationACLsManager, this.queueACLsManager, - this.rmContext.getRMDelegationTokenSecretManager()); + this.rmContext.getRMDelegationTokenSecretManager(), + this.clock); } protected ApplicationMasterService createApplicationMasterService() { diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/AbstractReservationSystem.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/AbstractReservationSystem.java index d9e4be9e522..fcc1e03b09c 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/AbstractReservationSystem.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/AbstractReservationSystem.java @@ -18,17 +18,6 @@ package org.apache.hadoop.yarn.server.resourcemanager.reservation; -import java.util.HashMap; -import java.util.Map; -import java.util.Map.Entry; -import java.util.Set; -import java.util.concurrent.ScheduledExecutorService; -import java.util.concurrent.ScheduledThreadPoolExecutor; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicLong; -import java.util.concurrent.locks.Lock; -import java.util.concurrent.locks.ReentrantReadWriteLock; - import org.apache.hadoop.classification.InterfaceAudience.LimitedPrivate; import org.apache.hadoop.classification.InterfaceStability.Unstable; import org.apache.hadoop.conf.Configuration; @@ -54,11 +43,21 @@ import org.apache.hadoop.yarn.server.resourcemanager.security.FairReservationsACLsManager; import org.apache.hadoop.yarn.server.resourcemanager.security.ReservationsACLsManager; import org.apache.hadoop.yarn.util.Clock; -import org.apache.hadoop.yarn.util.UTCClock; +import org.apache.hadoop.yarn.util.EventClock; import org.apache.hadoop.yarn.util.resource.ResourceCalculator; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.util.HashMap; +import java.util.Map; +import java.util.Map.Entry; +import java.util.Set; +import java.util.concurrent.ScheduledThreadPoolExecutor; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicLong; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantReadWriteLock; + /** * This is the implementation of {@link ReservationSystem} based on the * {@link ResourceScheduler} @@ -80,7 +79,7 @@ private boolean initialized = false; - private final Clock clock = new UTCClock(); + private EventClock clock; private AtomicLong resCounter = new AtomicLong(); @@ -93,8 +92,6 @@ private ResourceScheduler scheduler; - private ScheduledExecutorService scheduledExecutorService; - protected Configuration conf; protected long planStepSize; @@ -323,10 +320,19 @@ public void synchronizePlan(String planName, boolean shouldReplan) { } } + @Override + public EventClock getClock() { + return clock; + } + + @Override + public void setClock(final EventClock clock) { + this.clock = clock; + } + private void startPlanFollower(long initialDelay) { if (planFollower != null) { - scheduledExecutorService = new ScheduledThreadPoolExecutor(1); - scheduledExecutorService.scheduleWithFixedDelay(planFollower, + clock.scheduleWithFixedDelay(planFollower, initialDelay, planStepSize, TimeUnit.MILLISECONDS); } } @@ -354,9 +360,8 @@ public void serviceStart() throws Exception { @Override public void serviceStop() { // Stop the plan follower - if (scheduledExecutorService != null - && !scheduledExecutorService.isShutdown()) { - scheduledExecutorService.shutdown(); + if (!clock.isShutdown()) { + clock.shutdown(); } // Clear the plans plans.clear(); @@ -431,7 +436,7 @@ protected Plan initializePlan(String planQueueName) throws YarnException { getAgent(planQueuePath), totCap, planStepSize, rescCalc, minAllocation, maxAllocation, planQueueName, getReplanner(planQueuePath), getReservationSchedulerConfiguration().getMoveOnExpiry(planQueuePath), - maxPeriodicity, rmContext); + maxPeriodicity, rmContext, clock); LOG.info("Initialized plan {} based on reservable queue {}", plan.toString(), planQueueName); return plan; @@ -448,7 +453,7 @@ protected Planner getReplanner(String planQueueName) { if (Planner.class.isAssignableFrom(plannerClazz)) { Planner planner = (Planner) ReflectionUtils.newInstance(plannerClazz, conf); - planner.init(planQueueName, reservationConfig); + planner.init(clock, planQueueName, reservationConfig); return planner; } else { throw new YarnRuntimeException("Class: " + plannerClazz diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationSystem.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationSystem.java index a6c8fcfea74..9200c7722d9 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationSystem.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationSystem.java @@ -30,6 +30,8 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Queue; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler; import org.apache.hadoop.yarn.server.resourcemanager.security.ReservationsACLsManager; +import org.apache.hadoop.yarn.util.Clock; +import org.apache.hadoop.yarn.util.EventClock; /** * This interface is the one implemented by any system that wants to support @@ -132,4 +134,15 @@ void reinitialize(Configuration conf, RMContext rmContext) * */ ReservationsACLsManager getReservationsACLsManager(); + + /** + * Gets the {@link Clock}. + * @return clock + */ + EventClock getClock(); + + /** + * Sets the {@link EventClock}. + */ + void setClock(EventClock clock); } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/Planner.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/Planner.java index af0e7124652..9435e614e94 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/Planner.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/Planner.java @@ -24,6 +24,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.reservation.Plan; import org.apache.hadoop.yarn.server.resourcemanager.reservation.ReservationSchedulerConfiguration; import org.apache.hadoop.yarn.server.resourcemanager.reservation.exceptions.PlanningException; +import org.apache.hadoop.yarn.util.Clock; public interface Planner { @@ -42,8 +43,9 @@ public void plan(Plan plan, List contracts) /** * Initialize the replanner * + * @param clock the clock * @param planQueueName the name of the queue for this plan * @param conf the scheduler configuration */ - void init(String planQueueName, ReservationSchedulerConfiguration conf); + void init(Clock clock, String planQueueName, ReservationSchedulerConfiguration conf); } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/SimpleCapacityReplanner.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/SimpleCapacityReplanner.java index 32b912d5126..a1b8f766b99 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/SimpleCapacityReplanner.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/SimpleCapacityReplanner.java @@ -32,12 +32,12 @@ import org.apache.hadoop.yarn.server.resourcemanager.reservation.ReservationSchedulerConfiguration; import org.apache.hadoop.yarn.server.resourcemanager.reservation.exceptions.PlanningException; import org.apache.hadoop.yarn.util.Clock; -import org.apache.hadoop.yarn.util.UTCClock; import org.apache.hadoop.yarn.util.resource.ResourceCalculator; import org.apache.hadoop.yarn.util.resource.Resources; import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting; + /** * This (re)planner scan a period of time from now to a maximum time window (or * the end of the last session, whichever comes first) checking the overall @@ -53,25 +53,19 @@ private static final Resource ZERO_RESOURCE = Resource.newInstance(0, 0); - private final Clock clock; + private Clock clock; // this allows to control to time-span of this replanning // far into the future time instants might be worth replanning for // later on private long lengthOfCheckZone; - public SimpleCapacityReplanner() { - this(new UTCClock()); - } - - @VisibleForTesting - SimpleCapacityReplanner(Clock clock) { - this.clock = clock; - } - @Override - public void init(String planQueueName, + public void init( + Clock clock, + String planQueueName, ReservationSchedulerConfiguration conf) { + this.clock = clock; this.lengthOfCheckZone = conf.getEnforcementWindow(planQueueName); } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppImpl.java index 94f7bb97bfe..6d4d38be9be 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppImpl.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppImpl.java @@ -396,8 +396,7 @@ .installTopology(); - private final StateMachine - stateMachine; + private final StateMachine stateMachine; private static final int DUMMY_APPLICATION_ATTEMPT_NUMBER = -1; private static final float MINIMUM_AM_BLACKLIST_THRESHOLD_VALUE = 0.0f; @@ -411,7 +410,7 @@ public RMAppImpl(ApplicationId applicationId, RMContext rmContext, List amReqs) { this(applicationId, rmContext, config, name, user, queue, submissionContext, scheduler, masterService, submitTime, applicationType, applicationTags, - amReqs, null, -1); + amReqs, null, -1, SystemClock.getInstance()); } public RMAppImpl(ApplicationId applicationId, RMContext rmContext, @@ -419,10 +418,31 @@ public RMAppImpl(ApplicationId applicationId, RMContext rmContext, ApplicationSubmissionContext submissionContext, YarnScheduler scheduler, ApplicationMasterService masterService, long submitTime, String applicationType, Set applicationTags, - List amReqs, ApplicationPlacementContext - placementContext, long startTime) { + List amReqs, ApplicationPlacementContext placementContext, long startTime) { + this(applicationId, rmContext, config, name, user, queue, submissionContext, + scheduler, masterService, submitTime, applicationType, applicationTags, + amReqs, placementContext, startTime, SystemClock.getInstance()); + } - this.systemClock = SystemClock.getInstance(); + public RMAppImpl(ApplicationId applicationId, RMContext rmContext, + Configuration config, String name, String user, String queue, + ApplicationSubmissionContext submissionContext, YarnScheduler scheduler, + ApplicationMasterService masterService, long submitTime, + String applicationType, Set applicationTags, + List amReqs, Clock clock) { + this(applicationId, rmContext, config, name, user, queue, submissionContext, + scheduler, masterService, submitTime, applicationType, applicationTags, + amReqs, null, -1, clock); + } + + public RMAppImpl(ApplicationId applicationId, RMContext rmContext, + Configuration config, String name, String user, String queue, + ApplicationSubmissionContext submissionContext, YarnScheduler scheduler, + ApplicationMasterService masterService, long submitTime, + String applicationType, Set applicationTags, + List amReqs, ApplicationPlacementContext + placementContext, long startTime, Clock clock) { + this.systemClock = clock; this.applicationId = applicationId; this.name = StringInterner.weakIntern(name); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AbstractYarnScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AbstractYarnScheduler.java index 97e25d5cafb..b4e6216992a 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AbstractYarnScheduler.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AbstractYarnScheduler.java @@ -1026,7 +1026,7 @@ public Clock getClock() { return clock; } - @VisibleForTesting + @Override public void setClock(Clock clock) { this.clock = clock; } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/ResourceScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/ResourceScheduler.java index dcb6edd3ba3..126f9dcfe6e 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/ResourceScheduler.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/ResourceScheduler.java @@ -28,6 +28,7 @@ import org.apache.hadoop.yarn.api.records.SchedulingRequest; import org.apache.hadoop.yarn.server.resourcemanager.RMContext; import org.apache.hadoop.yarn.server.resourcemanager.recovery.Recoverable; +import org.apache.hadoop.yarn.util.Clock; /** * This interface is the one implemented by the schedulers. It mainly extends @@ -46,6 +47,13 @@ */ void setRMContext(RMContext rmContext); + /** + * Set the shared clock for ResourceScheduler + * This method should be called immediately after instantiating a scheduler once. + * @param clock the clock of the ResourceManager + */ + void setClock(Clock clock); + /** * Re-initialize the ResourceScheduler. * @param conf configuration diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacityScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacityScheduler.java index 70738c0a5a7..e5a9fb5e38d 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacityScheduler.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacityScheduler.java @@ -340,7 +340,7 @@ void initScheduler(Configuration configuration) throws this.labelManager = rmContext.getNodeLabelManager(); this.appPriorityACLManager = new AppPriorityACLsManager(conf); this.queueManager = new CapacitySchedulerQueueManager(yarnConf, - this.labelManager, this.appPriorityACLManager); + this.labelManager, this.appPriorityACLManager, getClock()); this.queueManager.setCapacitySchedulerContext(this); this.autoQueueHandler = new CapacitySchedulerAutoQueueHandler( @@ -1511,6 +1511,9 @@ private void updateLabelsOnNode(NodeId nodeId, node.updateLabels(newLabels); } + /** + * This method sets the heartbeat message for allocations. + */ private void updateSchedulerHealth(long now, NodeId nodeId, CSAssignment assignment) { List allocations = @@ -1793,6 +1796,7 @@ private CSAssignment allocateOrReserveNewContainers( SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); assignment.setSchedulingMode(SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); + submitResourceCommitRequest(getClusterResource(), assignment); if (Resources.greaterThan(calculator, getClusterResource(), diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacitySchedulerQueueManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacitySchedulerQueueManager.java index 00d1cda3088..026571d4f0e 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacitySchedulerQueueManager.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacitySchedulerQueueManager.java @@ -52,6 +52,8 @@ import org.apache.hadoop.yarn.server.resourcemanager.security.AppPriorityACLsManager; import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting; +import org.apache.hadoop.yarn.util.Clock; +import org.apache.hadoop.yarn.util.SystemClock; /** * @@ -79,6 +81,7 @@ public CSQueue hook(CSQueue queue) { private CSQueue root; private final RMNodeLabelsManager labelManager; private AppPriorityACLsManager appPriorityACLManager; + private final Clock clock; private QueueStateManager queueStateManager; @@ -91,11 +94,13 @@ public CSQueue hook(CSQueue queue) { */ public CapacitySchedulerQueueManager(Configuration conf, RMNodeLabelsManager labelManager, - AppPriorityACLsManager appPriorityACLManager) { + AppPriorityACLsManager appPriorityACLManager, + Clock clock) { this.authorizer = YarnAuthorizationProvider.getInstance(conf); this.labelManager = labelManager; this.queueStateManager = new QueueStateManager<>(); this.appPriorityACLManager = appPriorityACLManager; + this.clock = clock; } @Override @@ -257,6 +262,7 @@ static CSQueue parseQueue( throw new IllegalStateException( "Queue configuration missing child queue names for " + queueName); } + // Check if the queue will be dynamically managed by the Reservation // system if (isReservableQueue) { @@ -314,7 +320,6 @@ static CSQueue parseQueue( childQueues.add(childQueue); } parentQueue.setChildQueues(childQueues); - } newQueues.add(queue); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/LeafQueue.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/LeafQueue.java index 89e7f838003..ad18f0e812d 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/LeafQueue.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/LeafQueue.java @@ -162,6 +162,7 @@ public LeafQueue(CapacitySchedulerContext cs, // One time initialization is enough since it is static ordering policy this.pendingOrderingPolicy = new FifoOrderingPolicyForPendingApps(); + pendingOrderingPolicy.setClock(cs.getClock()); LOG.debug("LeafQueue: name={}, fullname={}", queueName, getQueuePath()); @@ -192,6 +193,7 @@ protected void setupQueueConfigs(Resource clusterResource, setOrderingPolicy( conf.getAppOrderingPolicy(getQueuePath())); + orderingPolicy.setClock(csContext.getClock()); usersManager.setUserLimit(conf.getUserLimit(getQueuePath())); usersManager.setUserLimitFactor(conf.getUserLimitFactor(getQueuePath())); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/policy/AbstractComparatorOrderingPolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/policy/AbstractComparatorOrderingPolicy.java index b5e870bf34f..ba3fb11cf37 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/policy/AbstractComparatorOrderingPolicy.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/policy/AbstractComparatorOrderingPolicy.java @@ -26,6 +26,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.*; import org.apache.hadoop.yarn.nodelabels.CommonNodeLabelsManager; +import org.apache.hadoop.yarn.util.Clock; /** @@ -37,6 +38,7 @@ private static final Logger LOG = LoggerFactory.getLogger(OrderingPolicy.class); + protected Clock clock; protected ConcurrentSkipListSet schedulableEntities; protected Comparator comparator; protected Map entitiesToReorder = new HashMap(); @@ -142,4 +144,10 @@ public abstract void containerReleased(S schedulableEntity, @Override public abstract String getConfigName(); + /** + * This method should be called once right after configuration. + */ + public void setClock(final Clock clock) { + this.clock = clock; + } } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/policy/FifoOrderingPolicyWithExclusivePartitions.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/policy/FifoOrderingPolicyWithExclusivePartitions.java index e6f6139305a..6873561ef8b 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/policy/FifoOrderingPolicyWithExclusivePartitions.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/policy/FifoOrderingPolicyWithExclusivePartitions.java @@ -25,6 +25,7 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration; import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer; +import org.apache.hadoop.yarn.util.Clock; /** @@ -37,6 +38,7 @@ private static final String DEFAULT_PARTITION = "DEFAULT_PARTITION"; + protected Clock clock; private Map> orderingPolicies; public FifoOrderingPolicyWithExclusivePartitions() { @@ -148,4 +150,11 @@ public String getConfigName() { partition : DEFAULT_PARTITION; return orderingPolicies.get(keyPartition); } + + /** + * This method should be called once right after configuration. + */ + public void setClock(final Clock clock) { + this.clock = clock; + } } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/policy/OrderingPolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/policy/OrderingPolicy.java index 5c622282b8a..6c9ca690a71 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/policy/OrderingPolicy.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/policy/OrderingPolicy.java @@ -19,7 +19,11 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler.policy; import java.util.*; + +import org.apache.hadoop.yarn.api.records.Resource; import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer; +import org.apache.hadoop.yarn.util.Clock; +import org.apache.hadoop.yarn.util.resource.ResourceCalculator; /** @@ -135,4 +139,5 @@ */ String getConfigName(); + void setClock(Clock clock); } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/TestSimpleCapacityReplanner.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/TestSimpleCapacityReplanner.java index b8521969b34..425fc5674aa 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/TestSimpleCapacityReplanner.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/TestSimpleCapacityReplanner.java @@ -69,14 +69,14 @@ public void testReplanningPlanCapacityLoss() throws PlanningException { QueueMetrics queueMetrics = mock(QueueMetrics.class); when(clock.getTime()).thenReturn(0L); - SimpleCapacityReplanner enf = new SimpleCapacityReplanner(clock); + SimpleCapacityReplanner enf = new SimpleCapacityReplanner(); RMContext context = ReservationSystemTestUtil.createMockRMContext(); ReservationSchedulerConfiguration conf = mock(ReservationSchedulerConfiguration.class); when(conf.getEnforcementWindow(any(String.class))).thenReturn(6L); - enf.init("blah", conf); + enf.init(clock, "blah", conf); // Initialize the plan with more resources InMemoryPlan plan = new InMemoryPlan(queueMetrics, policy, agent, diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCSMaxRunningAppsEnforcer.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCSMaxRunningAppsEnforcer.java index 43347c76cc9..a6dff09e0b7 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCSMaxRunningAppsEnforcer.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCSMaxRunningAppsEnforcer.java @@ -93,7 +93,7 @@ public void setup() throws IOException { .thenReturn(Resource.newInstance(0, 0)); when(scheduler.getPreemptionManager()).thenReturn(preemptionManager); queueManager = new CapacitySchedulerQueueManager(csConfig, labelManager, - appPriorityACLManager); + appPriorityACLManager, clock); queueManager.setCapacitySchedulerContext(scheduler); queueManager.initializeQueues(csConfig); }